From 4f1f535fcd3746f1cca5793fd09cc2b4eea3186b Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 26 Apr 2019 00:56:39 +0800 Subject: [PATCH 01/49] infoschema/slow_query: fix parser bug when sql contain # (#10271) --- infoschema/slow_log.go | 2 +- infoschema/slow_log_test.go | 17 +++++++++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 17203cd52e760..333d49f390ad3 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -94,7 +94,7 @@ func ParseSlowLog(tz *time.Location, scanner *bufio.Scanner) ([][]types.Datum, e if startFlag { // Parse slow log field. - if strings.Contains(line, variable.SlowLogRowPrefixStr) { + if strings.HasPrefix(line, variable.SlowLogRowPrefixStr) { line = line[len(variable.SlowLogRowPrefixStr):] fieldValues := strings.Split(line, " ") for i := 0; i < len(fieldValues)-1; i += 2 { diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index f05eb28d7077b..a619f4b532642 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -50,6 +50,23 @@ select * from t;`) } expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,select * from t;" c.Assert(expectRecordString, Equals, recordString) + + // fix sql contain '# ' bug + slowLog = bytes.NewBufferString( + `# Time: 2019-01-24-22:32:29.313255 +0800 +select a# from t; +# Time: 2019-01-24-22:32:29.313255 +0800 +# Txn_start_ts: 405888132465033227 +# Query_time: 0.216905 +# Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 +# Is_internal: true +# Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 +# Stats: t1:1,t2:2 +select * from t; +`) + scanner = bufio.NewScanner(slowLog) + _, err = infoschema.ParseSlowLog(loc, scanner) + c.Assert(err, IsNil) } func (s *testSuite) TestSlowLogParseTime(c *C) { From c2b975b217ebade4d2e8bcf29a1deecaf9a6889e Mon Sep 17 00:00:00 2001 From: lysu Date: Fri, 26 Apr 2019 10:12:45 +0800 Subject: [PATCH 02/49] server: fix `TestTLS` when use RSA-PSS (#10274) --- server/tidb_test.go | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/server/tidb_test.go b/server/tidb_test.go index c325fd33d3916..faafef9c164ae 100644 --- a/server/tidb_test.go +++ b/server/tidb_test.go @@ -210,7 +210,7 @@ func (ts *TidbTestSuite) TestSocket(c *C) { // If parentCert and parentCertKey is specified, the new certificate will be signed by the parentCert. // Otherwise, the new certificate will be self-signed and is a CA. func generateCert(sn int, commonName string, parentCert *x509.Certificate, parentCertKey *rsa.PrivateKey, outKeyFile string, outCertFile string) (*x509.Certificate, *rsa.PrivateKey, error) { - privateKey, err := rsa.GenerateKey(rand.Reader, 512) + privateKey, err := rsa.GenerateKey(rand.Reader, 528) if err != nil { return nil, nil, errors.Trace(err) } @@ -295,11 +295,6 @@ func registerTLSConfig(configName string, caCertPath string, clientCertPath stri } func (ts *TidbTestSuite) TestTLS(c *C) { - preEnv := os.Getenv("GODEBUG") - os.Setenv("GODEBUG", "tls13=0") - defer func() { - os.Setenv("GODEBUG", preEnv) - }() // Generate valid TLS certificates. caCert, caKey, err := generateCert(0, "TiDB CA", nil, nil, "/tmp/ca-key.pem", "/tmp/ca-cert.pem") c.Assert(err, IsNil) From 7611a7dae8d9d1779b9ae3f88e224b33557a92c1 Mon Sep 17 00:00:00 2001 From: crazycs Date: Fri, 26 Apr 2019 12:49:36 +0800 Subject: [PATCH 03/49] *: add more cop task and memory info to slow_query table (#10264) --- infoschema/slow_log.go | 107 ++++++++++++++++++++++++++++++------ infoschema/slow_log_test.go | 5 +- infoschema/tables_test.go | 8 ++- 3 files changed, 100 insertions(+), 20 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 333d49f390ad3..93d86525e9207 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -49,6 +49,15 @@ var slowQueryCols = []columnInfo{ {variable.SlowLogIsInternalStr, mysql.TypeTiny, 1, 0, nil, nil}, {variable.SlowLogDigestStr, mysql.TypeVarchar, 64, 0, nil, nil}, {variable.SlowLogStatsInfoStr, mysql.TypeVarchar, 512, 0, nil, nil}, + {variable.SlowLogCopProcAvg, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopProcP90, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopProcMax, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopProcAddr, mysql.TypeVarchar, 64, 0, nil, nil}, + {variable.SlowLogCopWaitAvg, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopWaitP90, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopWaitMax, mysql.TypeDouble, 22, 0, nil, nil}, + {variable.SlowLogCopWaitAddr, mysql.TypeVarchar, 64, 0, nil, nil}, + {variable.SlowLogMemMax, mysql.TypeLonglong, 20, 0, nil, nil}, {variable.SlowLogQuerySQLStr, mysql.TypeVarchar, 4096, 0, nil, nil}, } @@ -125,23 +134,32 @@ func ParseSlowLog(tz *time.Location, scanner *bufio.Scanner) ([][]types.Datum, e } type slowQueryTuple struct { - time time.Time - txnStartTs uint64 - user string - connID uint64 - queryTime float64 - processTime float64 - waitTime float64 - backOffTime float64 - requestCount uint64 - totalKeys uint64 - processKeys uint64 - db string - indexIDs string - isInternal bool - digest string - statsInfo string - sql string + time time.Time + txnStartTs uint64 + user string + connID uint64 + queryTime float64 + processTime float64 + waitTime float64 + backOffTime float64 + requestCount uint64 + totalKeys uint64 + processKeys uint64 + db string + indexIDs string + isInternal bool + digest string + statsInfo string + avgProcessTime float64 + p90ProcessTime float64 + maxProcessTime float64 + maxProcessAddress string + avgWaitTime float64 + p90WaitTime float64 + maxWaitTime float64 + maxWaitAddress string + memMax int64 + sql string } func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) error { @@ -221,6 +239,52 @@ func (st *slowQueryTuple) setFieldValue(tz *time.Location, field, value string) st.digest = value case variable.SlowLogStatsInfoStr: st.statsInfo = value + case variable.SlowLogCopProcAvg: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.avgProcessTime = num + case variable.SlowLogCopProcP90: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.p90ProcessTime = num + case variable.SlowLogCopProcMax: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.maxProcessTime = num + case variable.SlowLogCopProcAddr: + st.maxProcessAddress = value + case variable.SlowLogCopWaitAvg: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.avgWaitTime = num + case variable.SlowLogCopWaitP90: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.p90WaitTime = num + case variable.SlowLogCopWaitMax: + num, err := strconv.ParseFloat(value, 64) + if err != nil { + return errors.AddStack(err) + } + st.maxWaitTime = num + case variable.SlowLogCopWaitAddr: + st.maxWaitAddress = value + case variable.SlowLogMemMax: + num, err := strconv.ParseInt(value, 10, 64) + if err != nil { + return errors.AddStack(err) + } + st.memMax = num case variable.SlowLogQuerySQLStr: st.sql = value } @@ -249,6 +313,15 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { record = append(record, types.NewDatum(st.isInternal)) record = append(record, types.NewStringDatum(st.digest)) record = append(record, types.NewStringDatum(st.statsInfo)) + record = append(record, types.NewFloat64Datum(st.avgProcessTime)) + record = append(record, types.NewFloat64Datum(st.p90ProcessTime)) + record = append(record, types.NewFloat64Datum(st.maxProcessTime)) + record = append(record, types.NewStringDatum(st.maxProcessAddress)) + record = append(record, types.NewFloat64Datum(st.avgWaitTime)) + record = append(record, types.NewFloat64Datum(st.p90WaitTime)) + record = append(record, types.NewFloat64Datum(st.maxWaitTime)) + record = append(record, types.NewStringDatum(st.maxWaitAddress)) + record = append(record, types.NewIntDatum(st.memMax)) record = append(record, types.NewStringDatum(st.sql)) return record } diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index a619f4b532642..8a839ae89cf75 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -32,6 +32,9 @@ func (s *testSuite) TestParseSlowLogFile(c *C) { # Is_internal: true # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 # Stats: t1:1,t2:2 +# Cop_proc_avg: 0.1 Cop_proc_p90: 0.2 Cop_proc_max: 0.03 Cop_proc_addr: 127.0.0.1:20160 +# Cop_wait_avg: 0.05 Cop_wait_p90: 0.6 Cop_wait_max: 0.8 Cop_wait_addr: 0.0.0.0:20160 +# Mem_max: 70724 select * from t;`) scanner := bufio.NewScanner(slowLog) loc, err := time.LoadLocation("Asia/Shanghai") @@ -48,7 +51,7 @@ select * from t;`) } recordString += str } - expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,select * from t;" + expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,select * from t;" c.Assert(expectRecordString, Equals, recordString) // fix sql contain '# ' bug diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index d01d69b20fb0f..b0e0f2f52ad6d 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -341,6 +341,9 @@ func (s *testTableSuite) TestSlowQuery(c *C) { # Is_internal: false # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772 # Stats: t1:1,t2:2 +# Cop_proc_avg: 0.1 Cop_proc_p90: 0.2 Cop_proc_max: 0.03 Cop_proc_addr: 127.0.0.1:20160 +# Cop_wait_avg: 0.05 Cop_wait_p90: 0.6 Cop_wait_max: 0.8 Cop_wait_addr: 0.0.0.0:20160 +# Mem_max: 70724 select * from t_slim;`)) c.Assert(f.Close(), IsNil) c.Assert(err, IsNil) @@ -348,10 +351,11 @@ select * from t_slim;`)) tk.MustExec(fmt.Sprintf("set @@tidb_slow_query_file='%v'", slowLogFileName)) tk.MustExec("set time_zone = '+08:00';") re := tk.MustQuery("select * from information_schema.slow_query") - re.Check(testutil.RowsWithSep("|", "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", + "2019-02-12 19:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|select * from t_slim;")) tk.MustExec("set time_zone = '+00:00';") re = tk.MustQuery("select * from information_schema.slow_query") - re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|select * from t_slim;")) + re.Check(testutil.RowsWithSep("|", "2019-02-12 11:33:56.571953|406315658548871171|root@127.0.0.1|6|4.895492|0.161|0.101|0.092|1|100001|100000|test||0|42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772|t1:1,t2:2|0.1|0.2|0.03|127.0.0.1:20160|0.05|0.6|0.8|0.0.0.0:20160|70724|select * from t_slim;")) } func (s *testTableSuite) TestForAnalyzeStatus(c *C) { From 5f03462166c976ca2a229f00b8ae5e2c6c102fbd Mon Sep 17 00:00:00 2001 From: crazycs Date: Sun, 28 Apr 2019 10:52:04 +0800 Subject: [PATCH 04/49] ddl: add comment for onRebaseAutoID (#10206) --- ddl/ddl_api.go | 5 +++++ ddl/table.go | 1 + 2 files changed, 6 insertions(+) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d49b228c46677..d8e96e056d206 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1807,6 +1807,11 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 if err != nil { return errors.Trace(err) } + // If newBase < autoIncID, we need to do a rebase before returning. + // Assume there are 2 TiDB servers: TiDB-A with allocator range of 0 ~ 30000; TiDB-B with allocator range of 30001 ~ 60000. + // If the user sends SQL `alter table t1 auto_increment = 100` to TiDB-B, + // and TiDB-B finds 100 < 30001 but returns without any handling, + // then TiDB-A may still allocate 99 for auto_increment column. This doesn't make sense for the user. newBase = mathutil.MaxInt64(newBase, autoIncID) job := &model.Job{ SchemaID: schema.ID, diff --git a/ddl/table.go b/ddl/table.go index 225a11208a62b..cec8c7ca07a8e 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -535,6 +535,7 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, job.State = model.JobStateCancelled return ver, errors.Trace(err) } + // No need to check `newBase` again, because `RebaseAutoID` will do this check. tblInfo.AutoIncID = newBase tbl, err := getTable(store, schemaID, tblInfo) if err != nil { From 9bf22531d0ed5b8e3173c3742a1f5fd78514271b Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Sun, 28 Apr 2019 11:48:16 +0800 Subject: [PATCH 05/49] statistics: fix panic when move analyze jobs to history (#10286) --- statistics/analyze_jobs.go | 2 +- statistics/analyze_jobs_test.go | 34 +++++++++++++++++++++++++++++++++ 2 files changed, 35 insertions(+), 1 deletion(-) create mode 100644 statistics/analyze_jobs_test.go diff --git a/statistics/analyze_jobs.go b/statistics/analyze_jobs.go index b6910aa4a77f0..ff1fedfcf6464 100644 --- a/statistics/analyze_jobs.go +++ b/statistics/analyze_jobs.go @@ -63,7 +63,7 @@ func MoveToHistory(job *AnalyzeJob) { analyzeStatus.Lock() delete(analyzeStatus.jobs, job) analyzeStatus.history = append(analyzeStatus.history, job) - numJobs := len(analyzeStatus.jobs) + numJobs := len(analyzeStatus.history) if numJobs > numMaxHistoryJobs { analyzeStatus.history = analyzeStatus.history[numJobs-numMaxHistoryJobs:] } diff --git a/statistics/analyze_jobs_test.go b/statistics/analyze_jobs_test.go new file mode 100644 index 0000000000000..0c91672ea9bc6 --- /dev/null +++ b/statistics/analyze_jobs_test.go @@ -0,0 +1,34 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package statistics + +import ( + . "github.com/pingcap/check" +) + +func (s *testStatisticsSuite) TestMoveToHistory(c *C) { + numJobs := numMaxHistoryJobs*2 + 1 + jobs := make([]*AnalyzeJob, 0, numJobs) + for i := 0; i < numJobs; i++ { + job := &AnalyzeJob{} + AddNewAnalyzeJob(job) + jobs = append(jobs, job) + } + MoveToHistory(jobs[0]) + c.Assert(len(GetAllAnalyzeJobs()), Equals, numJobs) + for i := 1; i < numJobs; i++ { + MoveToHistory(jobs[i]) + } + c.Assert(len(GetAllAnalyzeJobs()), Equals, numMaxHistoryJobs) +} From 3f89a6e64d28cbdda7d97bfcc5d85708ad3bd8b1 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Sun, 28 Apr 2019 12:05:29 +0800 Subject: [PATCH 06/49] store/helper, infoschema: add mem table for tikv store status (#10248) --- infoschema/tables.go | 85 +++++++++++++++++++++++++++++++++++++ store/helper/helper.go | 80 ++++++++++++++++++++++++++++++++++ store/helper/helper_test.go | 71 ++++++++++++++++++++++++++++++- util/pdapi/const.go | 1 + 4 files changed, 236 insertions(+), 1 deletion(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index 67646bd64a684..faa83fd22570d 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -14,6 +14,7 @@ package infoschema import ( + "encoding/json" "fmt" "sort" "sync" @@ -33,6 +34,7 @@ import ( "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" + binaryJson "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/pdapi" "github.com/pingcap/tidb/util/sqlexec" ) @@ -73,6 +75,7 @@ const ( tableTiDBIndexes = "TIDB_INDEXES" tableSlowLog = "SLOW_QUERY" tableTiDBHotRegions = "TIDB_HOT_REGIONS" + tableTiKVStoreStatus = "TIKV_STORE_STATUS" tableAnalyzeStatus = "ANALYZE_STATUS" ) @@ -563,6 +566,28 @@ var tableTiDBHotRegionsCols = []columnInfo{ {"FLOW_BYTES", mysql.TypeLonglong, 21, 0, nil, nil}, } +var tableTiKVStoreStatusCols = []columnInfo{ + {"STORE_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"ADDRESS", mysql.TypeVarchar, 64, 0, nil, nil}, + {"STORE_STATE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"STORE_STATE_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, + {"LABEL", mysql.TypeJSON, 51, 0, nil, nil}, + {"VERSION", mysql.TypeVarchar, 64, 0, nil, nil}, + {"CAPACITY", mysql.TypeVarchar, 64, 0, nil, nil}, + {"AVAILABLE", mysql.TypeVarchar, 64, 0, nil, nil}, + {"LEADER_COUNT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_WEIGHT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_SCORE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"LEADER_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_COUNT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_WEIGHT", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_SCORE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"REGION_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"START_TS", mysql.TypeDatetime, 0, 0, nil, nil}, + {"LAST_HEARTBEAT_TS", mysql.TypeDatetime, 0, 0, nil, nil}, + {"UPTIME", mysql.TypeVarchar, 64, 0, nil, nil}, +} + var tableAnalyzeStatusCols = []columnInfo{ {"TABLE_SCHEMA", mysql.TypeVarchar, 64, 0, nil, nil}, {"TABLE_NAME", mysql.TypeVarchar, 64, 0, nil, nil}, @@ -573,6 +598,63 @@ var tableAnalyzeStatusCols = []columnInfo{ {"STATE", mysql.TypeVarchar, 64, 0, nil, nil}, } +func dataForTiKVStoreStatus(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about TiKV store status can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + storesStat, err := tikvHelper.GetStoresStat() + if err != nil { + return nil, err + } + for _, storeStat := range storesStat.Stores { + row := make([]types.Datum, len(tableTiKVStoreStatusCols)) + row[0].SetInt64(storeStat.Store.ID) + row[1].SetString(storeStat.Store.Address) + row[2].SetInt64(storeStat.Store.State) + row[3].SetString(storeStat.Store.StateName) + data, err := json.Marshal(storeStat.Store.Labels) + if err != nil { + return nil, err + } + bj := binaryJson.BinaryJSON{} + if err = bj.UnmarshalJSON(data); err != nil { + return nil, err + } + row[4].SetMysqlJSON(bj) + row[5].SetString(storeStat.Store.Version) + row[6].SetString(storeStat.Status.Capacity) + row[7].SetString(storeStat.Status.Available) + row[8].SetInt64(storeStat.Status.LeaderCount) + row[9].SetInt64(storeStat.Status.LeaderWeight) + row[10].SetInt64(storeStat.Status.LeaderScore) + row[11].SetInt64(storeStat.Status.LeaderSize) + row[12].SetInt64(storeStat.Status.RegionCount) + row[13].SetInt64(storeStat.Status.RegionWeight) + row[14].SetInt64(storeStat.Status.RegionScore) + row[15].SetInt64(storeStat.Status.RegionSize) + startTs := types.Time{ + Time: types.FromGoTime(storeStat.Status.StartTs), + Type: mysql.TypeDatetime, + Fsp: types.DefaultFsp, + } + row[16].SetMysqlTime(startTs) + lastHeartbeatTs := types.Time{ + Time: types.FromGoTime(storeStat.Status.LastHeartbeatTs), + Type: mysql.TypeDatetime, + Fsp: types.DefaultFsp, + } + row[17].SetMysqlTime(lastHeartbeatTs) + row[18].SetString(storeStat.Status.Uptime) + records = append(records, row) + } + return records, nil +} + func dataForCharacterSets() (records [][]types.Datum) { charsets := charset.GetSupportedCharsets() @@ -1552,6 +1634,7 @@ var tableNameToColumns = map[string][]columnInfo{ tableTiDBIndexes: tableTiDBIndexesCols, tableSlowLog: slowQueryCols, tableTiDBHotRegions: tableTiDBHotRegionsCols, + tableTiKVStoreStatus: tableTiKVStoreStatusCols, tableAnalyzeStatus: tableAnalyzeStatusCols, } @@ -1648,6 +1731,8 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) fullRows, err = dataForSlowLog(ctx) case tableTiDBHotRegions: fullRows, err = dataForTiDBHotRegions(ctx) + case tableTiKVStoreStatus: + fullRows, err = dataForTiKVStoreStatus(ctx) case tableAnalyzeStatus: fullRows = DataForAnalyzeStatus() } diff --git a/store/helper/helper.go b/store/helper/helper.go index 392a67404704b..d9f05f86c15d2 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/logutil" + "github.com/pingcap/tidb/util/pdapi" "go.uber.org/zap" ) @@ -365,3 +366,82 @@ func (r *RegionFrameRange) GetIndexFrame(tableID, indexID int64, dbName, tableNa } return nil } + +// StoresStat stores all information get from PD's api. +type StoresStat struct { + Count int `json:"count"` + Stores []StoreStat `json:"stores"` +} + +// StoreStat stores information of one store. +type StoreStat struct { + Store StoreBaseStat `json:"store"` + Status StoreDetailStat `json:"status"` +} + +// StoreBaseStat stores the basic information of one store. +type StoreBaseStat struct { + ID int64 `json:"id"` + Address string `json:"address"` + State int64 `json:"state"` + StateName string `json:"state_name"` + Version string `json:"version"` + Labels []StoreLabel `json:"labels"` +} + +// StoreLabel stores the information of one store label. +type StoreLabel struct { + Key string `json:"key"` + Value string `json:"value"` +} + +// StoreDetailStat stores the detail information of one store. +type StoreDetailStat struct { + Capacity string `json:"capacity"` + Available string `json:"available"` + LeaderCount int64 `json:"leader_count"` + LeaderWeight int64 `json:"leader_weight"` + LeaderScore int64 `json:"leader_score"` + LeaderSize int64 `json:"leader_size"` + RegionCount int64 `json:"region_count"` + RegionWeight int64 `json:"region_weight"` + RegionScore int64 `json:"region_score"` + RegionSize int64 `json:"region_size"` + StartTs time.Time `json:"start_ts"` + LastHeartbeatTs time.Time `json:"last_heartbeat_ts"` + Uptime string `json:"uptime"` +} + +// GetStoresStat gets the TiKV store information by accessing PD's api. +func (h *Helper) GetStoresStat() (*StoresStat, error) { + etcd, ok := h.Store.(tikv.EtcdBackend) + if !ok { + return nil, errors.WithStack(errors.New("not implemented")) + } + pdHosts := etcd.EtcdAddrs() + if len(pdHosts) == 0 { + return nil, errors.New("pd unavailable") + } + req, err := http.NewRequest("GET", protocol+pdHosts[0]+pdapi.Stores, nil) + if err != nil { + return nil, errors.Trace(err) + } + timeout, cancelFunc := context.WithTimeout(context.Background(), 50*time.Millisecond) + resp, err := http.DefaultClient.Do(req.WithContext(timeout)) + defer cancelFunc() + if err != nil { + return nil, errors.Trace(err) + } + defer func() { + err = resp.Body.Close() + if err != nil { + logutil.Logger(context.Background()).Error("close body failed", zap.Error(err)) + } + }() + var storesStat StoresStat + err = json.NewDecoder(resp.Body).Decode(&storesStat) + if err != nil { + return nil, errors.Trace(err) + } + return &storesStat, nil +} diff --git a/store/helper/helper_test.go b/store/helper/helper_test.go index db98ede717a00..7343ce99e5add 100644 --- a/store/helper/helper_test.go +++ b/store/helper/helper_test.go @@ -82,9 +82,22 @@ func (s *HelperTestSuite) TestHotRegion(c *C) { c.Assert(fmt.Sprintf("%v", regionMetric), Equals, "map[1:{100 1 0}]") } +func (s *HelperTestSuite) TestTiKVStoresStat(c *C) { + h := helper.Helper{ + Store: s.store, + RegionCache: s.store.GetRegionCache(), + } + stat, err := h.GetStoresStat() + c.Assert(err, IsNil, Commentf("err: %+v", err)) + data, err := json.Marshal(stat) + c.Assert(err, IsNil) + c.Assert(fmt.Sprintf("%s", data), Equals, "{\"count\":1,\"stores\":[{\"store\":{\"id\":1,\"address\":\"127.0.0.1:20160\",\"state\":0,\"state_name\":\"Up\",\"version\":\"3.0.0-beta\",\"labels\":[{\"key\":\"test\",\"value\":\"test\"}]},\"status\":{\"capacity\":\"60 GiB\",\"available\":\"100 GiB\",\"leader_count\":10,\"leader_weight\":1,\"leader_score\":1000,\"leader_size\":1000,\"region_count\":200,\"region_weight\":1,\"region_score\":1000,\"region_size\":1000,\"start_ts\":\"2019-04-23T19:30:30+08:00\",\"last_heartbeat_ts\":\"2019-04-23T19:31:30+08:00\",\"uptime\":\"1h30m\"}}]}") +} + func (s *HelperTestSuite) mockPDHTTPServer(c *C) { router := mux.NewRouter() - router.HandleFunc("/pd/api/v1/hotspot/regions/read", s.mockHotRegionResponse) + router.HandleFunc(pdapi.HotRead, s.mockHotRegionResponse) + router.HandleFunc(pdapi.Stores, s.mockStoreStatResponse) serverMux := http.NewServeMux() serverMux.Handle("/", router) server := &http.Server{Addr: "127.0.0.1:10100", Handler: serverMux} @@ -118,3 +131,59 @@ func (s *HelperTestSuite) mockHotRegionResponse(w http.ResponseWriter, req *http } } + +func (s *HelperTestSuite) mockStoreStatResponse(w http.ResponseWriter, req *http.Request) { + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(http.StatusOK) + startTs, err := time.Parse(time.RFC3339, "2019-04-23T19:30:30+08:00") + if err != nil { + log.Panic("mock tikv store api response failed", zap.Error(err)) + } + lastHeartbeatTs, err := time.Parse(time.RFC3339, "2019-04-23T19:31:30+08:00") + if err != nil { + log.Panic("mock tikv store api response failed", zap.Error(err)) + } + storesStat := helper.StoresStat{ + Count: 1, + Stores: []helper.StoreStat{ + { + Store: helper.StoreBaseStat{ + ID: 1, + Address: "127.0.0.1:20160", + State: 0, + StateName: "Up", + Version: "3.0.0-beta", + Labels: []helper.StoreLabel{ + { + Key: "test", + Value: "test", + }, + }, + }, + Status: helper.StoreDetailStat{ + Capacity: "60 GiB", + Available: "100 GiB", + LeaderCount: 10, + LeaderWeight: 1, + LeaderScore: 1000, + LeaderSize: 1000, + RegionCount: 200, + RegionWeight: 1, + RegionScore: 1000, + RegionSize: 1000, + StartTs: startTs, + LastHeartbeatTs: lastHeartbeatTs, + Uptime: "1h30m", + }, + }, + }, + } + data, err := json.MarshalIndent(storesStat, "", " ") + if err != nil { + log.Panic("json marshal failed", zap.Error(err)) + } + _, err = w.Write(data) + if err != nil { + log.Panic("write http response failed", zap.Error(err)) + } +} diff --git a/util/pdapi/const.go b/util/pdapi/const.go index 1cf4d935ff4d0..086d55f99a781 100644 --- a/util/pdapi/const.go +++ b/util/pdapi/const.go @@ -17,4 +17,5 @@ package pdapi const ( HotRead = "/pd/api/v1/hotspot/regions/read" HotWrite = "/pd/api/v1/hotspot/regions/read" + Stores = "/pd/api/v1/stores" ) From 9630d57c8007a0968ef0672a342e27ed8aebe106 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Sun, 28 Apr 2019 13:08:46 +0800 Subject: [PATCH 07/49] planner/core: make join reorder by dp work (#8816) --- expression/util.go | 13 ++ expression/util_test.go | 14 ++ planner/core/expression_rewriter.go | 2 +- planner/core/logical_plan_builder.go | 2 +- planner/core/logical_plan_test.go | 2 +- planner/core/optimizer.go | 2 +- planner/core/rule_join_reorder.go | 29 ++-- planner/core/rule_join_reorder_dp.go | 170 +++++++++++++++++----- planner/core/rule_join_reorder_dp_test.go | 35 +++-- planner/core/rule_join_reorder_greedy.go | 15 +- sessionctx/variable/session.go | 55 ++++--- sessionctx/variable/sysvar.go | 1 + sessionctx/variable/tidb_vars.go | 5 + sessionctx/variable/varsutil.go | 8 + sessionctx/variable/varsutil_test.go | 9 ++ 15 files changed, 265 insertions(+), 97 deletions(-) diff --git a/expression/util.go b/expression/util.go index d11f3b45a54b8..881a8baa19a9b 100644 --- a/expression/util.go +++ b/expression/util.go @@ -44,6 +44,19 @@ func Filter(result []Expression, input []Expression, filter func(Expression) boo return result } +// FilterOutInPlace do the filtering out in place. +// The remained are the ones who doesn't match the filter, storing in the original slice. +// The filteredOut are the ones match the filter, storing in a new slice. +func FilterOutInPlace(input []Expression, filter func(Expression) bool) (remained, filteredOut []Expression) { + for i := len(input) - 1; i >= 0; i-- { + if filter(input[i]) { + filteredOut = append(filteredOut, input[i]) + input = append(input[:i], input[i+1:]...) + } + } + return input, filteredOut +} + // ExtractColumns extracts all columns from an expression. func ExtractColumns(expr Expression) (cols []*Column) { // Pre-allocate a slice to reduce allocation, 8 doesn't have special meaning. diff --git a/expression/util_test.go b/expression/util_test.go index 4aaef213bb483..1d9f9b936860f 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -83,6 +83,20 @@ func (s *testUtilSuite) TestFilter(c *check.C) { c.Assert(result, check.HasLen, 1) } +func (s *testUtilSuite) TestFilterOutInPlace(c *check.C) { + conditions := []Expression{ + newFunction(ast.EQ, newColumn(0), newColumn(1)), + newFunction(ast.EQ, newColumn(1), newColumn(2)), + newFunction(ast.LogicOr, newLonglong(1), newColumn(0)), + } + remained, filtered := FilterOutInPlace(conditions, isLogicOrFunction) + c.Assert(len(remained), check.Equals, 2) + c.Assert(remained[0].(*ScalarFunction).FuncName.L, check.Equals, "eq") + c.Assert(remained[1].(*ScalarFunction).FuncName.L, check.Equals, "eq") + c.Assert(len(filtered), check.Equals, 1) + c.Assert(filtered[0].(*ScalarFunction).FuncName.L, check.Equals, "or") +} + func isLogicOrFunction(e Expression) bool { if f, ok := e.(*ScalarFunction); ok { return f.FuncName.L == ast.LogicOr diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index d5e163d1e931b..0f36b2bc32eb2 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -710,7 +710,7 @@ func (er *expressionRewriter) handleInSubquery(v *ast.PatternInExpr) (ast.Node, // We need to try to eliminate the agg and the projection produced by this operation. er.b.optFlag |= flagEliminateAgg er.b.optFlag |= flagEliminateProjection - er.b.optFlag |= flagJoinReOrderGreedy + er.b.optFlag |= flagJoinReOrder // Build distinct for the inner query. agg := er.b.buildDistinct(np, np.Schema().Len()) for _, col := range agg.schema.Columns { diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index d00f28d561647..b9dcec36df240 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -390,7 +390,7 @@ func (b *PlanBuilder) buildJoin(joinNode *ast.Join) (LogicalPlan, error) { joinPlan.JoinType = RightOuterJoin resetNotNullFlag(joinPlan.schema, 0, leftPlan.Schema().Len()) default: - b.optFlag = b.optFlag | flagJoinReOrderGreedy + b.optFlag = b.optFlag | flagJoinReOrder joinPlan.JoinType = InnerJoin } diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index a1e795bf59ed9..578155d266563 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -916,7 +916,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { p, err := BuildLogicalPlan(s.ctx, stmt, s.is) c.Assert(err, IsNil) - p, err = logicalOptimize(flagPredicatePushDown|flagJoinReOrderGreedy, p.(LogicalPlan)) + p, err = logicalOptimize(flagPredicatePushDown|flagJoinReOrder, p.(LogicalPlan)) c.Assert(err, IsNil) c.Assert(ToString(p), Equals, tt.best, Commentf("for %s", tt.sql)) } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index 1baf596854a66..cdee37b93fef2 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -44,7 +44,7 @@ const ( flagPartitionProcessor flagPushDownAgg flagPushDownTopN - flagJoinReOrderGreedy + flagJoinReOrder ) var optRuleList = []logicalOptRule{ diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 8e5a1c4f4d7d2..fac63d725cbb5 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -71,11 +71,19 @@ func (s *joinReOrderSolver) optimizeRecursive(ctx sessionctx.Context, p LogicalP ctx: ctx, otherConds: otherConds, } - groupSolver := &joinReorderGreedySingleGroupSolver{ - baseSingleGroupJoinOrderSolver: baseGroupSolver, - eqEdges: eqEdges, + if len(curJoinGroup) > ctx.GetSessionVars().TiDBOptJoinReorderThreshold { + groupSolver := &joinReorderGreedySolver{ + baseSingleGroupJoinOrderSolver: baseGroupSolver, + eqEdges: eqEdges, + } + p, err = groupSolver.solve(curJoinGroup) + } else { + dpSolver := &joinReorderDPSolver{ + baseSingleGroupJoinOrderSolver: baseGroupSolver, + } + dpSolver.newJoin = dpSolver.newJoinWithEdges + p, err = dpSolver.solve(curJoinGroup, expression.ScalarFuncs2Exprs(eqEdges)) } - p, err = groupSolver.solve(curJoinGroup) if err != nil { return nil, err } @@ -143,22 +151,15 @@ func (s *baseSingleGroupJoinOrderSolver) newCartesianJoin(lChild, rChild Logical return join } -func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(eqEdges []*expression.ScalarFunction, remainedOtherConds []expression.Expression, - lChild, rChild LogicalPlan) (*LogicalJoin, []expression.Expression) { +func (s *baseSingleGroupJoinOrderSolver) newJoinWithEdges(lChild, rChild LogicalPlan, eqEdges []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan { newJoin := s.newCartesianJoin(lChild, rChild) newJoin.EqualConditions = eqEdges + newJoin.OtherConditions = otherConds for _, eqCond := range newJoin.EqualConditions { newJoin.LeftJoinKeys = append(newJoin.LeftJoinKeys, eqCond.GetArgs()[0].(*expression.Column)) newJoin.RightJoinKeys = append(newJoin.RightJoinKeys, eqCond.GetArgs()[1].(*expression.Column)) } - for i := len(remainedOtherConds) - 1; i >= 0; i-- { - cols := expression.ExtractColumns(remainedOtherConds[i]) - if newJoin.schema.ColumnsIndices(cols) != nil { - newJoin.OtherConditions = append(newJoin.OtherConditions, remainedOtherConds[i]) - remainedOtherConds = append(remainedOtherConds[:i], remainedOtherConds[i+1:]...) - } - } - return newJoin, remainedOtherConds + return newJoin } // calcJoinCumCost calculates the cumulative cost of the join node. diff --git a/planner/core/rule_join_reorder_dp.go b/planner/core/rule_join_reorder_dp.go index f4820fe7380fd..18b549d7813c8 100644 --- a/planner/core/rule_join_reorder_dp.go +++ b/planner/core/rule_join_reorder_dp.go @@ -18,24 +18,39 @@ import ( "github.com/pingcap/parser/ast" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/sessionctx" ) type joinReorderDPSolver struct { - ctx sessionctx.Context - newJoin func(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction) LogicalPlan + *baseSingleGroupJoinOrderSolver + newJoin func(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan } -type joinGroupEdge struct { +type joinGroupEqEdge struct { nodeIDs []int edge *expression.ScalarFunction } -func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression.Expression) (LogicalPlan, error) { - adjacents := make([][]int, len(joinGroup)) - totalEdges := make([]joinGroupEdge, 0, len(conds)) - addEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { - totalEdges = append(totalEdges, joinGroupEdge{ +type joinGroupNonEqEdge struct { + nodeIDs []int + nodeIDMask uint + expr expression.Expression +} + +func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, eqConds []expression.Expression) (LogicalPlan, error) { + for _, node := range joinGroup { + _, err := node.recursiveDeriveStats() + if err != nil { + return nil, err + } + s.curJoinGroup = append(s.curJoinGroup, &jrNode{ + p: node, + cumCost: s.baseNodeCumCost(node), + }) + } + adjacents := make([][]int, len(s.curJoinGroup)) + totalEqEdges := make([]joinGroupEqEdge, 0, len(eqConds)) + addEqEdge := func(node1, node2 int, edgeContent *expression.ScalarFunction) { + totalEqEdges = append(totalEqEdges, joinGroupEqEdge{ nodeIDs: []int{node1, node2}, edge: edgeContent, }) @@ -43,7 +58,7 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. adjacents[node2] = append(adjacents[node2], node1) } // Build Graph for join group - for _, cond := range conds { + for _, cond := range eqConds { sf := cond.(*expression.ScalarFunction) lCol := sf.GetArgs()[0].(*expression.Column) rCol := sf.GetArgs()[1].(*expression.Column) @@ -55,7 +70,26 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. if err != nil { return nil, err } - addEdge(lIdx, rIdx, sf) + addEqEdge(lIdx, rIdx, sf) + } + totalNonEqEdges := make([]joinGroupNonEqEdge, 0, len(s.otherConds)) + for _, cond := range s.otherConds { + cols := expression.ExtractColumns(cond) + mask := uint(0) + ids := make([]int, 0, len(cols)) + for _, col := range cols { + idx, err := findNodeIndexInGroup(joinGroup, col) + if err != nil { + return nil, err + } + ids = append(ids, idx) + mask |= 1 << uint(idx) + } + totalNonEqEdges = append(totalNonEqEdges, joinGroupNonEqEdge{ + nodeIDs: ids, + nodeIDMask: mask, + expr: cond, + }) } visited := make([]bool, len(joinGroup)) nodeID2VisitID := make([]int, len(joinGroup)) @@ -66,15 +100,37 @@ func (s *joinReorderDPSolver) solve(joinGroup []LogicalPlan, conds []expression. continue } visitID2NodeID := s.bfsGraph(i, visited, adjacents, nodeID2VisitID) + nodeIDMask := uint(0) + for _, nodeID := range visitID2NodeID { + nodeIDMask |= 1 << uint(nodeID) + } + var subNonEqEdges []joinGroupNonEqEdge + for i := len(totalNonEqEdges) - 1; i >= 0; i-- { + // If this edge is not the subset of the current sub graph. + if totalNonEqEdges[i].nodeIDMask&nodeIDMask != totalNonEqEdges[i].nodeIDMask { + continue + } + newMask := uint(0) + for _, nodeID := range totalNonEqEdges[i].nodeIDs { + newMask |= 1 << uint(nodeID2VisitID[nodeID]) + } + totalNonEqEdges[i].nodeIDMask = newMask + subNonEqEdges = append(subNonEqEdges, totalNonEqEdges[i]) + totalNonEqEdges = append(totalNonEqEdges[:i], totalNonEqEdges[i+1:]...) + } // Do DP on each sub graph. - join, err := s.dpGraph(visitID2NodeID, nodeID2VisitID, joinGroup, totalEdges) + join, err := s.dpGraph(visitID2NodeID, nodeID2VisitID, joinGroup, totalEqEdges, subNonEqEdges) if err != nil { return nil, err } joins = append(joins, join) } + remainedOtherConds := make([]expression.Expression, 0, len(totalNonEqEdges)) + for _, edge := range totalNonEqEdges { + remainedOtherConds = append(remainedOtherConds, edge.expr) + } // Build bushy tree for cartesian joins. - return s.makeBushyJoin(joins), nil + return s.makeBushyJoin(joins, remainedOtherConds), nil } // bfsGraph bfs a sub graph starting at startPos. And relabel its label for future use. @@ -98,13 +154,16 @@ func (s *joinReorderDPSolver) bfsGraph(startNode int, visited []bool, adjacents return visitID2NodeID } -func (s *joinReorderDPSolver) dpGraph(newPos2OldPos, oldPos2NewPos []int, joinGroup []LogicalPlan, totalEdges []joinGroupEdge) (LogicalPlan, error) { - nodeCnt := uint(len(newPos2OldPos)) - bestPlan := make([]LogicalPlan, 1< join.statsInfo().Count()+bestCost[remain]+bestCost[sub] { - bestPlan[nodeBitmap] = join - bestCost[nodeBitmap] = join.statsInfo().Count() + bestCost[remain] + bestCost[sub] + curCost := s.calcJoinCumCost(join, bestPlan[sub], bestPlan[remain]) + if bestPlan[nodeBitmap] == nil { + bestPlan[nodeBitmap] = &jrNode{ + p: join, + cumCost: curCost, + } + } else if bestPlan[nodeBitmap].cumCost > curCost { + bestPlan[nodeBitmap].p = join + bestPlan[nodeBitmap].cumCost = curCost } } } - return bestPlan[(1< 0 && (rightMask&(1< 0 { - usedEdges = append(usedEdges, edge) - } else if (leftMask&(1< 0 && (rightMask&(1< 0 { - usedEdges = append(usedEdges, edge) + if ((leftMask&(1< 0 && (rightMask&(1< 0) || ((leftMask&(1< 0 && (rightMask&(1< 0) { + usedEqEdges = append(usedEqEdges, edge) } } - return usedEdges + for _, edge := range totalNonEqEdges { + // If the result is false, means that the current group hasn't covered the columns involved in the expression. + if edge.nodeIDMask&(leftMask|rightMask) != edge.nodeIDMask { + continue + } + // Check whether this expression is only built from one side of the join. + if edge.nodeIDMask&leftMask == 0 || edge.nodeIDMask&rightMask == 0 { + continue + } + otherConds = append(otherConds, edge.expr) + } + return usedEqEdges, otherConds } -func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, edges []joinGroupEdge) (LogicalPlan, error) { +func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, edges []joinGroupEqEdge, otherConds []expression.Expression) (LogicalPlan, error) { var eqConds []*expression.ScalarFunction for _, edge := range edges { lCol := edge.edge.GetArgs()[0].(*expression.Column) @@ -165,13 +244,13 @@ func (s *joinReorderDPSolver) newJoinWithEdge(leftPlan, rightPlan LogicalPlan, e eqConds = append(eqConds, newSf) } } - join := s.newJoin(leftPlan, rightPlan, eqConds) + join := s.newJoin(leftPlan, rightPlan, eqConds, otherConds) _, err := join.recursiveDeriveStats() return join, err } // Make cartesian join as bushy tree. -func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) LogicalPlan { +func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan, otherConds []expression.Expression) LogicalPlan { for len(cartesianJoinGroup) > 1 { resultJoinGroup := make([]LogicalPlan, 0, len(cartesianJoinGroup)) for i := 0; i < len(cartesianJoinGroup); i += 2 { @@ -179,7 +258,15 @@ func (s *joinReorderDPSolver) makeBushyJoin(cartesianJoinGroup []LogicalPlan) Lo resultJoinGroup = append(resultJoinGroup, cartesianJoinGroup[i]) break } - resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil)) + // TODO:Since the other condition may involve more than two tables, e.g. t1.a = t2.b+t3.c. + // So We'll need a extra stage to deal with it. + // Currently, we just add it when building cartesianJoinGroup. + mergedSchema := expression.MergeSchema(cartesianJoinGroup[i].Schema(), cartesianJoinGroup[i+1].Schema()) + var usedOtherConds []expression.Expression + otherConds, usedOtherConds = expression.FilterOutInPlace(otherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) + resultJoinGroup = append(resultJoinGroup, s.newJoin(cartesianJoinGroup[i], cartesianJoinGroup[i+1], nil, usedOtherConds)) } cartesianJoinGroup = resultJoinGroup } @@ -194,3 +281,14 @@ func findNodeIndexInGroup(group []LogicalPlan, col *expression.Column) (int, err } return -1, ErrUnknownColumn.GenWithStackByArgs(col, "JOIN REORDER RULE") } + +func (s *joinReorderDPSolver) newJoinWithConds(leftPlan, rightPlan LogicalPlan, eqConds []*expression.ScalarFunction, otherConds []expression.Expression) LogicalPlan { + join := s.newCartesianJoin(leftPlan, rightPlan) + join.EqualConditions = eqConds + join.OtherConditions = otherConds + for _, eqCond := range join.EqualConditions { + join.LeftJoinKeys = append(join.LeftJoinKeys, eqCond.GetArgs()[0].(*expression.Column)) + join.RightJoinKeys = append(join.RightJoinKeys, eqCond.GetArgs()[1].(*expression.Column)) + } + return join +} diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index c3d2790e18fb7..72e25a6507f4b 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -56,7 +56,7 @@ func (mj *mockLogicalJoin) recursiveDeriveStats() (*property.StatsInfo, error) { return mj.statsMap[mj.involvedNodeSet], nil } -func (s *testJoinReorderDPSuite) newMockJoin(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction) LogicalPlan { +func (s *testJoinReorderDPSuite) newMockJoin(lChild, rChild LogicalPlan, eqConds []*expression.ScalarFunction, _ []expression.Expression) LogicalPlan { retJoin := mockLogicalJoin{}.init(s.ctx) retJoin.schema = expression.MergeSchema(lChild.Schema(), rChild.Schema()) retJoin.statsMap = s.statsMap @@ -145,7 +145,7 @@ func (s *testJoinReorderDPSuite) makeStatsMapForTPCHQ5() { } -func (s *testJoinReorderDPSuite) newDataSource(name string) LogicalPlan { +func (s *testJoinReorderDPSuite) newDataSource(name string, count int) LogicalPlan { ds := DataSource{}.Init(s.ctx) tan := model.NewCIStr(name) ds.TableAsName = &tan @@ -158,6 +158,9 @@ func (s *testJoinReorderDPSuite) newDataSource(name string) LogicalPlan { DBName: model.NewCIStr("test"), RetType: types.NewFieldType(mysql.TypeLonglong), }) + ds.stats = &property.StatsInfo{ + RowCount: float64(count), + } return ds } @@ -174,12 +177,12 @@ func (s *testJoinReorderDPSuite) planToString(plan LogicalPlan) string { func (s *testJoinReorderDPSuite) TestDPReorderTPCHQ5(c *C) { s.makeStatsMapForTPCHQ5() joinGroups := make([]LogicalPlan, 0, 6) - joinGroups = append(joinGroups, s.newDataSource("lineitem")) - joinGroups = append(joinGroups, s.newDataSource("orders")) - joinGroups = append(joinGroups, s.newDataSource("customer")) - joinGroups = append(joinGroups, s.newDataSource("supplier")) - joinGroups = append(joinGroups, s.newDataSource("nation")) - joinGroups = append(joinGroups, s.newDataSource("region")) + joinGroups = append(joinGroups, s.newDataSource("lineitem", 59986052)) + joinGroups = append(joinGroups, s.newDataSource("orders", 15000000)) + joinGroups = append(joinGroups, s.newDataSource("customer", 1500000)) + joinGroups = append(joinGroups, s.newDataSource("supplier", 100000)) + joinGroups = append(joinGroups, s.newDataSource("nation", 25)) + joinGroups = append(joinGroups, s.newDataSource("region", 5)) var eqConds []expression.Expression eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[0].Schema().Columns[0], joinGroups[1].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[1].Schema().Columns[0], joinGroups[2].Schema().Columns[0])) @@ -189,7 +192,9 @@ func (s *testJoinReorderDPSuite) TestDPReorderTPCHQ5(c *C) { eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[3].Schema().Columns[0], joinGroups[4].Schema().Columns[0])) eqConds = append(eqConds, expression.NewFunctionInternal(s.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), joinGroups[4].Schema().Columns[0], joinGroups[5].Schema().Columns[0])) solver := &joinReorderDPSolver{ - ctx: s.ctx, + baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ + ctx: s.ctx, + }, newJoin: s.newMockJoin, } result, err := solver.solve(joinGroups, eqConds) @@ -199,12 +204,14 @@ func (s *testJoinReorderDPSuite) TestDPReorderTPCHQ5(c *C) { func (s *testJoinReorderDPSuite) TestDPReorderAllCartesian(c *C) { joinGroup := make([]LogicalPlan, 0, 4) - joinGroup = append(joinGroup, s.newDataSource("a")) - joinGroup = append(joinGroup, s.newDataSource("b")) - joinGroup = append(joinGroup, s.newDataSource("c")) - joinGroup = append(joinGroup, s.newDataSource("d")) + joinGroup = append(joinGroup, s.newDataSource("a", 100)) + joinGroup = append(joinGroup, s.newDataSource("b", 100)) + joinGroup = append(joinGroup, s.newDataSource("c", 100)) + joinGroup = append(joinGroup, s.newDataSource("d", 100)) solver := &joinReorderDPSolver{ - ctx: s.ctx, + baseSingleGroupJoinOrderSolver: &baseSingleGroupJoinOrderSolver{ + ctx: s.ctx, + }, newJoin: s.newMockJoin, } result, err := solver.solve(joinGroup, nil) diff --git a/planner/core/rule_join_reorder_greedy.go b/planner/core/rule_join_reorder_greedy.go index 5260fa1bde4b4..6bdf993ec54ba 100644 --- a/planner/core/rule_join_reorder_greedy.go +++ b/planner/core/rule_join_reorder_greedy.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/tidb/expression" ) -type joinReorderGreedySingleGroupSolver struct { +type joinReorderGreedySolver struct { *baseSingleGroupJoinOrderSolver eqEdges []*expression.ScalarFunction } @@ -40,7 +40,7 @@ type joinReorderGreedySingleGroupSolver struct { // // For the nodes and join trees which don't have a join equal condition to // connect them, we make a bushy join tree to do the cartesian joins finally. -func (s *joinReorderGreedySingleGroupSolver) solve(joinNodePlans []LogicalPlan) (LogicalPlan, error) { +func (s *joinReorderGreedySolver) solve(joinNodePlans []LogicalPlan) (LogicalPlan, error) { for _, node := range joinNodePlans { _, err := node.recursiveDeriveStats() if err != nil { @@ -67,7 +67,7 @@ func (s *joinReorderGreedySingleGroupSolver) solve(joinNodePlans []LogicalPlan) return s.makeBushyJoin(cartesianGroup), nil } -func (s *joinReorderGreedySingleGroupSolver) constructConnectedJoinTree() (*jrNode, error) { +func (s *joinReorderGreedySolver) constructConnectedJoinTree() (*jrNode, error) { curJoinTree := s.curJoinGroup[0] s.curJoinGroup = s.curJoinGroup[1:] for { @@ -106,7 +106,7 @@ func (s *joinReorderGreedySingleGroupSolver) constructConnectedJoinTree() (*jrNo return curJoinTree, nil } -func (s *joinReorderGreedySingleGroupSolver) checkConnectionAndMakeJoin(leftNode, rightNode LogicalPlan) (LogicalPlan, []expression.Expression) { +func (s *joinReorderGreedySolver) checkConnectionAndMakeJoin(leftNode, rightNode LogicalPlan) (LogicalPlan, []expression.Expression) { var usedEdges []*expression.ScalarFunction remainOtherConds := make([]expression.Expression, len(s.otherConds)) copy(remainOtherConds, s.otherConds) @@ -123,5 +123,10 @@ func (s *joinReorderGreedySingleGroupSolver) checkConnectionAndMakeJoin(leftNode if len(usedEdges) == 0 { return nil, nil } - return s.newJoinWithEdges(usedEdges, remainOtherConds, leftNode, rightNode) + var otherConds []expression.Expression + mergedSchema := expression.MergeSchema(leftNode.Schema(), rightNode.Schema()) + remainOtherConds, otherConds = expression.FilterOutInPlace(remainOtherConds, func(expr expression.Expression) bool { + return expression.ExprFromSchema(expr, mergedSchema) + }) + return s.newJoinWithEdges(leftNode, rightNode, usedEdges, otherConds), remainOtherConds } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index c9e46d7b30ecc..da2a76c05d413 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -347,6 +347,10 @@ type SessionVars struct { // CommandValue indicates which command current session is doing. CommandValue uint32 + // TIDBOptJoinOrderAlgoThreshold defines the minimal number of join nodes + // to use the greedy join reorder algorithm. + TiDBOptJoinReorderThreshold int + // SlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. SlowQueryFile string @@ -377,30 +381,31 @@ type ConnectionInfo struct { // NewSessionVars creates a session vars object. func NewSessionVars() *SessionVars { vars := &SessionVars{ - Users: make(map[string]string), - systems: make(map[string]string), - PreparedStmts: make(map[uint32]*ast.Prepared), - PreparedStmtNameToID: make(map[string]uint32), - PreparedParams: make([]types.Datum, 0, 10), - TxnCtx: &TransactionContext{}, - KVVars: kv.NewVariables(), - RetryInfo: &RetryInfo{}, - ActiveRoles: make([]*auth.RoleIdentity, 0, 10), - StrictSQLMode: true, - Status: mysql.ServerStatusAutocommit, - StmtCtx: new(stmtctx.StatementContext), - AllowAggPushDown: false, - OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, - RetryLimit: DefTiDBRetryLimit, - DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, - DDLReorgPriority: kv.PriorityLow, - AllowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, - CorrelationThreshold: DefOptCorrelationThreshold, - CorrelationExpFactor: DefOptCorrelationExpFactor, - EnableRadixJoin: false, - L2CacheSize: cpuid.CPU.Cache.L2, - CommandValue: uint32(mysql.ComSleep), - SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, + Users: make(map[string]string), + systems: make(map[string]string), + PreparedStmts: make(map[uint32]*ast.Prepared), + PreparedStmtNameToID: make(map[string]uint32), + PreparedParams: make([]types.Datum, 0, 10), + TxnCtx: &TransactionContext{}, + KVVars: kv.NewVariables(), + RetryInfo: &RetryInfo{}, + ActiveRoles: make([]*auth.RoleIdentity, 0, 10), + StrictSQLMode: true, + Status: mysql.ServerStatusAutocommit, + StmtCtx: new(stmtctx.StatementContext), + AllowAggPushDown: false, + OptimizerSelectivityLevel: DefTiDBOptimizerSelectivityLevel, + RetryLimit: DefTiDBRetryLimit, + DisableTxnAutoRetry: DefTiDBDisableTxnAutoRetry, + DDLReorgPriority: kv.PriorityLow, + AllowInSubqToJoinAndAgg: DefOptInSubqToJoinAndAgg, + CorrelationThreshold: DefOptCorrelationThreshold, + CorrelationExpFactor: DefOptCorrelationExpFactor, + EnableRadixJoin: false, + L2CacheSize: cpuid.CPU.Cache.L2, + CommandValue: uint32(mysql.ComSleep), + TiDBOptJoinReorderThreshold: DefTiDBOptJoinReorderThreshold, + SlowQueryFile: config.GetGlobalConfig().Log.SlowQueryFile, } vars.Concurrency = Concurrency{ IndexLookupConcurrency: DefIndexLookupConcurrency, @@ -754,6 +759,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableRadixJoin = TiDBOptOn(val) case TiDBEnableWindowFunction: s.EnableWindowFunction = TiDBOptOn(val) + case TiDBOptJoinReorderThreshold: + s.TiDBOptJoinReorderThreshold = tidbOptPositiveInt32(val, DefTiDBOptJoinReorderThreshold) case TiDBCheckMb4ValueInUTF8: config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) case TiDBSlowQueryFile: diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 38b0f6754cab6..2a88b23393b71 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -689,6 +689,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, {ScopeSession, TiDBForcePriority, mysql.Priority2Str[DefTiDBForcePriority]}, {ScopeSession, TiDBEnableRadixJoin, BoolToIntStr(DefTiDBUseRadixJoin)}, + {ScopeGlobal | ScopeSession, TiDBOptJoinReorderThreshold, strconv.Itoa(DefTiDBOptJoinReorderThreshold)}, {ScopeSession, TiDBCheckMb4ValueInUTF8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, {ScopeSession, TiDBSlowQueryFile, ""}, {ScopeSession, TiDBWaitTableSplitFinish, BoolToIntStr(DefTiDBWaitTableSplitFinish)}, diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index d40dacb2318f3..f16aec3d5cc4e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -250,6 +250,10 @@ const ( // tidb_enable_window_function is used to control whether to enable the window function. TiDBEnableWindowFunction = "tidb_enable_window_function" + // TIDBOptJoinReorderThreshold defines the threshold less than which + // we'll choose a rather time consuming algorithm to calculate the join order. + TiDBOptJoinReorderThreshold = "tidb_opt_join_reorder_threshold" + // SlowQueryFile indicates which slow query log file for SLOW_QUERY table to parse. TiDBSlowQueryFile = "tidb_slow_query_file" @@ -309,6 +313,7 @@ const ( DefTiDBForcePriority = mysql.NoPriority DefTiDBUseRadixJoin = false DefEnableWindowFunction = false + DefTiDBOptJoinReorderThreshold = 0 DefTiDBDDLSlowOprThreshold = 300 DefTiDBUseFastAnalyze = false DefTiDBSkipIsolationLevelCheck = false diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index 6783975b1b1ef..a9e49e323daae 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -485,6 +485,14 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, return value, errors.Errorf("tidb_max_chunk_size(%d) cannot be smaller than %d", v, maxChunkSizeLowerBound) } return value, nil + case TiDBOptJoinReorderThreshold: + v, err := strconv.Atoi(value) + if err != nil { + return value, ErrWrongTypeForVar.GenWithStackByArgs(name) + } + if v < 0 || v >= 64 { + return value, errors.Errorf("tidb_join_order_algo_threshold(%d) cannot be smaller than 0 or larger than 63", v) + } } return value, nil } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 2b4c04992a4a9..4875582fd39e0 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -79,6 +79,7 @@ func (s *testVarsutilSuite) TestNewSessionVars(c *C) { c.Assert(vars.MemQuotaNestedLoopApply, Equals, int64(DefTiDBMemQuotaNestedLoopApply)) c.Assert(vars.EnableRadixJoin, Equals, DefTiDBUseRadixJoin) c.Assert(vars.AllowWriteRowID, Equals, DefOptWriteRowID) + c.Assert(vars.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) c.Assert(vars.EnableFastAnalyze, Equals, DefTiDBUseFastAnalyze) assertFieldsGreaterThanZero(c, reflect.ValueOf(vars.Concurrency)) @@ -254,6 +255,14 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "on") c.Assert(v.EnableTablePartition, Equals, "on") + c.Assert(v.TiDBOptJoinReorderThreshold, Equals, DefTiDBOptJoinReorderThreshold) + err = SetSessionSystemVar(v, TiDBOptJoinReorderThreshold, types.NewIntDatum(5)) + c.Assert(err, IsNil) + val, err = GetSessionSystemVar(v, TiDBOptJoinReorderThreshold) + c.Assert(err, IsNil) + c.Assert(val, Equals, "5") + c.Assert(v.TiDBOptJoinReorderThreshold, Equals, 5) + err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, types.NewStringDatum("1")) c.Assert(err, IsNil) val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) From 168776dcfa3a82f5278366765d3e63fe2be62f1a Mon Sep 17 00:00:00 2001 From: liyuzhou <2541781827@qq.com> Date: Sun, 28 Apr 2019 13:20:13 +0800 Subject: [PATCH 08/49] * : support show session binding (#10285) --- bindinfo/bind_test.go | 17 +++++++++++++++ bindinfo/cache.go | 8 ++++---- bindinfo/handle.go | 42 +++++++++++++++++++------------------- bindinfo/session_handle.go | 24 ++++++++++++++-------- executor/show.go | 8 ++++++-- 5 files changed, 64 insertions(+), 35 deletions(-) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 423ef629bc3d8..76d93dc476277 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -256,4 +256,21 @@ func (s *testSuite) TestSessionBinding(c *C) { err = rs.Next(context.TODO(), chk) c.Check(err, IsNil) c.Check(chk.NumRows(), Equals, 0) + + rs, err = tk.Exec("show session bindings") + c.Assert(err, IsNil) + chk = rs.NewRecordBatch() + err = rs.Next(context.TODO(), chk) + c.Check(err, IsNil) + c.Check(chk.NumRows(), Equals, 1) + row := chk.GetRow(0) + c.Check(row.GetString(0), Equals, "select * from t where i > ?") + c.Check(row.GetString(1), Equals, "select * from t use index(index_t) where i>99") + c.Check(row.GetString(2), Equals, "test") + c.Check(row.GetString(3), Equals, "using") + c.Check(row.GetTime(4), NotNil) + c.Check(row.GetTime(5), NotNil) + c.Check(row.GetString(6), NotNil) + c.Check(row.GetString(7), NotNil) + } diff --git a/bindinfo/cache.go b/bindinfo/cache.go index b8731628ec080..b8ef2583c0f69 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -26,14 +26,14 @@ const ( deleted = "deleted" ) -// bindMeta stores the basic bind info and bindSql astNode. -type bindMeta struct { +// BindMeta stores the basic bind info and bindSql astNode. +type BindMeta struct { *BindRecord ast ast.StmtNode //ast will be used to do query sql bind check } -// cache is a k-v map, key is original sql, value is a slice of bindMeta. -type cache map[string][]*bindMeta +// cache is a k-v map, key is original sql, value is a slice of BindMeta. +type cache map[string][]*BindMeta // BindRecord represents a sql bind record retrieved from the storage. type BindRecord struct { diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 0be680217c186..a6b54e937219d 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -113,7 +113,7 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { return nil } -// AddBindRecord adds a BindRecord to the storage and bindMeta to the cache. +// AddBindRecord adds a BindRecord to the storage and BindMeta to the cache. func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) h.sctx.Lock() @@ -137,7 +137,7 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { return } - // update the bindMeta to the cache. + // update the BindMeta to the cache. hash, meta, err1 := h.newBindMeta(record) if err1 != nil { err = err1 @@ -171,7 +171,7 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { return err } -// DropBindRecord drops a BindRecord to the storage and bindMeta int the cache. +// DropBindRecord drops a BindRecord to the storage and BindMeta int the cache. func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { exec, _ := h.sctx.Context.(sqlexec.SQLExecutor) h.sctx.Lock() @@ -226,8 +226,8 @@ func (h *BindHandle) Size() int { return size } -// GetBindRecord return the bindMeta of the (normdOrigSQL,db) if bindMeta exist. -func (h *BindHandle) GetBindRecord(normdOrigSQL, db string) *bindMeta { +// GetBindRecord return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. +func (h *BindHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { hash := parser.DigestHash(normdOrigSQL) bindRecords := h.bindInfo.Load().(cache)[hash] if bindRecords != nil { @@ -241,7 +241,7 @@ func (h *BindHandle) GetBindRecord(normdOrigSQL, db string) *bindMeta { } // GetAllBindRecord return all bind record in cache. -func (h *BindHandle) GetAllBindRecord() (bindRecords []*bindMeta) { +func (h *BindHandle) GetAllBindRecord() (bindRecords []*BindMeta) { bindRecordMap := h.bindInfo.Load().(cache) for _, bindRecord := range bindRecordMap { bindRecords = append(bindRecords, bindRecord...) @@ -249,25 +249,25 @@ func (h *BindHandle) GetAllBindRecord() (bindRecords []*bindMeta) { return bindRecords } -func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *bindMeta, err error) { +func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) { hash = parser.DigestHash(record.OriginalSQL) stmtNodes, _, err := h.parser.Parse(record.BindSQL, record.Charset, record.Collation) if err != nil { return "", nil, err } - meta = &bindMeta{BindRecord: record, ast: stmtNodes[0]} + meta = &BindMeta{BindRecord: record, ast: stmtNodes[0]} return hash, meta, nil } -func (h *BindHandle) newBindMetaWithoutAst(record *BindRecord) (hash string, meta *bindMeta) { +func (h *BindHandle) newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) { hash = parser.DigestHash(record.OriginalSQL) - meta = &bindMeta{BindRecord: record} + meta = &BindMeta{BindRecord: record} return hash, meta } -// appendBindMeta addes the bindMeta to the cache, all the stale bindMetas are +// appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are // removed from the cache after this operation. -func (h *BindHandle) appendBindMeta(hash string, meta *bindMeta) { +func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) { // Make sure there is only one goroutine writes the cache. h.bindInfo.Lock() newCache := h.bindInfo.Value.Load().(cache).copy() @@ -280,8 +280,8 @@ func (h *BindHandle) appendBindMeta(hash string, meta *bindMeta) { newCache[hash] = append(newCache[hash], meta) } -// removeBindMeta removes the bindMeta from the cache. -func (h *BindHandle) removeBindMeta(hash string, meta *bindMeta) { +// removeBindMeta removes the BindMeta from the cache. +func (h *BindHandle) removeBindMeta(hash string, meta *BindMeta) { h.bindInfo.Lock() newCache := h.bindInfo.Value.Load().(cache).copy() defer func() { @@ -292,8 +292,8 @@ func (h *BindHandle) removeBindMeta(hash string, meta *bindMeta) { newCache.removeDeletedBindMeta(hash, meta) } -// removeDeletedBindMeta removes all the bindMeta which originSQL and db are the same with the parameter's meta. -func (c cache) removeDeletedBindMeta(hash string, meta *bindMeta) { +// removeDeletedBindMeta removes all the BindMeta which originSQL and db are the same with the parameter's meta. +func (c cache) removeDeletedBindMeta(hash string, meta *BindMeta) { metas, ok := c[hash] if !ok { return @@ -310,8 +310,8 @@ func (c cache) removeDeletedBindMeta(hash string, meta *bindMeta) { } } -// removeStaleBindMetas removes all the stale bindMeta in the cache. -func (c cache) removeStaleBindMetas(hash string, meta *bindMeta) { +// removeStaleBindMetas removes all the stale BindMeta in the cache. +func (c cache) removeStaleBindMetas(hash string, meta *BindMeta) { metas, ok := c[hash] if !ok { return @@ -337,13 +337,13 @@ func (c cache) copy() cache { return newCache } -// isStale checks whether this bindMeta is stale compared with the other bindMeta. -func (m *bindMeta) isStale(other *bindMeta) bool { +// isStale checks whether this BindMeta is stale compared with the other BindMeta. +func (m *BindMeta) isStale(other *BindMeta) bool { return m.OriginalSQL == other.OriginalSQL && m.Db == other.Db && m.UpdateTime.Compare(other.UpdateTime) <= 0 } -func (m *bindMeta) isSame(other *bindMeta) bool { +func (m *BindMeta) isSame(other *BindMeta) bool { return m.OriginalSQL == other.OriginalSQL && m.Db == other.Db } diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index e6ae2f0d37089..1bc9b2e6c4d18 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -34,25 +34,25 @@ func NewSessionBindHandle(parser *parser.Parser) *SessionHandle { return sessionHandle } -// appendBindMeta addes the bindMeta to the cache, all the stale bindMetas are +// appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are // removed from the cache after this operation. -func (h *SessionHandle) appendBindMeta(hash string, meta *bindMeta) { +func (h *SessionHandle) appendBindMeta(hash string, meta *BindMeta) { // Make sure there is only one goroutine writes the cache. h.ch.removeStaleBindMetas(hash, meta) h.ch[hash] = append(h.ch[hash], meta) } -func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *bindMeta, err error) { +func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) { hash = parser.DigestHash(record.OriginalSQL) stmtNodes, _, err := h.parser.Parse(record.BindSQL, record.Charset, record.Collation) if err != nil { return "", nil, err } - meta = &bindMeta{BindRecord: record, ast: stmtNodes[0]} + meta = &BindMeta{BindRecord: record, ast: stmtNodes[0]} return hash, meta, nil } -// AddBindRecord new a BindRecord with bindMeta, add it to the cache. +// AddBindRecord new a BindRecord with BindMeta, add it to the cache. func (h *SessionHandle) AddBindRecord(record *BindRecord) error { record.Status = using record.CreateTime = types.Time{ @@ -62,7 +62,7 @@ func (h *SessionHandle) AddBindRecord(record *BindRecord) error { } record.UpdateTime = record.CreateTime - // update the bindMeta to the cache. + // update the BindMeta to the cache. hash, meta, err := h.newBindMeta(record) if err == nil { h.appendBindMeta(hash, meta) @@ -70,8 +70,8 @@ func (h *SessionHandle) AddBindRecord(record *BindRecord) error { return err } -// GetBindRecord return the bindMeta of the (normdOrigSQL,db) if bindMeta exist. -func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *bindMeta { +// GetBindRecord return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. +func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { hash := parser.DigestHash(normdOrigSQL) bindRecords := h.ch[hash] if bindRecords != nil { @@ -84,6 +84,14 @@ func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *bindMeta { return nil } +// GetAllBindRecord return all session bind info. +func (h *SessionHandle) GetAllBindRecord() (bindRecords []*BindMeta) { + for _, bindRecord := range h.ch { + bindRecords = append(bindRecords, bindRecord...) + } + return bindRecords +} + // sessionBindInfoKeyType is a dummy type to avoid naming collision in context. type sessionBindInfoKeyType int diff --git a/executor/show.go b/executor/show.go index 99b1aa7a51050..5d9b129755502 100644 --- a/executor/show.go +++ b/executor/show.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/etcd" "github.com/pingcap/tidb-tools/pkg/utils" "github.com/pingcap/tidb-tools/tidb-binlog/node" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" @@ -182,10 +183,13 @@ func (e *ShowExec) fetchAll() error { } func (e *ShowExec) fetchShowBind() error { + var bindRecords []*bindinfo.BindMeta if !e.GlobalScope { - return errors.New("show non-global bind sql is not supported") + handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + bindRecords = handle.GetAllBindRecord() + } else { + bindRecords = domain.GetDomain(e.ctx).BindHandle().GetAllBindRecord() } - bindRecords := domain.GetDomain(e.ctx).BindHandle().GetAllBindRecord() for _, bindData := range bindRecords { e.appendRow([]interface{}{ bindData.OriginalSQL, From 45c0e51eca6d5966022aa338c3530bcaa0b2ac13 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Sun, 28 Apr 2019 13:35:19 +0800 Subject: [PATCH 09/49] executor: check inconsistent index in `IndexLookupExecutor` (#10237) --- executor/distsql.go | 26 ++++++++++++++------ executor/distsql_test.go | 53 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 72 insertions(+), 7 deletions(-) diff --git a/executor/distsql.go b/executor/distsql.go index e93d67cb16670..5bd77376292c1 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -519,6 +519,7 @@ func (e *IndexLookUpExecutor) startTableWorker(ctx context.Context, workCh <-cha e.tblWorkerWg.Add(lookupConcurrencyLimit) for i := 0; i < lookupConcurrencyLimit; i++ { worker := &tableWorker{ + idxLookup: e, workCh: workCh, finished: e.finished, buildTblReader: e.buildTableReader, @@ -732,6 +733,7 @@ func (w *indexWorker) buildTableTask(handles []int64) *lookupTableTask { // tableWorker is used by IndexLookUpExecutor to maintain table lookup background goroutines. type tableWorker struct { + idxLookup *IndexLookUpExecutor workCh <-chan *lookupTableTask finished <-chan struct{} buildTblReader func(ctx context.Context, handles []int64) (Executor, error) @@ -809,6 +811,7 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er task.rows = append(task.rows, row) } } + memUsage = int64(cap(task.rows)) * int64(unsafe.Sizeof(chunk.Row{})) task.memUsage += memUsage task.memTracker.Consume(memUsage) @@ -824,14 +827,23 @@ func (w *tableWorker) executeTask(ctx context.Context, task *lookupTableTask) er sort.Sort(task) } - if w.isCheckOp && handleCnt != len(task.rows) { - obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) - for _, row := range task.rows { - handle := row.GetInt64(w.handleIdx) - obtainedHandlesMap[handle] = struct{}{} + if handleCnt != len(task.rows) { + if w.isCheckOp { + obtainedHandlesMap := make(map[int64]struct{}, len(task.rows)) + for _, row := range task.rows { + handle := row.GetInt64(w.handleIdx) + obtainedHandlesMap[handle] = struct{}{} + } + return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d, missing handles %v in a batch", + w.idxLookup.index.Name.O, handleCnt, len(task.rows), GetLackHandles(task.handles, obtainedHandlesMap)) + } + + if len(w.idxLookup.tblPlans) == 1 { + // table scan in double read can never has conditions according to convertToIndexScan. + // if this table scan has no condition, the number of rows it returns must equal to the length of handles. + return errors.Errorf("inconsistent index %s handle count %d isn't equal to value count %d", + w.idxLookup.index.Name.O, handleCnt, len(task.rows)) } - return errors.Errorf("handle count %d isn't equal to value count %d, missing handles %v in a batch", - handleCnt, len(task.rows), GetLackHandles(task.handles, obtainedHandlesMap)) } return nil diff --git a/executor/distsql_test.go b/executor/distsql_test.go index 5ffb18036839b..814e6f47fe1bb 100644 --- a/executor/distsql_test.go +++ b/executor/distsql_test.go @@ -25,7 +25,10 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" ) @@ -184,3 +187,53 @@ func (s *testSuite3) TestIssue10178(c *C) { tk.MustQuery("select * from t where a > 9223372036854775807").Check(testkit.Rows("18446744073709551615")) tk.MustQuery("select * from t where a < 9223372036854775808").Check(testkit.Rows("9223372036854775807")) } + +func (s *testSuite3) TestInconsistentIndex(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, index idx_a(a))") + is := s.domain.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + idx := tbl.Meta().FindIndexByName("idx_a") + idxOp := tables.NewIndex(tbl.Meta().ID, tbl.Meta(), idx) + ctx := mock.NewContext() + ctx.Store = s.store + + for i := 0; i < 10; i++ { + tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i+10, i)) + c.Assert(tk.QueryToErr("select * from t where a>=0"), IsNil) + } + + for i := 0; i < 10; i++ { + tk.MustExec(fmt.Sprintf("update t set a=%d where a=%d", i, i+10)) + c.Assert(tk.QueryToErr("select * from t where a>=0"), IsNil) + } + + for i := 0; i < 10; i++ { + txn, err := s.store.Begin() + c.Assert(err, IsNil) + _, err = idxOp.Create(ctx, txn, types.MakeDatums(i+10), int64(100+i)) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + + err = tk.QueryToErr("select * from t use index(idx_a) where a >= 0") + c.Assert(err.Error(), Equals, fmt.Sprintf("inconsistent index idx_a handle count %d isn't equal to value count 10", i+11)) + + // if has other conditions, the inconsistent index check doesn't work. + err = tk.QueryToErr("select * from t where a>=0 and b<10") + c.Assert(err, IsNil) + } + + // fix inconsistent problem to pass CI + for i := 0; i < 10; i++ { + txn, err := s.store.Begin() + c.Assert(err, IsNil) + err = idxOp.Delete(ctx.GetSessionVars().StmtCtx, txn, types.MakeDatums(i+10), int64(100+i), nil) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + } +} From 5fa16a8030eb32cb75b17e1bd54ed8d89889df71 Mon Sep 17 00:00:00 2001 From: "Zhuomin(Charming) Liu" Date: Sun, 28 Apr 2019 13:49:55 +0800 Subject: [PATCH 10/49] executor: support building stats for fast analyze. (#10258) --- executor/analyze.go | 71 +++++++++++++++++++++++------- executor/analyze_test.go | 85 +++++++++++++++++++++++++++++++++++- executor/builder.go | 4 ++ planner/core/common_plans.go | 4 +- planner/core/planbuilder.go | 6 ++- statistics/builder.go | 22 +++++++--- statistics/cmsketch.go | 6 +-- statistics/cmsketch_test.go | 3 +- statistics/sample.go | 11 +++++ 9 files changed, 182 insertions(+), 30 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index 9c0e671d77270..12dde20fe0df1 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -24,6 +24,7 @@ import ( "sync/atomic" "time" + "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/debugpb" "github.com/pingcap/parser/model" @@ -160,8 +161,8 @@ var errAnalyzeWorkerPanic = errors.New("analyze worker panic") func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<- analyzeResult, isCloseChanThread bool) { defer func() { e.wg.Done() - e.wg.Wait() if isCloseChanThread { + e.wg.Wait() close(resultCh) } if r := recover(); r != nil { @@ -190,6 +191,8 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<- task.job.Start() resultCh <- analyzeIndexPushdown(task.idxExec) case fastTask: + task.fastExec.job = task.job + task.job.Start() for _, result := range analyzeFastExec(task.fastExec) { resultCh <- result } @@ -507,7 +510,7 @@ func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms [] func analyzeFastExec(exec *AnalyzeFastExec) []analyzeResult { hists, cms, err := exec.buildStats() if err != nil { - return []analyzeResult{{Err: err}} + return []analyzeResult{{Err: err, job: exec.job}} } var results []analyzeResult hasPKInfo := 0 @@ -522,6 +525,7 @@ func analyzeFastExec(exec *AnalyzeFastExec) []analyzeResult { Cms: []*statistics.CMSketch{cms[i]}, IsIndex: 1, Count: hists[i].NullCount, + job: exec.job, } if hists[i].Len() > 0 { idxResult.Count += hists[i].Buckets[hists[i].Len()-1].Count @@ -535,6 +539,7 @@ func analyzeFastExec(exec *AnalyzeFastExec) []analyzeResult { Hist: hists[:hasPKInfo+len(exec.colsInfo)], Cms: cms[:hasPKInfo+len(exec.colsInfo)], Count: hist.NullCount, + job: exec.job, } if hist.Len() > 0 { colResult.Count += hist.Buckets[hist.Len()-1].Count @@ -560,6 +565,7 @@ type AnalyzeFastExec struct { idxsInfo []*model.IndexInfo concurrency int maxNumBuckets uint64 + tblInfo *model.TableInfo cache *tikv.RegionCache wg *sync.WaitGroup sampLocs chan *tikv.KeyLocation @@ -569,6 +575,7 @@ type AnalyzeFastExec struct { scanTasks []*tikv.KeyLocation collectors []*statistics.SampleCollector randSeed int64 + job *statistics.AnalyzeJob } func (e *AnalyzeFastExec) getSampRegionsRowCount(bo *tikv.Backoffer, needRebuild *bool, err *error, sampTasks *[]*AnalyzeFastTask) { @@ -627,12 +634,12 @@ func (e *AnalyzeFastExec) getSampRegionsRowCount(bo *tikv.Backoffer, needRebuild } } -// buildSampTask return tow variable, the first bool is whether the task meeting region error +// buildSampTask returns tow variables, the first bool is whether the task meets region error // and need to rebuild. func (e *AnalyzeFastExec) buildSampTask() (needRebuild bool, err error) { // Do get regions row count. bo := tikv.NewBackoffer(context.Background(), 500) - atomic.StoreUint64(&e.rowCount, 0) + e.rowCount = 0 needRebuildForRoutine := make([]bool, e.concurrency) errs := make([]error, e.concurrency) sampTasksForRoutine := make([][]*AnalyzeFastTask, e.concurrency) @@ -721,6 +728,11 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if err != nil { return err } + var rowID int64 + rowID, err = tablecodec.DecodeRowKey(sKey) + if err != nil { + return err + } // Update the primary key collector. if hasPKInfo > 0 { v, ok := values[e.pkInfo.ID] @@ -735,7 +747,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[0].Samples[samplePos] == nil { e.collectors[0].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[0].Samples[samplePos].Ordinal = int(samplePos) + e.collectors[0].Samples[samplePos].RowID = rowID e.collectors[0].Samples[samplePos].Value = v } // Update the columns' collectors. @@ -747,7 +759,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[hasPKInfo+j].Samples[samplePos] == nil { e.collectors[hasPKInfo+j].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[hasPKInfo+j].Samples[samplePos].Ordinal = int(samplePos) + e.collectors[hasPKInfo+j].Samples[samplePos].RowID = rowID e.collectors[hasPKInfo+j].Samples[samplePos].Value = v } // Update the indexes' collectors. @@ -773,7 +785,7 @@ func (e *AnalyzeFastExec) updateCollectorSamples(sValue []byte, sKey kv.Key, sam if e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos] == nil { e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos] = &statistics.SampleItem{} } - e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].Ordinal = int(samplePos) + e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].RowID = rowID e.collectors[len(e.colsInfo)+hasPKInfo+j].Samples[samplePos].Value = types.NewBytesDatum(bytes) } return nil @@ -844,9 +856,7 @@ func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) error { } func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *error) { - defer func() { - e.wg.Done() - }() + defer e.wg.Done() var snapshot kv.Snapshot snapshot, *err = e.ctx.GetStore().(tikv.Storage).GetSnapshot(kv.MaxVersion) rander := rand.New(rand.NewSource(e.randSeed + int64(workID))) @@ -869,9 +879,6 @@ func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *e if *err != nil { return } - if maxRowID <= minRowID { - continue - } keys := make([]kv.Key, 0, task.SampSize) for i := 0; i < int(task.SampSize); i++ { @@ -897,8 +904,37 @@ func (e *AnalyzeFastExec) handleSampTasks(bo *tikv.Backoffer, workID int, err *e } func (e *AnalyzeFastExec) buildHist(ID int64, collector *statistics.SampleCollector, tp *types.FieldType) (*statistics.Histogram, error) { - // TODO: build histogram and cmsketch here for one collector. - return nil, nil + // build collector properties. + collector.Samples = collector.Samples[:e.sampCursor] + sort.Slice(collector.Samples, func(i, j int) bool { return collector.Samples[i].RowID < collector.Samples[j].RowID }) + collector.CalcTotalSize() + data := make([][]byte, 0, len(collector.Samples)) + for i, sample := range collector.Samples { + sample.Ordinal = i + if sample.Value.IsNull() { + collector.NullCount++ + continue + } + bytes, err := tablecodec.EncodeValue(e.ctx.GetSessionVars().StmtCtx, sample.Value) + if err != nil { + return nil, err + } + data = append(data, bytes) + } + stats := domain.GetDomain(e.ctx).StatsHandle() + rowCount := int64(e.rowCount) + if stats.Lease > 0 { + rowCount = mathutil.MinInt64(stats.GetTableStats(e.tblInfo).Count, rowCount) + } + // build CMSketch + var ndv, scaleRatio uint64 + collector.CMSketch, ndv, scaleRatio = statistics.NewCMSketchWithTopN(defaultCMSketchDepth, defaultCMSketchWidth, data, 20, uint64(rowCount)) + // build Histogram + hist, err := statistics.BuildColumnHist(e.ctx, int64(e.maxNumBuckets), ID, collector, tp, rowCount, int64(ndv), collector.NullCount*int64(scaleRatio)) + if err != nil { + return nil, err + } + return hist, nil } func (e *AnalyzeFastExec) runTasks() ([]*statistics.Histogram, []*statistics.CMSketch, error) { @@ -974,6 +1010,8 @@ func (e *AnalyzeFastExec) buildStats() (hists []*statistics.Histogram, cms []*st return nil, nil, errors.Errorf(errMsg, maxBuildTimes) } + defer e.job.Update(int64(e.rowCount)) + // If total row count of the table is smaller than 2*MaxSampleSize, we // translate all the sample tasks to scan tasks. if e.rowCount < uint64(MaxSampleSize)*2 { @@ -1009,6 +1047,7 @@ type AnalyzeTestFastExec struct { IdxsInfo []*model.IndexInfo Concurrency int Collectors []*statistics.SampleCollector + TblInfo *model.TableInfo } // TestFastSample only test the fast sample in unit test. @@ -1020,6 +1059,8 @@ func (e *AnalyzeTestFastExec) TestFastSample() error { e.concurrency = e.Concurrency e.physicalTableID = e.PhysicalTableID e.wg = &sync.WaitGroup{} + e.job = &statistics.AnalyzeJob{} + e.tblInfo = e.TblInfo _, _, err := e.buildStats() e.Collectors = e.collectors return err diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 1c9c8c7060649..e4407e88e2df3 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -183,6 +183,7 @@ func (s *testSuite1) TestAnalyzeFastSample(c *C) { IdxsInfo: indicesInfo, Concurrency: 1, PhysicalTableID: tbl.(table.PhysicalTable).GetPhysicalID(), + TblInfo: tblInfo, } err = mockExec.TestFastSample() c.Assert(err, IsNil) @@ -197,5 +198,87 @@ func (s *testSuite1) TestAnalyzeFastSample(c *C) { vals[i] = append(vals[i], s) } } - c.Assert(fmt.Sprintln(vals), Equals, "[[0 34 35 57 4 24 6 25 58 9 10 11 12 30 14 52 29 17 44 54] [0 34 35 57 4 24 6 25 58 9 10 11 12 30 14 52 29 17 44 54]]\n") + c.Assert(fmt.Sprintln(vals), Equals, "[[0 4 6 9 10 11 12 14 17 24 25 29 30 34 35 44 52 54 57 58] [0 4 6 9 10 11 12 14 17 24 25 29 30 34 35 44 52 54 57 58]]\n") +} + +func (s *testSuite1) TestFastAnalyze(c *C) { + cluster := mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(cluster) + store, err := mockstore.NewMockTikvStore( + mockstore.WithCluster(cluster), + ) + c.Assert(err, IsNil) + var dom *domain.Domain + dom, err = session.BootstrapSession(store) + c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + executor.MaxSampleSize = 1000 + executor.RandSeed = 123 + + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int primary key, b int, index index_b(b))") + tk.MustExec("set @@session.tidb_enable_fast_analyze=1") + tk.MustExec("set @@session.tidb_build_stats_concurrency=1") + for i := 0; i < 3000; i++ { + tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", i, i)) + } + tblInfo, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tid := tblInfo.Meta().ID + + // construct 5 regions split by {600, 1200, 1800, 2400} + splitKeys := generateTableSplitKeyForInt(tid, []int{600, 1200, 1800, 2400}) + manipulateCluster(cluster, splitKeys) + + tk.MustExec("analyze table t with 5 buckets") + + is := executor.GetInfoSchema(tk.Se.(sessionctx.Context)) + table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := table.Meta() + tbl := dom.StatsHandle().GetTableStats(tableInfo) + sTbl := fmt.Sprintln(tbl) + matched := false + if sTbl == "Table:37 Count:3000\n"+ + "column:1 ndv:3000 totColSize:0\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+ + "column:2 ndv:3000 totColSize:0\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+ + "index:1 ndv:3000\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n" || + sTbl == "Table:37 Count:3000\n"+ + "column:2 ndv:3000 totColSize:0\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+ + "column:1 ndv:3000 totColSize:0\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n"+ + "index:1 ndv:3000\n"+ + "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ + "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ + "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ + "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ + "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n" { + matched = true + } + c.Assert(matched, Equals, true) } diff --git a/executor/builder.go b/executor/builder.go index 55142e42995cf..11de4897155ac 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1435,9 +1435,11 @@ func (b *executorBuilder) buildAnalyzeFastColumn(e *AnalyzeExec, task plannercor colsInfo: task.ColsInfo, pkInfo: task.PKInfo, maxNumBuckets: maxNumBuckets, + tblInfo: task.TblInfo, concurrency: concurrency, wg: &sync.WaitGroup{}, }, + job: &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: "fast analyze columns"}, }) } } @@ -1464,9 +1466,11 @@ func (b *executorBuilder) buildAnalyzeFastIndex(e *AnalyzeExec, task plannercore physicalTableID: task.PhysicalTableID, idxsInfo: []*model.IndexInfo{task.IndexInfo}, maxNumBuckets: maxNumBuckets, + tblInfo: task.TblInfo, concurrency: concurrency, wg: &sync.WaitGroup{}, }, + job: &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: "fast analyze index " + task.IndexInfo.Name.O}, }) } } diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 933c48e4697cc..0c63590a7e507 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -427,20 +427,20 @@ type analyzeInfo struct { PartitionName string // PhysicalTableID is the id for a partition or a table. PhysicalTableID int64 - PKInfo *model.ColumnInfo - ColsInfo []*model.ColumnInfo } // AnalyzeColumnsTask is used for analyze columns. type AnalyzeColumnsTask struct { PKInfo *model.ColumnInfo ColsInfo []*model.ColumnInfo + TblInfo *model.TableInfo analyzeInfo } // AnalyzeIndexTask is used for analyze index. type AnalyzeIndexTask struct { IndexInfo *model.IndexInfo + TblInfo *model.TableInfo analyzeInfo } diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 64bef437ed633..2d231b07dadc4 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -812,6 +812,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) (Plan, error) p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{ IndexInfo: idx, analyzeInfo: info, + TblInfo: tbl.TableInfo, }) } } @@ -822,6 +823,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) (Plan, error) PKInfo: pkInfo, ColsInfo: colInfo, analyzeInfo: info, + TblInfo: tbl.TableInfo, }) } } @@ -843,7 +845,7 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt) (Plan, error) } for i, id := range physicalIDs { info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id} - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info}) + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo}) } } return p, nil @@ -860,7 +862,7 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt) (Plan, erro if idx.State == model.StatePublic { for i, id := range physicalIDs { info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id} - p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info}) + p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo}) } } } diff --git a/statistics/builder.go b/statistics/builder.go index 9ede4ad58cb0b..c675656135557 100644 --- a/statistics/builder.go +++ b/statistics/builder.go @@ -93,15 +93,20 @@ func (b *SortedBuilder) Iterate(data types.Datum) error { return nil } -// BuildColumn builds histogram from samples for column. -func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType) (*Histogram, error) { - count := collector.Count - ndv := collector.FMSketch.NDV() +// BuildColumnHist build a histogram for a column. +// numBuckets: number of buckets for the histogram. +// id: the id of the table. +// collector: the collector of samples. +// tp: the FieldType for the column. +// count: represents the row count for the column. +// ndv: represents the number of distinct values for the column. +// nullCount: represents the number of null values for the column. +func BuildColumnHist(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType, count int64, ndv int64, nullCount int64) (*Histogram, error) { if ndv > count { ndv = count } if count == 0 || len(collector.Samples) == 0 { - return NewHistogram(id, ndv, collector.NullCount, 0, tp, 0, collector.TotalSize), nil + return NewHistogram(id, ndv, nullCount, 0, tp, 0, collector.TotalSize), nil } sc := ctx.GetSessionVars().StmtCtx samples := collector.Samples @@ -109,7 +114,7 @@ func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *Sample if err != nil { return nil, err } - hg := NewHistogram(id, ndv, collector.NullCount, 0, tp, int(numBuckets), collector.TotalSize) + hg := NewHistogram(id, ndv, nullCount, 0, tp, int(numBuckets), collector.TotalSize) sampleNum := int64(len(samples)) // As we use samples to build the histogram, the bucket number and repeat should multiply a factor. @@ -174,3 +179,8 @@ func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *Sample hg.Correlation = (itemsCount*corrXYSum - corrXSum*corrXSum) / (itemsCount*corrX2Sum - corrXSum*corrXSum) return hg, nil } + +// BuildColumn builds histogram from samples for column. +func BuildColumn(ctx sessionctx.Context, numBuckets, id int64, collector *SampleCollector, tp *types.FieldType) (*Histogram, error) { + return BuildColumnHist(ctx, numBuckets, id, collector, tp, collector.Count, collector.FMSketch.NDV(), collector.NullCount) +} diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 6961d4733880e..e82ed041e1e11 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -109,8 +109,8 @@ func newTopNHelper(sample [][]byte, numTop uint32) *topNHelper { return &topNHelper{uint64(len(sample)), numTop, counter, sorted, onlyOnceItems, sumTopN, last} } -// NewCMSketchWithTopN returns a new CM sketch with TopN elements. -func NewCMSketchWithTopN(d, w int32, sample [][]byte, numTop uint32, rowCount uint64) *CMSketch { +// NewCMSketchWithTopN returns a new CM sketch with TopN elements, the estimate NDV and the scale ratio. +func NewCMSketchWithTopN(d, w int32, sample [][]byte, numTop uint32, rowCount uint64) (*CMSketch, uint64, uint64) { helper := newTopNHelper(sample, numTop) // rowCount is not a accurate value when fast analyzing // In some cases, if user triggers fast analyze when rowCount is close to sampleSize, unexpected bahavior might happen. @@ -118,7 +118,7 @@ func NewCMSketchWithTopN(d, w int32, sample [][]byte, numTop uint32, rowCount ui estimateNDV, scaleRatio := calculateEstimateNDV(helper, rowCount) c := buildCMSWithTopN(helper, d, w, scaleRatio) c.calculateDefaultVal(helper, estimateNDV, scaleRatio, rowCount) - return c + return c, estimateNDV, scaleRatio } func buildCMSWithTopN(helper *topNHelper, d, w int32, scaleRatio uint64) (c *CMSketch) { diff --git a/statistics/cmsketch_test.go b/statistics/cmsketch_test.go index f95e0b12e2fb4..084fe473f3adf 100644 --- a/statistics/cmsketch_test.go +++ b/statistics/cmsketch_test.go @@ -43,7 +43,8 @@ func prepareCMSWithTopN(d, w int32, vals []*types.Datum, n uint32, total uint64) } data = append(data, bytes) } - return NewCMSketchWithTopN(d, w, data, n, total), nil + cms, _, _ := NewCMSketchWithTopN(d, w, data, n, total) + return cms, nil } func buildCMSketchAndMap(d, w int32, seed int64, total, imax uint64, s float64) (*CMSketch, map[int64]uint32, error) { diff --git a/statistics/sample.go b/statistics/sample.go index 6805943046c8f..31b1f712b1719 100644 --- a/statistics/sample.go +++ b/statistics/sample.go @@ -36,6 +36,9 @@ type SampleItem struct { // Ordinal is original position of this item in SampleCollector before sorting. This // is used for computing correlation. Ordinal int + // RowID is the row id of the sample in its key. + // This property is used to calculate Ordinal in fast analyze. + RowID int64 } // SortSampleItems sorts a slice of SampleItem. @@ -173,6 +176,14 @@ func (c *SampleCollector) collect(sc *stmtctx.StatementContext, d types.Datum) e return nil } +// CalcTotalSize is to calculate total size based on samples. +func (c *SampleCollector) CalcTotalSize() { + c.TotalSize = 0 + for _, item := range c.Samples { + c.TotalSize += int64(len(item.Value.GetBytes())) + } +} + // SampleBuilder is used to build samples for columns. // Also, if primary key is handle, it will directly build histogram for it. type SampleBuilder struct { From 82782a1f112c090d107892a0f5bb840387e95fc6 Mon Sep 17 00:00:00 2001 From: Du Chuan Date: Sun, 28 Apr 2019 14:16:36 +0800 Subject: [PATCH 11/49] go.mod, ddl: update go.mod and fix related tests (#10283) --- ddl/db_integration_test.go | 2 +- go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 171a5128b27b4..81e35dc563745 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -551,7 +551,7 @@ func (s *testIntegrationSuite7) TestNullGeneratedColumn(c *C) { tk.MustExec("CREATE TABLE `t` (" + "`a` int(11) DEFAULT NULL," + "`b` int(11) DEFAULT NULL," + - "`c` int(11) GENERATED ALWAYS AS (`a` + `b`) VIRTUAL DEFAULT NULL," + + "`c` int(11) GENERATED ALWAYS AS (`a` + `b`) VIRTUAL," + "`h` varchar(10) DEFAULT NULL," + "`m` int(11) DEFAULT NULL" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin") diff --git a/go.mod b/go.mod index 4392ba138def8..9954b80d60b62 100644 --- a/go.mod +++ b/go.mod @@ -39,7 +39,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190424024541-e2cdb851bce2 + github.com/pingcap/parser v0.0.0-20190427000002-f3ecae036b23 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 diff --git a/go.sum b/go.sum index 26440dba9819c..fb39cc3a73a73 100644 --- a/go.sum +++ b/go.sum @@ -160,8 +160,8 @@ github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190424024541-e2cdb851bce2 h1:WZPcqMEpW1PFsusiCEiNlatZdI8dGURXneIUrFNmehI= -github.com/pingcap/parser v0.0.0-20190424024541-e2cdb851bce2/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190427000002-f3ecae036b23 h1:9RUvFHNfDplW4KZ90FMOC7QEDLORa15AlozhX0P8cCg= +github.com/pingcap/parser v0.0.0-20190427000002-f3ecae036b23/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= From 4b4fe8b549639d03f51cf7cbf89851429056628c Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Sun, 28 Apr 2019 15:34:52 +0800 Subject: [PATCH 12/49] stats: incremental analyze for index without feedback updates (#10102) --- executor/analyze.go | 154 ++++++++++++++++++++++++++++------- executor/analyze_test.go | 18 ++++ executor/builder.go | 81 ++++++++++++++---- planner/core/common_plans.go | 1 + planner/core/planbuilder.go | 27 ++++-- statistics/cmsketch.go | 5 ++ statistics/histogram.go | 45 ++++++++++ 7 files changed, 276 insertions(+), 55 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index 12dde20fe0df1..5dc307ec1a99b 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -16,6 +16,7 @@ package executor import ( "bytes" "context" + "math" "math/rand" "runtime" "sort" @@ -146,14 +147,18 @@ const ( colTask taskType = iota idxTask fastTask + pkIncrementalTask + idxIncrementalTask ) type analyzeTask struct { - taskType taskType - idxExec *AnalyzeIndexExec - colExec *AnalyzeColumnsExec - fastExec *AnalyzeFastExec - job *statistics.AnalyzeJob + taskType taskType + idxExec *AnalyzeIndexExec + colExec *AnalyzeColumnsExec + fastExec *AnalyzeFastExec + idxIncrementalExec *analyzeIndexIncrementalExec + colIncrementalExec *analyzePKIncrementalExec + job *statistics.AnalyzeJob } var errAnalyzeWorkerPanic = errors.New("analyze worker panic") @@ -181,14 +186,13 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<- if !ok { break } + task.job.Start() switch task.taskType { case colTask: task.colExec.job = task.job - task.job.Start() resultCh <- analyzeColumnsPushdown(task.colExec) case idxTask: task.idxExec.job = task.job - task.job.Start() resultCh <- analyzeIndexPushdown(task.idxExec) case fastTask: task.fastExec.job = task.job @@ -196,12 +200,27 @@ func (e *AnalyzeExec) analyzeWorker(taskCh <-chan *analyzeTask, resultCh chan<- for _, result := range analyzeFastExec(task.fastExec) { resultCh <- result } + case pkIncrementalTask: + task.colIncrementalExec.job = task.job + resultCh <- analyzePKIncremental(task.colIncrementalExec) + case idxIncrementalTask: + task.idxIncrementalExec.job = task.job + resultCh <- analyzeIndexIncremental(task.idxIncrementalExec) } } } func analyzeIndexPushdown(idxExec *AnalyzeIndexExec) analyzeResult { - hist, cms, err := idxExec.buildStats() + ranges := ranger.FullRange() + // For single-column index, we do not load null rows from TiKV, so the built histogram would not include + // null values, and its `NullCount` would be set by result of another distsql call to get null rows. + // For multi-column index, we cannot define null for the rows, so we still use full range, and the rows + // containing null fields would exist in built histograms. Note that, the `NullCount` of histograms for + // multi-column index is always 0 then. + if len(idxExec.idxInfo.Columns) == 1 { + ranges = ranger.FullNotNullRange() + } + hist, cms, err := idxExec.buildStats(ranges, true) if err != nil { return analyzeResult{Err: err, job: idxExec.job} } @@ -260,21 +279,12 @@ func (e *AnalyzeIndexExec) fetchAnalyzeResult(ranges []*ranger.Range, isNullRang return nil } -func (e *AnalyzeIndexExec) open() error { - ranges := ranger.FullRange() - // For single-column index, we do not load null rows from TiKV, so the built histogram would not include - // null values, and its `NullCount` would be set by result of another distsql call to get null rows. - // For multi-column index, we cannot define null for the rows, so we still use full range, and the rows - // containing null fields would exist in built histograms. Note that, the `NullCount` of histograms for - // multi-column index is always 0 then. - if len(e.idxInfo.Columns) == 1 { - ranges = ranger.FullNotNullRange() - } +func (e *AnalyzeIndexExec) open(ranges []*ranger.Range, considerNull bool) error { err := e.fetchAnalyzeResult(ranges, false) if err != nil { return err } - if len(e.idxInfo.Columns) == 1 { + if considerNull && len(e.idxInfo.Columns) == 1 { ranges = ranger.NullRange() err = e.fetchAnalyzeResult(ranges, true) if err != nil { @@ -323,8 +333,8 @@ func (e *AnalyzeIndexExec) buildStatsFromResult(result distsql.SelectResult, nee return hist, cms, nil } -func (e *AnalyzeIndexExec) buildStats() (hist *statistics.Histogram, cms *statistics.CMSketch, err error) { - if err = e.open(); err != nil { +func (e *AnalyzeIndexExec) buildStats(ranges []*ranger.Range, considerNull bool) (hist *statistics.Histogram, cms *statistics.CMSketch, err error) { + if err = e.open(ranges, considerNull); err != nil { return nil, nil, err } defer func() { @@ -348,7 +358,13 @@ func (e *AnalyzeIndexExec) buildStats() (hist *statistics.Histogram, cms *statis } func analyzeColumnsPushdown(colExec *AnalyzeColumnsExec) analyzeResult { - hists, cms, err := colExec.buildStats() + var ranges []*ranger.Range + if colExec.pkInfo != nil { + ranges = ranger.FullIntRange(mysql.HasUnsignedFlag(colExec.pkInfo.Flag)) + } else { + ranges = ranger.FullIntRange(false) + } + hists, cms, err := colExec.buildStats(ranges) if err != nil { return analyzeResult{Err: err, job: colExec.job} } @@ -380,13 +396,7 @@ type AnalyzeColumnsExec struct { job *statistics.AnalyzeJob } -func (e *AnalyzeColumnsExec) open() error { - var ranges []*ranger.Range - if e.pkInfo != nil { - ranges = ranger.FullIntRange(mysql.HasUnsignedFlag(e.pkInfo.Flag)) - } else { - ranges = ranger.FullIntRange(false) - } +func (e *AnalyzeColumnsExec) open(ranges []*ranger.Range) error { e.resultHandler = &tableResultHandler{} firstPartRanges, secondPartRanges := splitRanges(ranges, true, false) firstResult, err := e.buildResp(firstPartRanges) @@ -428,8 +438,8 @@ func (e *AnalyzeColumnsExec) buildResp(ranges []*ranger.Range) (distsql.SelectRe return result, nil } -func (e *AnalyzeColumnsExec) buildStats() (hists []*statistics.Histogram, cms []*statistics.CMSketch, err error) { - if err = e.open(); err != nil { +func (e *AnalyzeColumnsExec) buildStats(ranges []*ranger.Range) (hists []*statistics.Histogram, cms []*statistics.CMSketch, err error) { + if err = e.open(ranges); err != nil { return nil, nil, err } defer func() { @@ -1066,6 +1076,88 @@ func (e *AnalyzeTestFastExec) TestFastSample() error { return err } +type analyzeIndexIncrementalExec struct { + AnalyzeIndexExec + index *statistics.Index +} + +func analyzeIndexIncremental(idxExec *analyzeIndexIncrementalExec) analyzeResult { + idx := idxExec.index + highBound := idx.Histogram.GetUpper(idx.Len() - 1) + values, err := codec.Decode(highBound.GetBytes(), len(idxExec.idxInfo.Columns)) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + ran := ranger.Range{LowVal: values, HighVal: []types.Datum{types.MaxValueDatum()}} + hist, cms, err := idxExec.buildStats([]*ranger.Range{&ran}, false) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + oldHist, oldCMS, err := idx.RemoveUpperBound(idxExec.ctx.GetSessionVars().StmtCtx, values) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + hist, err = statistics.MergeHistograms(idxExec.ctx.GetSessionVars().StmtCtx, oldHist, hist, int(idxExec.maxNumBuckets)) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + if oldCMS != nil && cms != nil { + err = cms.MergeCMSketch(oldCMS) + if err != nil { + return analyzeResult{Err: err, job: idxExec.job} + } + } + result := analyzeResult{ + PhysicalTableID: idxExec.physicalTableID, + Hist: []*statistics.Histogram{hist}, + Cms: []*statistics.CMSketch{cms}, + IsIndex: 1, + job: idxExec.job, + } + result.Count = hist.NullCount + if hist.Len() > 0 { + result.Count += hist.Buckets[hist.Len()-1].Count + } + return result +} + +type analyzePKIncrementalExec struct { + AnalyzeColumnsExec + pkStats *statistics.Column +} + +func analyzePKIncremental(colExec *analyzePKIncrementalExec) analyzeResult { + pkStats := colExec.pkStats + high := pkStats.GetUpper(pkStats.Len() - 1) + var maxVal types.Datum + if mysql.HasUnsignedFlag(colExec.pkInfo.Flag) { + maxVal = types.NewUintDatum(math.MaxUint64) + } else { + maxVal = types.NewIntDatum(math.MaxInt64) + } + ran := ranger.Range{LowVal: []types.Datum{*high}, LowExclude: true, HighVal: []types.Datum{maxVal}} + hists, _, err := colExec.buildStats([]*ranger.Range{&ran}) + if err != nil { + return analyzeResult{Err: err, job: colExec.job} + } + hist := hists[0] + oldHist := pkStats.Histogram.Copy() + hist, err = statistics.MergeHistograms(colExec.ctx.GetSessionVars().StmtCtx, oldHist, hist, int(colExec.maxNumBuckets)) + if err != nil { + return analyzeResult{Err: err, job: colExec.job} + } + result := analyzeResult{ + PhysicalTableID: colExec.physicalTableID, + Hist: []*statistics.Histogram{hist}, + Cms: []*statistics.CMSketch{nil}, + job: colExec.job, + } + if hist.Len() > 0 { + result.Count += hist.Buckets[hist.Len()-1].Count + } + return result +} + // analyzeResult is used to represent analyze result. type analyzeResult struct { // PhysicalTableID is the id of a partition or a table. diff --git a/executor/analyze_test.go b/executor/analyze_test.go index e4407e88e2df3..2510a1e9a0f09 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -282,3 +282,21 @@ func (s *testSuite1) TestFastAnalyze(c *C) { } c.Assert(matched, Equals, true) } + +func (s *testSuite1) TestAnalyzeIncremental(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int, b int, primary key(a), index idx(b))") + tk.MustExec("analyze incremental table t index") + tk.MustQuery("show stats_buckets").Check(testkit.Rows()) + tk.MustExec("insert into t values (1,1)") + tk.MustExec("analyze incremental table t index") + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t idx 1 0 1 1 1 1")) + tk.MustExec("insert into t values (2,2)") + tk.MustExec("analyze incremental table t index") + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2")) + tk.MustExec("analyze incremental table t index") + // Result should not change. + tk.MustQuery("show stats_buckets").Check(testkit.Rows("test t a 0 0 1 1 1 1", "test t a 0 1 2 1 2 2", "test t idx 1 0 1 1 1 1", "test t idx 1 1 2 1 2 2")) +} diff --git a/executor/builder.go b/executor/builder.go index 11de4897155ac..fbd9d25444607 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1351,7 +1351,7 @@ func (b *executorBuilder) buildDelete(v *plannercore.Delete) Executor { return deleteExec } -func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeIndexTask, maxNumBuckets uint64) *AnalyzeIndexExec { +func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeIndexTask, maxNumBuckets uint64, autoAnalyze string) *analyzeTask { _, offset := timeutil.Zone(b.ctx.GetSessionVars().Location()) e := &AnalyzeIndexExec{ ctx: b.ctx, @@ -1374,10 +1374,35 @@ func (b *executorBuilder) buildAnalyzeIndexPushdown(task plannercore.AnalyzeInde width := int32(defaultCMSketchWidth) e.analyzePB.IdxReq.CmsketchDepth = &depth e.analyzePB.IdxReq.CmsketchWidth = &width - return e + job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze index " + task.IndexInfo.Name.O} + return &analyzeTask{taskType: idxTask, idxExec: e, job: job} } -func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, maxNumBuckets uint64) *AnalyzeColumnsExec { +func (b *executorBuilder) buildAnalyzeIndexIncremental(task plannercore.AnalyzeIndexTask, maxNumBuckets uint64) *analyzeTask { + h := domain.GetDomain(b.ctx).StatsHandle() + statsTbl := h.GetPartitionStats(&model.TableInfo{}, task.PhysicalTableID) + analyzeTask := b.buildAnalyzeIndexPushdown(task, maxNumBuckets, "") + if statsTbl.Pseudo { + return analyzeTask + } + idx, ok := statsTbl.Indices[task.IndexInfo.ID] + // TODO: If the index contains feedback, we may use other strategy. + if !ok || idx.Len() == 0 || idx.ContainsFeedback() { + return analyzeTask + } + exec := analyzeTask.idxExec + if idx.CMSketch != nil { + width, depth := idx.CMSketch.GetWidthAndDepth() + exec.analyzePB.IdxReq.CmsketchWidth = &width + exec.analyzePB.IdxReq.CmsketchDepth = &depth + } + analyzeTask.taskType = idxIncrementalTask + analyzeTask.idxIncrementalExec = &analyzeIndexIncrementalExec{AnalyzeIndexExec: *analyzeTask.idxExec, index: idx} + analyzeTask.job = &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: "analyze incremental index " + task.IndexInfo.Name.O} + return analyzeTask +} + +func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeColumnsTask, maxNumBuckets uint64, autoAnalyze string) *analyzeTask { cols := task.ColsInfo if task.PKInfo != nil { cols = append([]*model.ColumnInfo{task.PKInfo}, cols...) @@ -1409,7 +1434,27 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeCo CmsketchWidth: &width, } b.err = plannercore.SetPBColumnsDefaultValue(b.ctx, e.analyzePB.ColReq.ColumnsInfo, cols) - return e + job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze columns"} + return &analyzeTask{taskType: colTask, colExec: e, job: job} +} + +func (b *executorBuilder) buildAnalyzePKIncremental(task plannercore.AnalyzeColumnsTask, maxNumBuckets uint64) *analyzeTask { + h := domain.GetDomain(b.ctx).StatsHandle() + statsTbl := h.GetPartitionStats(&model.TableInfo{}, task.PhysicalTableID) + analyzeTask := b.buildAnalyzeColumnsPushdown(task, maxNumBuckets, "") + if statsTbl.Pseudo { + return analyzeTask + } + col, ok := statsTbl.Columns[task.PKInfo.ID] + // TODO: If the primary key contains feedback, we may use other strategy. + if !ok || col.Len() == 0 || col.ContainsFeedback() { + return analyzeTask + } + exec := analyzeTask.colExec + analyzeTask.taskType = pkIncrementalTask + analyzeTask.colIncrementalExec = &analyzePKIncrementalExec{AnalyzeColumnsExec: *exec, pkStats: col} + analyzeTask.job = &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: "analyze incremental primary key"} + return analyzeTask } func (b *executorBuilder) buildAnalyzeFastColumn(e *AnalyzeExec, task plannercore.AnalyzeColumnsTask, maxNumBuckets uint64) { @@ -1487,28 +1532,28 @@ func (b *executorBuilder) buildAnalyze(v *plannercore.Analyze) Executor { autoAnalyze = "auto " } for _, task := range v.ColTasks { - if enableFastAnalyze { - b.buildAnalyzeFastColumn(e, task, v.MaxNumBuckets) + if task.Incremental { + e.tasks = append(e.tasks, b.buildAnalyzePKIncremental(task, v.MaxNumBuckets)) } else { - e.tasks = append(e.tasks, &analyzeTask{ - taskType: colTask, - colExec: b.buildAnalyzeColumnsPushdown(task, v.MaxNumBuckets), - job: &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze columns"}, - }) + if enableFastAnalyze { + b.buildAnalyzeFastColumn(e, task, v.MaxNumBuckets) + } else { + e.tasks = append(e.tasks, b.buildAnalyzeColumnsPushdown(task, v.MaxNumBuckets, autoAnalyze)) + } } if b.err != nil { return nil } } for _, task := range v.IdxTasks { - if enableFastAnalyze { - b.buildAnalyzeFastIndex(e, task, v.MaxNumBuckets) + if task.Incremental { + e.tasks = append(e.tasks, b.buildAnalyzeIndexIncremental(task, v.MaxNumBuckets)) } else { - e.tasks = append(e.tasks, &analyzeTask{ - taskType: idxTask, - idxExec: b.buildAnalyzeIndexPushdown(task, v.MaxNumBuckets), - job: &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze index " + task.IndexInfo.Name.O}, - }) + if enableFastAnalyze { + b.buildAnalyzeFastIndex(e, task, v.MaxNumBuckets) + } else { + e.tasks = append(e.tasks, b.buildAnalyzeIndexPushdown(task, v.MaxNumBuckets, autoAnalyze)) + } } if b.err != nil { return nil diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 0c63590a7e507..ea467b3ef637d 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -427,6 +427,7 @@ type analyzeInfo struct { PartitionName string // PhysicalTableID is the id for a partition or a table. PhysicalTableID int64 + Incremental bool } // AnalyzeColumnsTask is used for analyze columns. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 2d231b07dadc4..1bde08d6f95b5 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -436,13 +436,13 @@ func getPathByIndexName(paths []*accessPath, idxName model.CIStr, tblInfo *model return path } } - if isPrimaryIndexHint(idxName) && tblInfo.PKIsHandle { + if isPrimaryIndex(idxName) && tblInfo.PKIsHandle { return tablePath } return nil } -func isPrimaryIndexHint(indexName model.CIStr) bool { +func isPrimaryIndex(indexName model.CIStr) bool { return indexName.L == "primary" } @@ -808,7 +808,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) (Plan, error) } for _, idx := range idxInfo { for i, id := range physicalIDs { - info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id} + info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{ IndexInfo: idx, analyzeInfo: info, @@ -818,7 +818,7 @@ func (b *PlanBuilder) buildAnalyzeTable(as *ast.AnalyzeTableStmt) (Plan, error) } if len(colInfo) > 0 || pkInfo != nil { for i, id := range physicalIDs { - info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id} + info := analyzeInfo{DBName: tbl.Schema.O, TableName: tbl.Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{ PKInfo: pkInfo, ColsInfo: colInfo, @@ -839,12 +839,20 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt) (Plan, error) return nil, err } for _, idxName := range as.IndexNames { + if isPrimaryIndex(idxName) && tblInfo.PKIsHandle { + pkCol := tblInfo.GetPkColInfo() + for i, id := range physicalIDs { + info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} + p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{PKInfo: pkCol, analyzeInfo: info}) + } + continue + } idx := tblInfo.FindIndexByName(idxName.L) if idx == nil || idx.State != model.StatePublic { return nil, ErrAnalyzeMissIndex.GenWithStackByArgs(idxName.O, tblInfo.Name.O) } for i, id := range physicalIDs { - info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id} + info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo}) } } @@ -861,11 +869,18 @@ func (b *PlanBuilder) buildAnalyzeAllIndex(as *ast.AnalyzeTableStmt) (Plan, erro for _, idx := range tblInfo.Indices { if idx.State == model.StatePublic { for i, id := range physicalIDs { - info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id} + info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} p.IdxTasks = append(p.IdxTasks, AnalyzeIndexTask{IndexInfo: idx, analyzeInfo: info, TblInfo: tblInfo}) } } } + if tblInfo.PKIsHandle { + pkCol := tblInfo.GetPkColInfo() + for i, id := range physicalIDs { + info := analyzeInfo{DBName: as.TableNames[0].Schema.O, TableName: as.TableNames[0].Name.O, PartitionName: names[i], PhysicalTableID: id, Incremental: as.Incremental} + p.ColTasks = append(p.ColTasks, AnalyzeColumnsTask{PKInfo: pkCol, analyzeInfo: info}) + } + } return p, nil } diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index e82ed041e1e11..07e6ebfc6e501 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -377,3 +377,8 @@ func (c *CMSketch) Copy() *CMSketch { } return &CMSketch{count: c.count, width: c.width, depth: c.depth, table: tbl} } + +// GetWidthAndDepth returns the width and depth of CM Sketch. +func (c *CMSketch) GetWidthAndDepth() (int32, int32) { + return c.width, c.depth +} diff --git a/statistics/histogram.go b/statistics/histogram.go index 2642ccd6dd520..31b70eabecd79 100644 --- a/statistics/histogram.go +++ b/statistics/histogram.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tipb/go-tipb" + "github.com/spaolacci/murmur3" "go.uber.org/zap" ) @@ -565,6 +566,28 @@ func (hg *Histogram) outOfRange(val types.Datum) bool { chunk.Compare(hg.Bounds.GetRow(hg.Bounds.NumRows()-1), 0, &val) < 0 } +// ContainsFeedback checks if the histogram contains feedback updates. +// We can test it from the `repeat` field because only feedback will update it to 0. +func (hg *Histogram) ContainsFeedback() bool { + for _, bkt := range hg.Buckets { + if bkt.Repeat == 0 { + return true + } + } + return false +} + +// Copy deep copies the histogram. +func (hg *Histogram) Copy() *Histogram { + newHist := *hg + newHist.Bounds = hg.Bounds.CopyConstruct() + newHist.Buckets = make([]Bucket, 0, len(hg.Buckets)) + for _, bkt := range hg.Buckets { + newHist.Buckets = append(newHist.Buckets, bkt) + } + return &newHist +} + // ErrorRate is the error rate of estimate row count by bucket and cm sketch. type ErrorRate struct { ErrorTotal float64 @@ -967,6 +990,28 @@ func (idx *Index) outOfRange(val types.Datum) bool { return !withInLowBoundOrPrefixMatch || !withInHighBound } +// RemoveUpperBound removes the upper bound the index stats. +// It is used when merge stats for incremental analyze. +func (idx *Index) RemoveUpperBound(sc *stmtctx.StatementContext, values []types.Datum) (*Histogram, *CMSketch, error) { + hist, cms := idx.Histogram.Copy(), idx.CMSketch.Copy() + hist.Buckets[hist.Len()-1].Count -= hist.Buckets[hist.Len()-1].Repeat + hist.Buckets[hist.Len()-1].Repeat = 0 + if cms == nil { + return hist, nil, nil + } + var data []byte + var err error + for _, val := range values { + data, err = codec.EncodeKey(sc, data, val) + if err != nil { + return nil, nil, err + } + h1, h2 := murmur3.Sum128(data) + cms.setValue(h1, h2, 0) + } + return hist, cms, nil +} + // matchPrefix checks whether ad is the prefix of value func matchPrefix(row chunk.Row, colIdx int, ad *types.Datum) bool { switch ad.Kind() { From 515b9476d98278b61d4b2a94dc3045b254df6b12 Mon Sep 17 00:00:00 2001 From: liyuzhou <2541781827@qq.com> Date: Sun, 28 Apr 2019 17:32:42 +0800 Subject: [PATCH 13/49] *: support drop session binding (#10287) --- bindinfo/bind_test.go | 6 ++++++ bindinfo/handle.go | 30 +++++++++++++++++------------- bindinfo/session_handle.go | 9 +++++++++ executor/bind.go | 9 +++++---- 4 files changed, 37 insertions(+), 17 deletions(-) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 76d93dc476277..9b764b9c6f12f 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -273,4 +273,10 @@ func (s *testSuite) TestSessionBinding(c *C) { c.Check(row.GetString(6), NotNil) c.Check(row.GetString(7), NotNil) + _, err = tk.Exec("drop session binding for select * from t where i>99") + c.Assert(err, IsNil) + bindData = handle.GetBindRecord("select * from t where i > ?", "test") + c.Check(bindData, NotNil) + c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") + c.Check(bindData.Status, Equals, "deleted") } diff --git a/bindinfo/handle.go b/bindinfo/handle.go index a6b54e937219d..f0ba865a779eb 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -196,7 +196,7 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { return } - hash, meta := h.newBindMetaWithoutAst(record) + hash, meta := newBindMetaWithoutAst(record) h.removeBindMeta(hash, meta) }() @@ -226,18 +226,9 @@ func (h *BindHandle) Size() int { return size } -// GetBindRecord return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. +// GetBindRecord return the bindMeta of the (normdOrigSQL,db) if bindMeta exist. func (h *BindHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { - hash := parser.DigestHash(normdOrigSQL) - bindRecords := h.bindInfo.Load().(cache)[hash] - if bindRecords != nil { - for _, bindRecord := range bindRecords { - if bindRecord.OriginalSQL == normdOrigSQL && bindRecord.Db == db { - return bindRecord - } - } - } - return nil + return h.bindInfo.Load().(cache).getBindRecord(normdOrigSQL, db) } // GetAllBindRecord return all bind record in cache. @@ -259,7 +250,7 @@ func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMet return hash, meta, nil } -func (h *BindHandle) newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) { +func newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) { hash = parser.DigestHash(record.OriginalSQL) meta = &BindMeta{BindRecord: record} return hash, meta @@ -337,6 +328,19 @@ func (c cache) copy() cache { return newCache } +func (c cache) getBindRecord(normdOrigSQL, db string) *BindMeta { + hash := parser.DigestHash(normdOrigSQL) + bindRecords := c[hash] + if bindRecords != nil { + for _, bindRecord := range bindRecords { + if bindRecord.OriginalSQL == normdOrigSQL && bindRecord.Db == db { + return bindRecord + } + } + } + return nil +} + // isStale checks whether this BindMeta is stale compared with the other BindMeta. func (m *BindMeta) isStale(other *BindMeta) bool { return m.OriginalSQL == other.OriginalSQL && m.Db == other.Db && diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index 1bc9b2e6c4d18..90b4d8ac3c457 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -70,6 +70,15 @@ func (h *SessionHandle) AddBindRecord(record *BindRecord) error { return err } +// DropBindRecord drops a BindRecord in the cache. +func (h *SessionHandle) DropBindRecord(record *BindRecord) { + meta := &BindMeta{BindRecord: record} + meta.Status = deleted + hash := parser.DigestHash(record.OriginalSQL) + h.ch.removeDeletedBindMeta(hash, meta) + h.appendBindMeta(hash, meta) +} + // GetBindRecord return the BindMeta of the (normdOrigSQL,db) if BindMeta exist. func (h *SessionHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { hash := parser.DigestHash(normdOrigSQL) diff --git a/executor/bind.go b/executor/bind.go index f23ba3cc1bb41..2e3ad0f8aa326 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -57,14 +57,15 @@ func (e *SQLBindExec) Next(ctx context.Context, req *chunk.RecordBatch) error { } func (e *SQLBindExec) dropSQLBind() error { - if !e.isGlobal { - return errors.New("drop non-global sql bind is not supported") - } - record := &bindinfo.BindRecord{ OriginalSQL: e.normdOrigSQL, Db: e.ctx.GetSessionVars().CurrentDB, } + if !e.isGlobal { + handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + handle.DropBindRecord(record) + return nil + } return domain.GetDomain(e.ctx).BindHandle().DropBindRecord(record) } From 98dbbffceb6642604726a9fa3ad0b5a5f8ddd388 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Sun, 28 Apr 2019 17:47:02 +0800 Subject: [PATCH 14/49] executor: fix IsPointGet judgment condition (#10278) index lookup should not the max ts optimization --- executor/adapter.go | 16 ++++++++-------- executor/executor_test.go | 33 ++++++++++++++++++++++++++++++++- executor/point_get.go | 19 +++++++++++++++++++ go.mod | 1 + go.sum | 2 ++ util/testkit/testkit.go | 7 ++++++- 6 files changed, 68 insertions(+), 10 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 983e071661ab8..01e005d7dd0f4 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -316,11 +316,11 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { if _, ok := a.Plan.(*plannercore.Execute); !ok { // Do not sync transaction for Execute statement, because the real optimization work is done in // "ExecuteExec.Build". - isPointGet, err := IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx, a.Plan) + useMaxTS, err := IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx, a.Plan) if err != nil { return nil, err } - if isPointGet { + if useMaxTS { logutil.Logger(context.Background()).Debug("init txnStartTS with MaxUint64", zap.Uint64("conn", ctx.GetSessionVars().ConnectionID), zap.String("text", a.Text)) err = ctx.InitTxnWithStartTS(math.MaxUint64) } else if ctx.GetSessionVars().SnapshotTS != 0 { @@ -335,7 +335,7 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) { stmtCtx := ctx.GetSessionVars().StmtCtx if stmtPri := stmtCtx.Priority; stmtPri == mysql.NoPriority { switch { - case isPointGet: + case useMaxTS: stmtCtx.Priority = kv.PriorityHigh case a.Expensive: stmtCtx.Priority = kv.PriorityLow @@ -472,7 +472,7 @@ func (a *ExecStmt) getStatsInfo() map[string]uint64 { // IsPointGetWithPKOrUniqueKeyByAutoCommit returns true when meets following conditions: // 1. ctx is auto commit tagged // 2. txn is not valid -// 2. plan is point get by pk or unique key +// 2. plan is point get by pk, or point get by unique index (no double read) func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p plannercore.Plan) (bool, error) { // check auto commit if !ctx.GetSessionVars().IsAutocommit() { @@ -500,14 +500,14 @@ func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p plannerco case *plannercore.PhysicalIndexReader: indexScan := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) return indexScan.IsPointGetByUniqueKey(ctx.GetSessionVars().StmtCtx), nil - case *plannercore.PhysicalIndexLookUpReader: - indexScan := v.IndexPlans[0].(*plannercore.PhysicalIndexScan) - return indexScan.IsPointGetByUniqueKey(ctx.GetSessionVars().StmtCtx), nil case *plannercore.PhysicalTableReader: tableScan := v.TablePlans[0].(*plannercore.PhysicalTableScan) return len(tableScan.Ranges) == 1 && tableScan.Ranges[0].IsPoint(ctx.GetSessionVars().StmtCtx), nil case *plannercore.PointGetPlan: - return true, nil + // If the PointGetPlan needs to read data using unique index (double read), we + // can't use max uint64, because using math.MaxUint64 can't guarantee repeatable-read + // and the data and index would be inconsistent! + return v.IndexInfo == nil, nil default: return false, nil } diff --git a/executor/executor_test.go b/executor/executor_test.go index 60707de717b93..9f24b23c0206a 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -29,6 +29,7 @@ import ( "github.com/golang/protobuf/proto" . "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" "github.com/pingcap/parser" @@ -66,6 +67,7 @@ import ( "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" + "github.com/tiancaiamao/debugger" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -1853,7 +1855,8 @@ func (s *testSuite) TestIsPointGet(c *C) { tk.MustExec("use mysql") ctx := tk.Se.(sessionctx.Context) tests := map[string]bool{ - "select * from help_topic where name='aaa'": true, + "select * from help_topic where name='aaa'": false, + "select 1 from help_topic where name='aaa'": true, "select * from help_topic where help_topic_id=1": true, "select * from help_topic where help_category_id=1": false, } @@ -1872,6 +1875,34 @@ func (s *testSuite) TestIsPointGet(c *C) { } } +func (s *testSuite) TestPointGetRepeatableRead(c *C) { + c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/pointGetRepeatableReadTest", `return(true)`), IsNil) + defer failpoint.Disable("github.com/pingcap/tidb/executor/pointGetRepeatableReadTest") + + tk1 := testkit.NewTestKit(c, s.store) + tk1.MustExec("use test") + tk1.MustExec(`create table point_get (a int, b int, c int, + primary key k_a(a), + unique key k_b(b))`) + tk1.MustExec("insert into point_get values (1, 1, 1)") + tk2 := testkit.NewTestKit(c, s.store) + tk2.MustExec("use test") + + go func() { + ctx := context.WithValue(context.Background(), "pointGetRepeatableReadTest", true) + rs, err := tk1.Se.Execute(ctx, "select c from point_get where b = 1") + c.Assert(err, IsNil) + result := tk1.ResultSetToResultWithCtx(ctx, rs[0], Commentf("execute sql fail")) + result.Check(testkit.Rows("1")) + }() + + label := debugger.Bind("point-get-g2") + debugger.Continue("point-get-g1") + debugger.Breakpoint(label) + tk2.MustExec("update point_get set b = 2, c = 2 where a = 1") + debugger.Continue("point-get-g1") +} + func (s *testSuite) TestRow(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/executor/point_get.go b/executor/point_get.go index 1d170b8dfe087..17f76e6079ae9 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -16,6 +16,7 @@ package executor import ( "context" + "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" @@ -28,6 +29,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/tiancaiamao/debugger" ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { @@ -88,6 +90,14 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.RecordBatch) err if err1 != nil { return err1 } + + failpoint.Inject("pointGetRepeatableReadTest", func(val failpoint.Value) { + if val.(bool) && ctx.Value("pointGetRepeatableReadTest") != nil { + label := debugger.Bind("point-get-g1") + debugger.Breakpoint(label) + } + }) + handleVal, err1 := e.get(idxKey) if err1 != nil && !kv.ErrNotExist.Equal(err1) { return err1 @@ -99,7 +109,16 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.RecordBatch) err if err1 != nil { return err1 } + + failpoint.Inject("pointGetRepeatableReadTest", func(val failpoint.Value) { + if val.(bool) && ctx.Value("pointGetRepeatableReadTest") != nil { + label := debugger.Bind("point-get-g1") + debugger.Continue("point-get-g2") + debugger.Breakpoint(label) + } + }) } + key := tablecodec.EncodeRowKeyWithHandle(e.tblInfo.ID, e.handle) val, err := e.get(key) if err != nil && !kv.ErrNotExist.Equal(err) { diff --git a/go.mod b/go.mod index 9954b80d60b62..ff60cdb589e76 100644 --- a/go.mod +++ b/go.mod @@ -55,6 +55,7 @@ require ( github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 github.com/struCoder/pidusage v0.1.2 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 + github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22 github.com/twinj/uuid v1.0.0 github.com/uber-go/atomic v1.3.2 // indirect github.com/uber/jaeger-client-go v2.15.0+incompatible diff --git a/go.sum b/go.sum index fb39cc3a73a73..4dc2ec528b2a5 100644 --- a/go.sum +++ b/go.sum @@ -215,6 +215,8 @@ github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d h1:4J9HCZVpvDmj2t github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d/go.mod h1:Z4AUp2Km+PwemOoO/VB5AOx9XSsIItzFjoJlOSiYmn0= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= +github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22 h1:P4sgavMKEdqNOws2VfR2c/Bye9nYFgV8gHyiW1wpQhE= +github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22/go.mod h1:qaShs3uDBYnvaQZJAJ6PjPg8kuAHR9zUJ8ilSLK1y/w= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6 h1:lYIiVDtZnyTWlNwiAxLj0bbpTcx1BWCFhXjfsvmPdNc= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 2eaf3302cbebb..d9e28ec531194 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -208,7 +208,12 @@ func (tk *TestKit) ExecToErr(sql string, args ...interface{}) error { // ResultSetToResult converts sqlexec.RecordSet to testkit.Result. // It is used to check results of execute statement in binary mode. func (tk *TestKit) ResultSetToResult(rs sqlexec.RecordSet, comment check.CommentInterface) *Result { - rows, err := session.GetRows4Test(context.Background(), tk.Se, rs) + return tk.ResultSetToResultWithCtx(context.Background(), rs, comment) +} + +// ResultSetToResultWithCtx converts sqlexec.RecordSet to testkit.Result. +func (tk *TestKit) ResultSetToResultWithCtx(ctx context.Context, rs sqlexec.RecordSet, comment check.CommentInterface) *Result { + rows, err := session.GetRows4Test(ctx, tk.Se, rs) tk.c.Assert(errors.ErrorStack(err), check.Equals, "", comment) err = rs.Close() tk.c.Assert(errors.ErrorStack(err), check.Equals, "", comment) From 9d74d6404f492c5b6f38f0c42511dbbec80cc5a3 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 29 Apr 2019 11:07:15 +0800 Subject: [PATCH 15/49] planner, executor: index join enhancement (#8471) --- cmd/explaintest/r/explain_complex.result | 6 +- .../r/explain_complex_stats.result | 6 +- cmd/explaintest/r/explain_easy.result | 2 +- cmd/explaintest/r/index_join.result | 4 +- cmd/explaintest/r/topn_push_down.result | 8 +- cmd/explaintest/r/tpch.result | 20 +- executor/builder.go | 71 ++- executor/executor_pkg_test.go | 20 +- executor/index_lookup_join.go | 72 ++- executor/index_lookup_join_test.go | 6 +- planner/core/cbo_test.go | 4 +- planner/core/exhaust_physical_plans.go | 506 ++++++++++++++---- planner/core/exhaust_physical_plans_test.go | 238 ++++++++ planner/core/explain.go | 4 +- planner/core/find_best_task.go | 3 +- planner/core/physical_plans.go | 9 +- planner/core/resolve_indices.go | 13 + planner/core/rule_partition_processor.go | 3 +- statistics/selectivity.go | 3 +- util/ranger/detacher.go | 14 +- util/ranger/ranger.go | 20 +- util/ranger/ranger_test.go | 3 +- 22 files changed, 831 insertions(+), 204 deletions(-) create mode 100644 planner/core/exhaust_physical_plans_test.go diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index f8b87a8ded5fe..e23a148310563 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -120,7 +120,7 @@ Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.d └─HashAgg_19 1.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(test.dd.ip, gad.ip), gt(test.dd.t, gad.t) ├─IndexLookUp_23 0.00 root - │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false, stats:pseudo + │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false, stats:pseudo │ └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) │ └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo └─IndexLookUp_33 3.33 root @@ -137,7 +137,7 @@ Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, │ └─Selection_26 0.00 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), not(isnull(gad.dic)) │ └─TableScan_25 3333.33 cop table:st, keep order:false, stats:pseudo └─IndexLookUp_17 0.00 root - ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [gad.aid gad.dic], keep order:false, stats:pseudo + ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false, stats:pseudo └─Selection_16 0.00 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) └─TableScan_15 10.00 cop table:dd, keep order:false, stats:pseudo explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; @@ -157,7 +157,7 @@ Projection_10 0.00 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr. │ └─Selection_40 0.00 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) │ └─TableScan_39 10000.00 cop table:dt, range:[0,+inf], keep order:false, stats:pseudo └─IndexLookUp_18 3.33 root - ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false, stats:pseudo + ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false, stats:pseudo └─Selection_17 3.33 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 10.00 cop table:rr, keep order:false, stats:pseudo explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index 830fcaba61da5..5db97abb7f63e 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -133,7 +133,7 @@ Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test │ └─Selection_28 424.00 cop eq(gad.bm, 0), eq(gad.pt, "android"), gt(gad.t, 1478143908), not(isnull(gad.ip)) │ └─TableScan_27 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_23 455.80 root - ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [gad.aid gad.ip], keep order:false + ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false └─Selection_22 455.80 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) └─TableScan_21 1.00 cop table:dd, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; @@ -145,7 +145,7 @@ Projection_10 170.34 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk. │ └─Selection_22 170.34 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), gt(gad.t, 1477971479), not(isnull(gad.dic)) │ └─TableScan_21 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_17 509.04 root - ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [gad.aid gad.dic], keep order:false + ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false └─Selection_16 509.04 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) └─TableScan_15 1.00 cop table:dd, keep order:false explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; @@ -165,7 +165,7 @@ Projection_10 428.32 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, r │ └─Selection_40 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) │ └─TableScan_39 2000.00 cop table:dt, range:[0,+inf], keep order:false └─IndexLookUp_18 970.00 root - ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [dt.aid dt.dic], keep order:false + ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false └─Selection_17 970.00 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) └─TableScan_16 1.00 cop table:rr, keep order:false explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 3a64d978d4b8d..1497fb03490de 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -47,7 +47,7 @@ IndexJoin_12 4166.67 root left outer join, inner:IndexLookUp_11, outer key:test. │ └─TableScan_23 3333.33 cop table:t1, range:(1,+inf], keep order:false, stats:pseudo └─IndexLookUp_11 0.00 root ├─Selection_10 0.00 cop not(isnull(test.t2.c1)) - │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [test.t1.c2], keep order:false, stats:pseudo + │ └─IndexScan_8 10.00 cop table:t2, index:c1, range: decided by [eq(test.t2.c1, test.t1.c2)], keep order:false, stats:pseudo └─TableScan_9 0.00 cop table:t2, keep order:false, stats:pseudo explain update t1 set t1.c2 = 2 where t1.c1 = 1; id count task operator info diff --git a/cmd/explaintest/r/index_join.result b/cmd/explaintest/r/index_join.result index 07d177671c292..6d5555bc8993e 100644 --- a/cmd/explaintest/r/index_join.result +++ b/cmd/explaintest/r/index_join.result @@ -9,7 +9,7 @@ id count task operator info IndexJoin_16 5.00 root inner join, inner:IndexLookUp_15, outer key:test.t2.a, inner key:test.t1.a ├─IndexLookUp_15 0.00 root │ ├─Selection_14 0.00 cop not(isnull(test.t1.a)) -│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [test.t2.a], keep order:false +│ │ └─IndexScan_12 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false │ └─TableScan_13 0.00 cop table:t1, keep order:false, stats:pseudo └─TableReader_19 1.00 root data:Selection_18 └─Selection_18 1.00 cop not(isnull(test.t2.a)) @@ -23,5 +23,5 @@ Projection_6 5.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b │ └─TableScan_28 1.00 cop table:t2, range:[-inf,+inf], keep order:false └─IndexLookUp_11 0.00 root ├─Selection_10 0.00 cop not(isnull(test.t1.a)) - │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [test.t2.a], keep order:false + │ └─IndexScan_8 5.00 cop table:t1, index:a, range: decided by [eq(test.t1.a, test.t2.a)], keep order:false └─TableScan_9 0.00 cop table:t1, keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index 132df50fc2f19..e8ba90678635a 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -178,12 +178,12 @@ Projection_13 0.00 root te.expect_time │ │ └─Selection_73 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) │ │ └─TableScan_71 0.00 cop table:tr, keep order:false, stats:pseudo │ └─IndexLookUp_35 250.00 root - │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [tr.id], keep order:false, stats:pseudo + │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [eq(te.trade_id, tr.id)], keep order:false, stats:pseudo │ └─Selection_34 250.00 cop ge(te.expect_time, 2018-04-23 00:00:00.000000), le(te.expect_time, 2018-04-23 23:59:59.000000) │ └─TableScan_33 10.00 cop table:te, keep order:false, stats:pseudo └─IndexReader_91 0.00 root index:Selection_90 └─Selection_90 0.00 cop not(isnull(p.relate_id)) - └─IndexScan_89 10.00 cop table:p, index:relate_id, range: decided by [tr.id], keep order:false, stats:pseudo + └─IndexScan_89 10.00 cop table:p, index:relate_id, range: decided by [eq(p.relate_id, tr.id)], keep order:false, stats:pseudo desc select 1 as a from dual order by a limit 1; id count task operator info Projection_6 1.00 root 1 @@ -226,7 +226,7 @@ Limit_11 5.00 root offset:0, count:5 ├─TableReader_17 4.00 root data:TableScan_16 │ └─TableScan_16 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_14 10.00 root index:IndexScan_13 - └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [t1.a], keep order:false, stats:pseudo + └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo explain select /*+ TIDB_INLJ(t2) */ * from t t1 left join t t2 on t1.a = t2.a where t2.a is null limit 5; id count task operator info Limit_12 5.00 root offset:0, count:5 @@ -235,7 +235,7 @@ Limit_12 5.00 root offset:0, count:5 ├─TableReader_19 4.00 root data:TableScan_18 │ └─TableScan_18 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_16 10.00 root index:IndexScan_15 - └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [t1.a], keep order:false, stats:pseudo + └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo explain select /*+ TIDB_SMJ(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a limit 5; id count task operator info Limit_11 5.00 root offset:0, count:5 diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index 5ee157d771853..a083977411ea6 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -261,7 +261,7 @@ Projection_14 10.00 root tpch.lineitem.l_orderkey, 7_col_0, tpch.orders.o_orderd │ └─Selection_51 36870000.00 cop lt(tpch.orders.o_orderdate, 1995-03-13 00:00:00.000000) │ └─TableScan_50 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_28 162945114.27 root - ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_25 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_27 162945114.27 cop gt(tpch.lineitem.l_shipdate, 1995-03-13 00:00:00.000000) └─TableScan_26 1.00 cop table:lineitem, keep order:false /* @@ -302,7 +302,7 @@ Sort_10 1.00 root tpch.orders.o_orderpriority:asc │ └─Selection_32 2925937.50 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1995-04-01) │ └─TableScan_31 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_20 240004648.80 root - ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_17 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_19 240004648.80 cop lt(tpch.lineitem.l_commitdate, tpch.lineitem.l_receiptdate) └─TableScan_18 1.00 cop table:lineitem, keep order:false /* @@ -538,7 +538,7 @@ Sort_29 718.01 root all_nations.o_year:asc │ │ │ │ └─Selection_77 22382008.93 cop ge(tpch.orders.o_orderdate, 1995-01-01 00:00:00.000000), le(tpch.orders.o_orderdate, 1996-12-31 00:00:00.000000) │ │ │ │ └─TableScan_76 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false │ │ │ └─IndexLookUp_55 1.00 root - │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + │ │ │ ├─IndexScan_53 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false │ │ │ └─TableScan_54 1.00 cop table:lineitem, keep order:false │ │ └─TableReader_83 61674.00 root data:Selection_82 │ │ └─Selection_82 61674.00 cop eq(tpch.part.p_type, "SMALL PLATED COPPER") @@ -612,7 +612,7 @@ Sort_25 2406.00 root profit.nation:asc, profit.o_year:desc │ └─TableReader_40 1.00 root data:TableScan_39 │ └─TableScan_39 1.00 cop table:orders, range: decided by [tpch.lineitem.l_orderkey], keep order:false └─IndexLookUp_34 1.00 root - ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.lineitem.l_suppkey tpch.lineitem.l_partkey], keep order:false + ├─IndexScan_32 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.lineitem.l_partkey) eq(tpch.partsupp.ps_suppkey, tpch.lineitem.l_suppkey)], keep order:false └─TableScan_33 1.00 cop table:partsupp, keep order:false /* Q10 Returned Item Reporting Query @@ -673,7 +673,7 @@ Projection_17 20.00 root tpch.customer.c_custkey, tpch.customer.c_name, 9_col_0, │ └─Selection_47 3017307.69 cop ge(tpch.orders.o_orderdate, 1993-08-01 00:00:00.000000), lt(tpch.orders.o_orderdate, 1993-11-01) │ └─TableScan_46 75000000.00 cop table:orders, range:[-inf,+inf], keep order:false └─IndexLookUp_31 73916005.00 root - ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_28 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─Selection_30 73916005.00 cop eq(tpch.lineitem.l_returnflag, "R") └─TableScan_29 1.00 cop table:lineitem, keep order:false /* @@ -936,7 +936,7 @@ Sort_13 3863988.24 root supplier_cnt:desc, tpch.part.p_brand:asc, tpch.part.p_ty │ │ └─Selection_40 1200618.43 cop in(tpch.part.p_size, 48, 19, 12, 4, 41, 7, 21, 39), ne(tpch.part.p_brand, "Brand#34"), not(like(tpch.part.p_type, "LARGE BRUSHED%", 92)) │ │ └─TableScan_39 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexReader_26 1.00 root index:IndexScan_25 - │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ └─IndexScan_25 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)], keep order:false └─TableReader_46 400000.00 root data:Selection_45 └─Selection_45 400000.00 cop like(tpch.supplier.s_comment, "%Customer%Complaints%", 92) └─TableScan_44 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false @@ -1042,7 +1042,7 @@ Projection_24 100.00 root tpch.customer.c_name, tpch.customer.c_custkey, tpch.or │ └─HashAgg_53 74063872.00 cop group by:tpch.lineitem.l_orderkey, funcs:sum(tpch.lineitem.l_quantity), firstrow(tpch.lineitem.l_orderkey) │ └─TableScan_58 300005811.00 cop table:lineitem, range:[-inf,+inf], keep order:false └─IndexLookUp_37 1.00 root - ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [tpch.orders.o_orderkey], keep order:false + ├─IndexScan_35 1.00 cop table:lineitem, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.lineitem.l_orderkey, tpch.orders.o_orderkey)], keep order:false └─TableScan_36 1.00 cop table:lineitem, keep order:false /* Q19 Discounted Revenue Query @@ -1165,7 +1165,7 @@ Sort_28 20000.00 root tpch.supplier.s_name:asc │ │ └─Selection_74 80007.93 cop like(tpch.part.p_name, "green%", 92) │ │ └─TableScan_73 10000000.00 cop table:part, range:[-inf,+inf], keep order:false │ └─IndexLookUp_58 1.00 root - │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [tpch.part.p_partkey], keep order:false + │ ├─IndexScan_56 1.00 cop table:partsupp, index:PS_PARTKEY, PS_SUPPKEY, range: decided by [eq(tpch.partsupp.ps_partkey, tpch.part.p_partkey)], keep order:false │ └─TableScan_57 1.00 cop table:partsupp, keep order:false └─TableReader_80 44189356.65 root data:Selection_79 └─Selection_79 44189356.65 cop ge(tpch.lineitem.l_shipdate, 1993-01-01 00:00:00.000000), lt(tpch.lineitem.l_shipdate, 1994-01-01) @@ -1239,10 +1239,10 @@ Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 │ │ └─Selection_60 0.80 cop eq(tpch.orders.o_orderstatus, "F") │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false │ └─IndexLookUp_55 1.00 root - │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l2.l_orderkey, l1.l_orderkey)], keep order:false │ └─TableScan_54 1.00 cop table:lineitem, keep order:false └─IndexLookUp_39 240004648.80 root - ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [l1.l_orderkey], keep order:false + ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l3.l_orderkey, l1.l_orderkey)], keep order:false └─Selection_38 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) └─TableScan_37 1.00 cop table:lineitem, keep order:false /* diff --git a/executor/builder.go b/executor/builder.go index fbd9d25444607..596b64c72b48c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -39,7 +39,6 @@ import ( "github.com/pingcap/tidb/metrics" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -1674,6 +1673,7 @@ func (b *executorBuilder) buildIndexLookUpJoin(v *plannercore.PhysicalIndexJoin) isOuterJoin: v.JoinType.IsOuterJoin(), indexRanges: v.Ranges, keyOff2IdxOff: v.KeyOff2IdxOff, + lastColHelper: v.CompareFilters, } outerKeyCols := make([]int, len(v.OuterJoinKeys)) for i := 0; i < len(v.OuterJoinKeys); i++ { @@ -1912,25 +1912,25 @@ type dataReaderBuilder struct { selectResultHook // for testing } -func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, datums [][]types.Datum, - IndexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { +func (builder *dataReaderBuilder) buildExecutorForIndexJoin(ctx context.Context, lookUpContents []*indexJoinLookUpContent, + IndexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { switch v := builder.Plan.(type) { case *plannercore.PhysicalTableReader: - return builder.buildTableReaderForIndexJoin(ctx, v, datums) + return builder.buildTableReaderForIndexJoin(ctx, v, lookUpContents) case *plannercore.PhysicalIndexReader: - return builder.buildIndexReaderForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildIndexReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) case *plannercore.PhysicalIndexLookUpReader: - return builder.buildIndexLookUpReaderForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildIndexLookUpReaderForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) case *plannercore.PhysicalUnionScan: - return builder.buildUnionScanForIndexJoin(ctx, v, datums, IndexRanges, keyOff2IdxOff) + return builder.buildUnionScanForIndexJoin(ctx, v, lookUpContents, IndexRanges, keyOff2IdxOff, cwc) } return nil, errors.New("Wrong plan type for dataReaderBuilder") } func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context, v *plannercore.PhysicalUnionScan, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + values []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { childBuilder := &dataReaderBuilder{Plan: v.Children()[0], executorBuilder: builder.executorBuilder} - reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff) + reader, err := childBuilder.buildExecutorForIndexJoin(ctx, values, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, err } @@ -1943,14 +1943,14 @@ func (builder *dataReaderBuilder) buildUnionScanForIndexJoin(ctx context.Context return us, nil } -func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, datums [][]types.Datum) (Executor, error) { +func (builder *dataReaderBuilder) buildTableReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalTableReader, lookUpContents []*indexJoinLookUpContent) (Executor, error) { e, err := buildNoRangeTableReader(builder.executorBuilder, v) if err != nil { return nil, err } - handles := make([]int64, 0, len(datums)) - for _, datum := range datums { - handles = append(handles, datum[0].GetInt64()) + handles := make([]int64, 0, len(lookUpContents)) + for _, content := range lookUpContents { + handles = append(handles, content.keys[0].GetInt64()) } return builder.buildTableReaderFromHandles(ctx, e, handles) } @@ -1984,12 +1984,12 @@ func (builder *dataReaderBuilder) buildTableReaderFromHandles(ctx context.Contex } func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexReader, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { e, err := buildNoRangeIndexReader(builder.executorBuilder, v) if err != nil { return nil, err } - kvRanges, err := buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, e.physicalTableID, e.index.ID, values, indexRanges, keyOff2IdxOff) + kvRanges, err := buildKvRangesForIndexJoin(e.ctx, e.physicalTableID, e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, err } @@ -1998,12 +1998,12 @@ func (builder *dataReaderBuilder) buildIndexReaderForIndexJoin(ctx context.Conte } func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context.Context, v *plannercore.PhysicalIndexLookUpReader, - values [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) (Executor, error) { + lookUpContents []*indexJoinLookUpContent, indexRanges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) (Executor, error) { e, err := buildNoRangeIndexLookUpReader(builder.executorBuilder, v) if err != nil { return nil, err } - e.kvRanges, err = buildKvRangesForIndexJoin(e.ctx.GetSessionVars().StmtCtx, getPhysicalTableID(e.table), e.index.ID, values, indexRanges, keyOff2IdxOff) + e.kvRanges, err = buildKvRangesForIndexJoin(e.ctx, getPhysicalTableID(e.table), e.index.ID, lookUpContents, indexRanges, keyOff2IdxOff, cwc) if err != nil { return nil, err } @@ -2012,17 +2012,40 @@ func (builder *dataReaderBuilder) buildIndexLookUpReaderForIndexJoin(ctx context } // buildKvRangesForIndexJoin builds kv ranges for index join when the inner plan is index scan plan. -func buildKvRangesForIndexJoin(sc *stmtctx.StatementContext, tableID, indexID int64, keyDatums [][]types.Datum, indexRanges []*ranger.Range, keyOff2IdxOff []int) ([]kv.KeyRange, error) { - kvRanges := make([]kv.KeyRange, 0, len(indexRanges)*len(keyDatums)) - for _, val := range keyDatums { - for _, ran := range indexRanges { +func buildKvRangesForIndexJoin(ctx sessionctx.Context, tableID, indexID int64, lookUpContents []*indexJoinLookUpContent, + ranges []*ranger.Range, keyOff2IdxOff []int, cwc *plannercore.ColWithCmpFuncManager) ([]kv.KeyRange, error) { + kvRanges := make([]kv.KeyRange, 0, len(ranges)*len(lookUpContents)) + lastPos := len(ranges[0].LowVal) - 1 + sc := ctx.GetSessionVars().StmtCtx + for _, content := range lookUpContents { + for _, ran := range ranges { for keyOff, idxOff := range keyOff2IdxOff { - ran.LowVal[idxOff] = val[keyOff] - ran.HighVal[idxOff] = val[keyOff] + ran.LowVal[idxOff] = content.keys[keyOff] + ran.HighVal[idxOff] = content.keys[keyOff] } } + if cwc != nil { + nextColRanges, err := cwc.BuildRangesByRow(ctx, content.row) + if err != nil { + return nil, err + } + for _, nextColRan := range nextColRanges { + for _, ran := range ranges { + ran.LowVal[lastPos] = nextColRan.LowVal[0] + ran.HighVal[lastPos] = nextColRan.HighVal[0] + ran.LowExclude = nextColRan.LowExclude + ran.HighExclude = nextColRan.HighExclude + } + tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, ranges, nil) + if err != nil { + return nil, errors.Trace(err) + } + kvRanges = append(kvRanges, tmpKvRanges...) + } + continue + } - tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, indexRanges, nil) + tmpKvRanges, err := distsql.IndexRangesToKVRanges(sc, tableID, indexID, ranges, nil) if err != nil { return nil, err } diff --git a/executor/executor_pkg_test.go b/executor/executor_pkg_test.go index c2c274455c7e2..b237236e9b31b 100644 --- a/executor/executor_pkg_test.go +++ b/executor/executor_pkg_test.go @@ -15,7 +15,6 @@ package executor import ( "context" - "time" . "github.com/pingcap/check" "github.com/pingcap/parser/ast" @@ -23,7 +22,6 @@ import ( "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" @@ -137,7 +135,7 @@ func buildSchema(names []string, ftypes []byte) *expression.Schema { return schema } -func (s *testExecSuite) TestBuildKvRangesForIndexJoin(c *C) { +func (s *testExecSuite) TestBuildKvRangesForIndexJoinWithoutCwc(c *C) { indexRanges := make([]*ranger.Range, 0, 6) indexRanges = append(indexRanges, generateIndexRange(1, 1, 1, 1, 1)) indexRanges = append(indexRanges, generateIndexRange(1, 1, 2, 1, 1)) @@ -146,16 +144,16 @@ func (s *testExecSuite) TestBuildKvRangesForIndexJoin(c *C) { indexRanges = append(indexRanges, generateIndexRange(2, 1, 1, 1, 1)) indexRanges = append(indexRanges, generateIndexRange(2, 1, 2, 1, 1)) - joinKeyRows := make([][]types.Datum, 0, 5) - joinKeyRows = append(joinKeyRows, generateDatumSlice(1, 1)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(1, 2)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 1)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 2)) - joinKeyRows = append(joinKeyRows, generateDatumSlice(2, 3)) + joinKeyRows := make([]*indexJoinLookUpContent, 0, 5) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(1, 1)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(1, 2)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 1)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 2)}) + joinKeyRows = append(joinKeyRows, &indexJoinLookUpContent{keys: generateDatumSlice(2, 3)}) keyOff2IdxOff := []int{1, 3} - sc := &stmtctx.StatementContext{TimeZone: time.Local} - kvRanges, err := buildKvRangesForIndexJoin(sc, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff) + ctx := mock.NewContext() + kvRanges, err := buildKvRangesForIndexJoin(ctx, 0, 0, joinKeyRows, indexRanges, keyOff2IdxOff, nil) c.Assert(err, IsNil) // Check the kvRanges is in order. for i, kvRange := range kvRanges { diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index e49baf3c6290a..ac30d73745219 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" + plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -73,6 +74,9 @@ type IndexLookUpJoin struct { keyOff2IdxOff []int innerPtrBytes [][]byte + // lastColHelper store the information for last col if there's complicated filter like col > x_col and col < x_col + 100. + lastColHelper *plannercore.ColWithCmpFuncManager + memTracker *memory.Tracker // track memory usage. } @@ -132,8 +136,9 @@ type innerWorker struct { ctx sessionctx.Context executorChk *chunk.Chunk - indexRanges []*ranger.Range - keyOff2IdxOff []int + indexRanges []*ranger.Range + nextColCompareFilters *plannercore.ColWithCmpFuncManager + keyOff2IdxOff []int } // Open implements the Executor interface. @@ -209,13 +214,14 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork copiedRanges = append(copiedRanges, ran.Clone()) } iw := &innerWorker{ - innerCtx: e.innerCtx, - outerCtx: e.outerCtx, - taskCh: taskCh, - ctx: e.ctx, - executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), - indexRanges: copiedRanges, - keyOff2IdxOff: e.keyOff2IdxOff, + innerCtx: e.innerCtx, + outerCtx: e.outerCtx, + taskCh: taskCh, + ctx: e.ctx, + executorChk: chunk.NewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize), + indexRanges: copiedRanges, + keyOff2IdxOff: e.keyOff2IdxOff, + nextColCompareFilters: e.lastColHelper, } return iw } @@ -447,13 +453,18 @@ func (iw *innerWorker) run(ctx context.Context, wg *sync.WaitGroup) { } } +type indexJoinLookUpContent struct { + keys []types.Datum + row chunk.Row +} + func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) error { - dLookUpKeys, err := iw.constructDatumLookupKeys(task) + lookUpContents, err := iw.constructLookupContent(task) if err != nil { return err } - dLookUpKeys = iw.sortAndDedupDatumLookUpKeys(dLookUpKeys) - err = iw.fetchInnerResults(ctx, task, dLookUpKeys) + lookUpContents = iw.sortAndDedupLookUpContents(lookUpContents) + err = iw.fetchInnerResults(ctx, task, lookUpContents) if err != nil { return err } @@ -464,8 +475,8 @@ func (iw *innerWorker) handleTask(ctx context.Context, task *lookUpJoinTask) err return nil } -func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([][]types.Datum, error) { - dLookUpKeys := make([][]types.Datum, 0, task.outerResult.NumRows()) +func (iw *innerWorker) constructLookupContent(task *lookUpJoinTask) ([]*indexJoinLookUpContent, error) { + lookUpContents := make([]*indexJoinLookUpContent, 0, task.outerResult.NumRows()) keyBuf := make([]byte, 0, 64) for i := 0; i < task.outerResult.NumRows(); i++ { dLookUpKey, err := iw.constructDatumLookupKey(task, i) @@ -484,11 +495,11 @@ func (iw *innerWorker) constructDatumLookupKeys(task *lookUpJoinTask) ([][]types } // Store the encoded lookup key in chunk, so we can use it to lookup the matched inners directly. task.encodedLookUpKeys.AppendBytes(0, keyBuf) - dLookUpKeys = append(dLookUpKeys, dLookUpKey) + lookUpContents = append(lookUpContents, &indexJoinLookUpContent{keys: dLookUpKey, row: task.outerResult.GetRow(i)}) } task.memTracker.Consume(task.encodedLookUpKeys.MemoryUsage()) - return dLookUpKeys, nil + return lookUpContents, nil } func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, rowIdx int) ([]types.Datum, error) { @@ -525,20 +536,23 @@ func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, rowIdx int) return dLookupKey, nil } -func (iw *innerWorker) sortAndDedupDatumLookUpKeys(dLookUpKeys [][]types.Datum) [][]types.Datum { - if len(dLookUpKeys) < 2 { - return dLookUpKeys +func (iw *innerWorker) sortAndDedupLookUpContents(lookUpContents []*indexJoinLookUpContent) []*indexJoinLookUpContent { + if len(lookUpContents) < 2 { + return lookUpContents } sc := iw.ctx.GetSessionVars().StmtCtx - sort.Slice(dLookUpKeys, func(i, j int) bool { - cmp := compareRow(sc, dLookUpKeys[i], dLookUpKeys[j]) - return cmp < 0 + sort.Slice(lookUpContents, func(i, j int) bool { + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[j].keys) + if cmp != 0 || iw.nextColCompareFilters == nil { + return cmp < 0 + } + return iw.nextColCompareFilters.CompareRow(lookUpContents[i].row, lookUpContents[j].row) < 0 }) - deDupedLookupKeys := dLookUpKeys[:1] - for i := 1; i < len(dLookUpKeys); i++ { - cmp := compareRow(sc, dLookUpKeys[i], dLookUpKeys[i-1]) - if cmp != 0 { - deDupedLookupKeys = append(deDupedLookupKeys, dLookUpKeys[i]) + deDupedLookupKeys := lookUpContents[:1] + for i := 1; i < len(lookUpContents); i++ { + cmp := compareRow(sc, lookUpContents[i].keys, lookUpContents[i-1].keys) + if cmp != 0 || (iw.nextColCompareFilters != nil && iw.nextColCompareFilters.CompareRow(lookUpContents[i].row, lookUpContents[i-1].row) != 0) { + deDupedLookupKeys = append(deDupedLookupKeys, lookUpContents[i]) } } return deDupedLookupKeys @@ -558,8 +572,8 @@ func compareRow(sc *stmtctx.StatementContext, left, right []types.Datum) int { return 0 } -func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTask, dLookUpKeys [][]types.Datum) error { - innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, dLookUpKeys, iw.indexRanges, iw.keyOff2IdxOff) +func (iw *innerWorker) fetchInnerResults(ctx context.Context, task *lookUpJoinTask, lookUpContent []*indexJoinLookUpContent) error { + innerExec, err := iw.readerBuilder.buildExecutorForIndexJoin(ctx, lookUpContent, iw.indexRanges, iw.keyOff2IdxOff, iw.nextColCompareFilters) if err != nil { return err } diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 2eb60cc747856..0df5fa6055edd 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -70,7 +70,7 @@ func (s *testSuite1) TestIndexJoinUnionScan(c *C) { "└─UnionScan_12 0.00 root not(isnull(test.t2.a))", " └─IndexLookUp_11 0.00 root ", " ├─Selection_10 0.00 cop not(isnull(test.t2.a))", - " │ └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [test.t1.a], keep order:false, stats:pseudo", + " │ └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", " └─TableScan_9 0.00 cop table:t2, keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ * from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( @@ -88,7 +88,7 @@ func (s *testSuite1) TestIndexJoinUnionScan(c *C) { " └─UnionScan_11 0.00 root not(isnull(test.t2.a))", " └─IndexReader_10 0.00 root index:Selection_9", " └─Selection_9 0.00 cop not(isnull(test.t2.a))", - " └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [test.t1.a], keep order:false, stats:pseudo", + " └─IndexScan_8 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ t1.a, t2.a from t1 join t2 on t1.a = t2.a").Check(testkit.Rows( "2 2", @@ -117,7 +117,7 @@ func (s *testSuite1) TestBatchIndexJoinUnionScan(c *C) { " └─UnionScan_26 0.00 root not(isnull(test.t2.a))", " └─IndexReader_25 0.00 root index:Selection_24", " └─Selection_24 0.00 cop not(isnull(test.t2.a))", - " └─IndexScan_23 10.00 cop table:t2, index:a, range: decided by [test.t1.a], keep order:false, stats:pseudo", + " └─IndexScan_23 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo", )) tk.MustQuery("select /*+ TIDB_INLJ(t1, t2)*/ count(*) from t1 join t2 on t1.a = t2.id").Check(testkit.Rows( "4", diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index eef47f0f053ae..3a739fb9028c5 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -910,7 +910,7 @@ func (s *testAnalyzeSuite) TestIssue9562(c *C) { "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", "└─IndexReader_8 0.00 root index:Selection_7", " └─Selection_7 0.00 cop not(isnull(test.t2.a)), not(isnull(test.t2.c))", - " └─IndexScan_6 10.00 cop table:t2, index:a, b, c, range: decided by [test.t1.a test.t1.c], keep order:false, stats:pseudo", + " └─IndexScan_6 10.00 cop table:t2, index:a, b, c, range: decided by [eq(test.t2.a, test.t1.a) gt(test.t2.b, minus(test.t1.b, 1)) lt(test.t2.b, plus(test.t1.b, 1))], keep order:false, stats:pseudo", )) tk.MustExec("create table t(a int, b int, index idx_ab(a, b))") @@ -1036,7 +1036,7 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { " │ └─TopN_30 1.00 cop t1.a:asc, offset:0, count:1", " │ └─IndexScan_29 6.00 cop table:t1, index:b, range:[-inf,6], keep order:false", " └─IndexReader_57 1.04 root index:IndexScan_56", - " └─IndexScan_56 1.04 cop table:t2, index:b, range: decided by [t1.a], keep order:false", + " └─IndexScan_56 1.04 cop table:t2, index:b, range: decided by [eq(t2.b, t1.a)], keep order:false", )) // Desc TableScan. tk.MustExec("truncate table t") diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index 6c5d08e5140a5..e8076fbf96477 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -14,20 +14,25 @@ package core import ( + "bytes" + "context" "fmt" "math" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/expression/aggregation" "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/set" + "go.uber.org/zap" ) func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) []PhysicalPlan { @@ -317,10 +322,21 @@ func joinKeysMatchIndex(keys, indexCols []*expression.Column, colLengths []int) // When inner plan is TableReader, the parameter `ranges` will be nil. Because pk only have one column. So all of its range // is generated during execution time. -func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, innerJoinKeys, outerJoinKeys []*expression.Column, outerIdx int, - innerPlan PhysicalPlan, ranges []*ranger.Range, keyOff2IdxOff []int) []PhysicalPlan { +func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, outerIdx int, innerPlan PhysicalPlan, + ranges []*ranger.Range, keyOff2IdxOff []int, compareFilters *ColWithCmpFuncManager) []PhysicalPlan { joinType := p.JoinType outerSchema := p.children[outerIdx].Schema() + var ( + innerJoinKeys []*expression.Column + outerJoinKeys []*expression.Column + ) + if outerIdx == 0 { + outerJoinKeys = p.LeftJoinKeys + innerJoinKeys = p.RightJoinKeys + } else { + innerJoinKeys = p.LeftJoinKeys + outerJoinKeys = p.RightJoinKeys + } all, _ := prop.AllSameOrder() // If the order by columns are not all from outer child, index join cannot promise the order. if !prop.AllColsFromSchema(outerSchema) || !all { @@ -358,6 +374,7 @@ func (p *LogicalJoin) constructIndexJoin(prop *property.PhysicalProperty, innerJ innerPlan: innerPlan, KeyOff2IdxOff: newKeyOff, Ranges: ranges, + CompareFilters: compareFilters, }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), chReqProps...) join.SetSchema(p.schema) return []PhysicalPlan{join} @@ -414,40 +431,77 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou innerPlan := p.constructInnerTableScan(ds, pkCol, outerJoinKeys, us) // Since the primary key means one value corresponding to exact one row, this will always be a no worse one // comparing to other index. - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, nil, keyOff2IdxOff) + return p.constructIndexJoin(prop, outerIdx, innerPlan, nil, keyOff2IdxOff, nil) } } - var ( - bestIndexInfo *model.IndexInfo - rangesOfBest []*ranger.Range - maxUsedCols int - remainedOfBest []expression.Expression - keyOff2IdxOff []int - ) + helper := &indexJoinBuildHelper{join: p} for _, path := range ds.possibleAccessPaths { if path.isTablePath { continue } indexInfo := path.index - ranges, remained, tmpKeyOff2IdxOff := p.buildRangeForIndexJoin(indexInfo, ds, innerJoinKeys) - // We choose the index by the number of used columns of the range, the much the better. - // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. - // But obviously when the range is nil, we don't need index join. - if len(ranges) > 0 && len(ranges[0].LowVal) > maxUsedCols { - bestIndexInfo = indexInfo - maxUsedCols = len(ranges[0].LowVal) - rangesOfBest = ranges - remainedOfBest = remained - keyOff2IdxOff = tmpKeyOff2IdxOff - } - } - if bestIndexInfo != nil { - innerPlan := p.constructInnerIndexScan(ds, bestIndexInfo, remainedOfBest, outerJoinKeys, us) - return p.constructIndexJoin(prop, innerJoinKeys, outerJoinKeys, outerIdx, innerPlan, rangesOfBest, keyOff2IdxOff) + err := helper.analyzeLookUpFilters(indexInfo, ds, innerJoinKeys) + if err != nil { + logutil.Logger(context.Background()).Warn("build index join failed", zap.Error(err)) + } + } + if helper.chosenIndexInfo != nil { + keyOff2IdxOff := make([]int, len(innerJoinKeys)) + for i := range keyOff2IdxOff { + keyOff2IdxOff[i] = -1 + } + for idxOff, keyOff := range helper.idxOff2KeyOff { + if keyOff != -1 { + keyOff2IdxOff[keyOff] = idxOff + } + } + idxCols, _ := expression.IndexInfo2Cols(ds.schema.Columns, helper.chosenIndexInfo) + rangeInfo := helper.buildRangeDecidedByInformation(idxCols, outerJoinKeys) + innerPlan := p.constructInnerIndexScan(ds, helper.chosenIndexInfo, helper.chosenRemained, outerJoinKeys, us, rangeInfo) + return p.constructIndexJoin(prop, outerIdx, innerPlan, helper.chosenRanges, keyOff2IdxOff, helper.lastColManager) } return nil } +type indexJoinBuildHelper struct { + join *LogicalJoin + + chosenIndexInfo *model.IndexInfo + maxUsedCols int + chosenAccess []expression.Expression + chosenRemained []expression.Expression + idxOff2KeyOff []int + lastColManager *ColWithCmpFuncManager + chosenRanges []*ranger.Range + + curPossibleUsedKeys []*expression.Column + curNotUsedIndexCols []*expression.Column + curNotUsedColLens []int + curIdxOff2KeyOff []int +} + +func (ijHelper *indexJoinBuildHelper) buildRangeDecidedByInformation(idxCols []*expression.Column, outerJoinKeys []*expression.Column) string { + buffer := bytes.NewBufferString("[") + isFirst := true + for idxOff, keyOff := range ijHelper.idxOff2KeyOff { + if keyOff == -1 { + continue + } + if !isFirst { + buffer.WriteString(" ") + } else { + isFirst = false + } + buffer.WriteString(fmt.Sprintf("eq(%v, %v)", idxCols[idxOff], outerJoinKeys[keyOff])) + } + for _, access := range ijHelper.chosenAccess { + // Since now there must be eq/in condition so here we can just append space directly. + buffer.WriteString(fmt.Sprintf(" %v", access)) + } + buffer.WriteString("]") + return buffer.String() +} + // constructInnerTableScan is specially used to construct the inner plan for PhysicalIndexJoin. func (p *LogicalJoin) constructInnerTableScan(ds *DataSource, pk *expression.Column, outerJoinKeys []*expression.Column, us *LogicalUnionScan) PhysicalPlan { ranges := ranger.FullIntRange(mysql.HasUnsignedFlag(pk.RetType.Flag)) @@ -491,7 +545,8 @@ func (p *LogicalJoin) constructInnerUnionScan(us *LogicalUnionScan, reader Physi } // constructInnerIndexScan is specially used to construct the inner plan for PhysicalIndexJoin. -func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexInfo, remainedConds []expression.Expression, outerJoinKeys []*expression.Column, us *LogicalUnionScan) PhysicalPlan { +func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexInfo, filterConds []expression.Expression, + outerJoinKeys []*expression.Column, us *LogicalUnionScan, rangeInfo string) PhysicalPlan { is := PhysicalIndexScan{ Table: ds.tableInfo, TableAsName: ds.TableAsName, @@ -501,9 +556,8 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn dataSourceSchema: ds.schema, KeepOrder: false, Ranges: ranger.FullRange(), - rangeDecidedBy: outerJoinKeys, + rangeInfo: rangeInfo, }.Init(ds.ctx) - is.filterCondition = remainedConds var rowCount float64 idxHist, ok := ds.statisticTable.Indices[idx.ID] @@ -529,7 +583,7 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn } is.initSchema(ds.id, idx, cop.tablePlan != nil) - indexConds, tblConds := splitIndexFilterConditions(remainedConds, idx.Columns, ds.tableInfo) + indexConds, tblConds := splitIndexFilterConditions(filterConds, idx.Columns, ds.tableInfo) path := &accessPath{indexFilters: indexConds, tableFilters: tblConds, countAfterIndex: math.MaxFloat64} is.addPushedDownSelection(cop, ds, math.MaxFloat64, path) t := finishCopTask(ds.ctx, cop) @@ -537,92 +591,360 @@ func (p *LogicalJoin) constructInnerIndexScan(ds *DataSource, idx *model.IndexIn return p.constructInnerUnionScan(us, reader) } -// buildRangeForIndexJoin checks whether this index can be used for building index join and return the range if this index is ok. -// If this index is invalid, just return nil range. -func (p *LogicalJoin) buildRangeForIndexJoin(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) ( - []*ranger.Range, []expression.Expression, []int) { - idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.Schema().Columns, indexInfo) - if len(idxCols) == 0 { - return nil, nil, nil - } +var symmetricOp = map[string]string{ + ast.LT: ast.GT, + ast.GE: ast.LE, + ast.GT: ast.LT, + ast.LE: ast.GE, +} - // Extract the filter to calculate access and the filters that must be remained ones. - access, eqConds, remained, keyOff2IdxOff := p.buildFakeEqCondsForIndexJoin(innerJoinKeys, idxCols, colLengths, innerPlan.pushedDownConds) +// ColWithCmpFuncManager is used in index join to handle the column with compare functions(>=, >, <, <=). +// It stores the compare functions and build ranges in execution phase. +type ColWithCmpFuncManager struct { + targetCol *expression.Column + colLength int + OpType []string + opArg []expression.Expression + tmpConstant []*expression.Constant + affectedColSchema *expression.Schema + compareFuncs []chunk.CompareFunc +} - if len(keyOff2IdxOff) == 0 { - return nil, nil, nil +func (cwc *ColWithCmpFuncManager) appendNewExpr(opName string, arg expression.Expression, affectedCols []*expression.Column) { + cwc.OpType = append(cwc.OpType, opName) + cwc.opArg = append(cwc.opArg, arg) + cwc.tmpConstant = append(cwc.tmpConstant, &expression.Constant{RetType: cwc.targetCol.RetType}) + for _, col := range affectedCols { + if cwc.affectedColSchema.Contains(col) { + continue + } + cwc.compareFuncs = append(cwc.compareFuncs, chunk.GetCompareFunc(col.RetType)) + cwc.affectedColSchema.Append(col) } +} - // In `buildFakeEqCondsForIndexJoin`, we construct the equal conditions for join keys and remove filters that contain the join keys' column. - // When t1.a = t2.a and t1.a > 1, we can also guarantee that t1.a > 1 won't be chosen as the access condition. - // So the equal conditions we built can be successfully used to build a range if they can be used. They won't be affected by the existing filters. - res, err := ranger.DetachCondAndBuildRangeForIndex(p.ctx, access, idxCols, colLengths) - if err != nil { - terror.Log(err) - return nil, nil, nil +// CompareRow compares the rows for deduplicate. +func (cwc *ColWithCmpFuncManager) CompareRow(lhs, rhs chunk.Row) int { + for i, col := range cwc.affectedColSchema.Columns { + ret := cwc.compareFuncs[i](lhs, col.Index, rhs, col.Index) + if ret != 0 { + return ret + } } + return 0 +} - // Guarantee res.AccessConds is not empty. - if len(res.AccessConds) == 0 { - return nil, nil, nil +// BuildRangesByRow will build range of the given row. It will eval each function's arg then call BuildRange. +func (cwc *ColWithCmpFuncManager) BuildRangesByRow(ctx sessionctx.Context, row chunk.Row) ([]*ranger.Range, error) { + exprs := make([]expression.Expression, len(cwc.OpType)) + for i, opType := range cwc.OpType { + constantArg, err := cwc.opArg[i].Eval(row) + if err != nil { + return nil, err + } + cwc.tmpConstant[i].Value = constantArg + newExpr, err := expression.NewFunction(ctx, opType, types.NewFieldType(mysql.TypeTiny), cwc.targetCol, cwc.tmpConstant[i]) + if err != nil { + return nil, err + } + exprs = append(exprs, newExpr) + } + ranges, err := ranger.BuildColumnRange(exprs, ctx.GetSessionVars().StmtCtx, cwc.targetCol.RetType, cwc.colLength) + if err != nil { + return nil, err } + return ranges, nil +} - // Find invalid fake condition and modify the joinKey's idxOff to -1. - var invalidFakeConds []expression.Expression - for i, eqCond := range eqConds { - if !expression.Contains(res.AccessConds, eqCond) { - keyOff2IdxOff[i] = -1 - invalidFakeConds = append(invalidFakeConds, eqCond) +func (cwc *ColWithCmpFuncManager) resolveIndices(schema *expression.Schema) (err error) { + for i := range cwc.opArg { + cwc.opArg[i], err = cwc.opArg[i].ResolveIndices(schema) + if err != nil { + return err } } + return nil +} - // Filter out invalidFakeConds from res.RemainedConds. - for _, cond := range res.RemainedConds { - if !expression.Contains(invalidFakeConds, cond) { - remained = append(remained, cond) +// String implements Stringer interface. +func (cwc *ColWithCmpFuncManager) String() string { + buffer := bytes.NewBufferString("") + for i := range cwc.OpType { + buffer.WriteString(fmt.Sprintf("%v(%v, %v)", cwc.OpType[i], cwc.targetCol, cwc.opArg[i])) + if i < len(cwc.OpType)-1 { + buffer.WriteString(" ") } } - - return res.Ranges, remained, keyOff2IdxOff + return buffer.String() } -func (p *LogicalJoin) buildFakeEqCondsForIndexJoin(keys, idxCols []*expression.Column, colLengths []int, - innerFilters []expression.Expression) (accesses, eqConds, remained []expression.Expression, keyOff2IdxOff []int) { - // Check whether all join keys match one column from index. - keyOff2IdxOff = joinKeysMatchIndex(keys, idxCols, colLengths) - if keyOff2IdxOff == nil { - return nil, nil, nil, nil +func (ijHelper *indexJoinBuildHelper) checkIndex(innerKeys []*expression.Column, idxCols []*expression.Column, colLens []int) bool { + tmpSchema := expression.NewSchema(innerKeys...) + ijHelper.curIdxOff2KeyOff = make([]int, len(idxCols)) + ijHelper.curNotUsedIndexCols = make([]*expression.Column, 0, len(idxCols)) + ijHelper.curNotUsedColLens = make([]int, 0, len(idxCols)) + keyMatched := false + for i, idxCol := range idxCols { + ijHelper.curIdxOff2KeyOff[i] = tmpSchema.ColumnIndex(idxCol) + if ijHelper.curIdxOff2KeyOff[i] >= 0 { + keyMatched = true + continue + } + ijHelper.curNotUsedIndexCols = append(ijHelper.curNotUsedIndexCols, idxCol) + ijHelper.curNotUsedColLens = append(ijHelper.curNotUsedColLens, colLens[i]) } + return keyMatched +} - usableKeys := make([]*expression.Column, 0, len(keys)) +// findUsefulEqAndInFilters analyzes the pushedDownConds held by inner child and split them to three parts. +// usefulEqOrInFilters is the continuous eq/in conditions on current unused index columns. +// uselessFilters is the conditions which cannot be used for building ranges. +// remainingRangeCandidates is the other conditions for future use. +func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSource) (usefulEqOrInFilters, uselessFilters, remainingRangeCandidates []expression.Expression) { + uselessFilters = make([]expression.Expression, 0, len(innerPlan.pushedDownConds)) + var remainedEqOrIn []expression.Expression + // Extract the eq/in functions of possible join key. + // you can see the comment of ExtractEqAndInCondition to get the meaning of the second return value. + usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates, _ = ranger.ExtractEqAndInCondition( + innerPlan.ctx, innerPlan.pushedDownConds, + ijHelper.curNotUsedIndexCols, + ijHelper.curNotUsedColLens, + ) + uselessFilters = append(uselessFilters, remainedEqOrIn...) + return usefulEqOrInFilters, uselessFilters, remainingRangeCandidates +} - conds := make([]expression.Expression, 0, len(keys)+len(innerFilters)) - eqConds = make([]expression.Expression, 0, len(keys)) - // Construct a fake equal expression for every join key for calculating the range. - for i, key := range keys { - if keyOff2IdxOff[i] < 0 { +// buildLastColManager analyze the `OtherConditions` of join to see whether there're some filters can be used in manager. +// The returned value is just for outputting explain information +func (ijHelper *indexJoinBuildHelper) buildLastColManager(nextCol *expression.Column, + innerPlan *DataSource, cwc *ColWithCmpFuncManager) []expression.Expression { + var lastColAccesses []expression.Expression +loopOtherConds: + for _, filter := range ijHelper.join.OtherConditions { + sf, ok := filter.(*expression.ScalarFunction) + if !ok || !(sf.FuncName.L == ast.LE || sf.FuncName.L == ast.LT || sf.FuncName.L == ast.GE || sf.FuncName.L == ast.GT) { + continue + } + var funcName string + var anotherArg expression.Expression + if lCol, ok := sf.GetArgs()[0].(*expression.Column); ok && lCol.Equal(nil, nextCol) { + anotherArg = sf.GetArgs()[1] + funcName = sf.FuncName.L + } else if rCol, ok := sf.GetArgs()[1].(*expression.Column); ok && rCol.Equal(nil, nextCol) { + anotherArg = sf.GetArgs()[0] + // The column manager always build expression in the form of col op arg1. + // So we need use the symmetric one of the current function. + funcName = symmetricOp[sf.FuncName.L] + } else { continue } - usableKeys = append(usableKeys, key) - // Int datum 1 can convert to all column's type(numeric type, string type, json, time type, enum, set) safely. - fakeConstant := &expression.Constant{Value: types.NewIntDatum(1), RetType: key.GetType()} - eqFunc := expression.NewFunctionInternal(p.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), key, fakeConstant) - conds = append(conds, eqFunc) - eqConds = append(eqConds, eqFunc) + affectedCols := expression.ExtractColumns(anotherArg) + if len(affectedCols) == 0 { + continue + } + for _, col := range affectedCols { + if innerPlan.schema.Contains(col) { + continue loopOtherConds + } + } + lastColAccesses = append(lastColAccesses, sf) + cwc.appendNewExpr(funcName, anotherArg, affectedCols) } + return lastColAccesses +} - // Look into every `innerFilter`, if it contains join keys' column, put this filter into `remained` part directly. - remained = make([]expression.Expression, 0, len(innerFilters)) - for _, filter := range innerFilters { - affectedCols := expression.ExtractColumns(filter) - if expression.ColumnSliceIsIntersect(affectedCols, usableKeys) { - remained = append(remained, filter) +// removeUselessEqAndInFunc removes the useless eq/in conditions. It's designed for the following case: +// t1 join t2 on t1.a=t2.a and t1.c=t2.c where t1.b > t2.b-10 and t1.b < t2.b+10 there's index(a, b, c) on t1. +// In this case the curIdxOff2KeyOff is [0 -1 1] and the notKeyEqAndIn is []. +// It's clearly that the column c cannot be used to access data. So we need to remove it and reset the IdxOff2KeyOff to +// [0 -1 -1]. +// So that we can use t1.a=t2.a and t1.b > t2.b-10 and t1.b < t2.b+10 to build ranges then access data. +func (ijHelper *indexJoinBuildHelper) removeUselessEqAndInFunc( + idxCols []*expression.Column, + notKeyEqAndIn []expression.Expression) ( + usefulEqAndIn, uselessOnes []expression.Expression, +) { + ijHelper.curPossibleUsedKeys = make([]*expression.Column, 0, len(idxCols)) + for idxColPos, notKeyColPos := 0, 0; idxColPos < len(idxCols); idxColPos++ { + if ijHelper.curIdxOff2KeyOff[idxColPos] != -1 { + ijHelper.curPossibleUsedKeys = append(ijHelper.curPossibleUsedKeys, idxCols[idxColPos]) + continue + } + if notKeyColPos < len(notKeyEqAndIn) && ijHelper.curNotUsedIndexCols[notKeyColPos].Equal(nil, idxCols[idxColPos]) { + notKeyColPos++ continue } - conds = append(conds, filter) + for i := idxColPos + 1; i < len(idxCols); i++ { + ijHelper.curIdxOff2KeyOff[i] = -1 + } + remained := make([]expression.Expression, 0, len(notKeyEqAndIn)-notKeyColPos) + remained = append(remained, notKeyEqAndIn[notKeyColPos:]...) + notKeyEqAndIn = notKeyEqAndIn[:notKeyColPos] + return notKeyEqAndIn, remained + } + return notKeyEqAndIn, nil +} + +func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(indexInfo *model.IndexInfo, innerPlan *DataSource, innerJoinKeys []*expression.Column) error { + idxCols, colLengths := expression.IndexInfo2Cols(innerPlan.schema.Columns, indexInfo) + if len(idxCols) == 0 { + return nil + } + accesses := make([]expression.Expression, 0, len(idxCols)) + // If no index column appears in join key, we just break. + // TODO: It may meet this case: There's no join key condition, but have compare filters. + // e.g. select * from t1, t2 on t1.a=t2.a and t2.b > t1.b-10 and t2.b < t1.b where t1.a=1 and t2.a=1. + // After constant propagation. The t1.a=t2.a is removed. And if we have index (t2.a, t2.b). It can apply index join + // to speed up. + if !ijHelper.checkIndex(innerJoinKeys, idxCols, colLengths) { + return nil + } + notKeyEqAndIn, remained, rangeFilterCandidates := ijHelper.findUsefulEqAndInFilters(innerPlan) + var remainedEqAndIn []expression.Expression + notKeyEqAndIn, remainedEqAndIn = ijHelper.removeUselessEqAndInFunc(idxCols, notKeyEqAndIn) + matchedKeyCnt := len(ijHelper.curPossibleUsedKeys) + if matchedKeyCnt <= 0 { + return nil + } + accesses = append(accesses, notKeyEqAndIn...) + remained = append(remained, remainedEqAndIn...) + lastColPos := matchedKeyCnt + len(notKeyEqAndIn) + // If all the index columns are covered by eq/in conditions, we don't need to consider other conditions anymore. + if lastColPos == len(idxCols) { + remained = append(remained, rangeFilterCandidates...) + ranges, err := ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nil, false) + if err != nil { + return err + } + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, nil) + return nil + } + lastPossibleCol := idxCols[lastColPos] + lastColManager := &ColWithCmpFuncManager{ + targetCol: lastPossibleCol, + colLength: colLengths[lastColPos], + affectedColSchema: expression.NewSchema(), + } + lastColAccess := ijHelper.buildLastColManager(lastPossibleCol, innerPlan, lastColManager) + // If the column manager holds no expression, then we fallback to find whether there're useful normal filters + if len(lastColAccess) == 0 { + colAccesses, colRemained := ranger.DetachCondsForColumn(ijHelper.join.ctx, rangeFilterCandidates, lastPossibleCol) + var ranges, nextColRange []*ranger.Range + var err error + if len(colAccesses) > 0 { + nextColRange, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx.GetSessionVars().StmtCtx, lastPossibleCol.RetType, colLengths[lastColPos]) + if err != nil { + return err + } + } + ranges, err = ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nextColRange, false) + if err != nil { + return err + } + remained = append(remained, colRemained...) + if colLengths[lastColPos] != types.UnspecifiedLength { + remained = append(remained, colAccesses...) + } + accesses = append(accesses, colAccesses...) + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, nil) + return nil } + accesses = append(accesses, lastColAccess...) + remained = append(remained, rangeFilterCandidates...) + ranges, err := ijHelper.buildTemplateRange(matchedKeyCnt, notKeyEqAndIn, nil, true) + if err != nil { + return err + } + ijHelper.updateBestChoice(ranges, indexInfo, accesses, remained, lastColManager) + return nil +} - return conds, eqConds, remained, keyOff2IdxOff +func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges []*ranger.Range, idxInfo *model.IndexInfo, accesses, + remained []expression.Expression, lastColManager *ColWithCmpFuncManager) { + // We choose the index by the number of used columns of the range, the much the better. + // Notice that there may be the cases like `t1.a=t2.a and b > 2 and b < 1`. So ranges can be nil though the conditions are valid. + // But obviously when the range is nil, we don't need index join. + if len(ranges) > 0 && len(ranges[0].LowVal) > ijHelper.maxUsedCols { + ijHelper.chosenIndexInfo = idxInfo + ijHelper.maxUsedCols = len(ranges[0].LowVal) + ijHelper.chosenRanges = ranges + ijHelper.chosenAccess = accesses + ijHelper.chosenRemained = remained + ijHelper.idxOff2KeyOff = ijHelper.curIdxOff2KeyOff + ijHelper.lastColManager = lastColManager + } +} + +func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool) (ranges []*ranger.Range, err error) { + pointLength := matchedKeyCnt + len(eqAndInFuncs) + if nextColRange != nil { + for _, colRan := range nextColRange { + // The range's exclude status is the same with last col's. + ran := &ranger.Range{ + LowVal: make([]types.Datum, pointLength, pointLength+1), + HighVal: make([]types.Datum, pointLength, pointLength+1), + LowExclude: colRan.LowExclude, + HighExclude: colRan.HighExclude, + } + ran.LowVal = append(ran.LowVal, colRan.LowVal[0]) + ran.HighVal = append(ran.HighVal, colRan.HighVal[0]) + ranges = append(ranges, ran) + } + } else if haveExtraCol { + // Reserve a position for the last col. + ranges = append(ranges, &ranger.Range{ + LowVal: make([]types.Datum, pointLength+1, pointLength+1), + HighVal: make([]types.Datum, pointLength+1, pointLength+1), + }) + } else { + ranges = append(ranges, &ranger.Range{ + LowVal: make([]types.Datum, pointLength, pointLength), + HighVal: make([]types.Datum, pointLength, pointLength), + }) + } + emptyRow := chunk.Row{} + for i, j := 0, 0; j < len(eqAndInFuncs); i++ { + // This position is occupied by join key. + if ijHelper.curIdxOff2KeyOff[i] != -1 { + continue + } + sf := eqAndInFuncs[j].(*expression.ScalarFunction) + // Deal with the first two args. + if _, ok := sf.GetArgs()[0].(*expression.Column); ok { + for _, ran := range ranges { + ran.LowVal[i], err = sf.GetArgs()[1].Eval(emptyRow) + if err != nil { + return nil, err + } + ran.HighVal[i] = ran.LowVal[i] + } + } else { + for _, ran := range ranges { + ran.LowVal[i], err = sf.GetArgs()[0].Eval(emptyRow) + if err != nil { + return nil, err + } + ran.HighVal[i] = ran.LowVal[i] + } + } + // If the length of in function's constant list is more than one, we will expand ranges. + curRangeLen := len(ranges) + for argIdx := 2; argIdx < len(sf.GetArgs()); argIdx++ { + newRanges := make([]*ranger.Range, 0, curRangeLen) + for oldRangeIdx := 0; oldRangeIdx < curRangeLen; oldRangeIdx++ { + newRange := ranges[oldRangeIdx].Clone() + newRange.LowVal[i], err = sf.GetArgs()[argIdx].Eval(emptyRow) + if err != nil { + return nil, err + } + newRange.HighVal[i] = newRange.LowVal[i] + newRanges = append(newRanges, newRange) + } + ranges = append(ranges, newRanges...) + } + j++ + } + return ranges, nil } // tryToGetIndexJoin will get index join by hints. If we can generate a valid index join by hint, the second return value @@ -651,10 +973,6 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ } }() - if len(p.EqualConditions) == 0 { - return nil, false - } - switch p.JoinType { case SemiJoin, AntiSemiJoin, LeftOuterSemiJoin, AntiLeftOuterSemiJoin, LeftOuterJoin: join := p.getIndexJoinByOuterIdx(prop, 0) diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go new file mode 100644 index 0000000000000..a7cf2a21a9a1f --- /dev/null +++ b/planner/core/exhaust_physical_plans_test.go @@ -0,0 +1,238 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package core + +import ( + "fmt" + + . "github.com/pingcap/check" + "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/types" +) + +func (s *testUnitTestSuit) rewriteSimpleExpr(str string, schema *expression.Schema) ([]expression.Expression, error) { + if str == "" { + return nil, nil + } + filters, err := expression.ParseSimpleExprsWithSchema(s.ctx, str, schema) + if err != nil { + return nil, err + } + if sf, ok := filters[0].(*expression.ScalarFunction); ok && sf.FuncName.L == ast.LogicAnd { + filters = expression.FlattenCNFConditions(sf) + } + return filters, nil +} + +func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) { + s.ctx.GetSessionVars().PlanID = -1 + joinNode := LogicalJoin{}.Init(s.ctx) + dataSourceNode := DataSource{}.Init(s.ctx) + dsSchema := expression.NewSchema() + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("a"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("b"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("c"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeVarchar), + }) + dsSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("d"), + TblName: model.NewCIStr("t"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + dataSourceNode.schema = dsSchema + outerChildSchema := expression.NewSchema() + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("e"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("f"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("g"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeVarchar), + }) + outerChildSchema.Append(&expression.Column{ + UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(), + ColName: model.NewCIStr("h"), + TblName: model.NewCIStr("t1"), + DBName: model.NewCIStr("test"), + RetType: types.NewFieldType(mysql.TypeLonglong), + }) + joinNode.SetSchema(expression.MergeSchema(dsSchema, outerChildSchema)) + var idxCols []*model.IndexColumn + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("a"), Length: types.UnspecifiedLength}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("b"), Length: types.UnspecifiedLength}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("c"), Length: 2}) + idxCols = append(idxCols, &model.IndexColumn{Name: model.NewCIStr("d"), Length: types.UnspecifiedLength}) + idxInfo := &model.IndexInfo{Columns: idxCols} + + tests := []struct { + innerKeys []*expression.Column + pushedDownConds string + otherConds string + ranges string + idxOff2KeyOff string + accesses string + remained string + compareFilters string + }{ + // Join key not continuous and no pushed filter to match. + { + innerKeys: []*expression.Column{dsSchema.Columns[0], dsSchema.Columns[2]}, + pushedDownConds: "", + otherConds: "", + ranges: "[[NULL,NULL]]", + idxOff2KeyOff: "[0 -1 -1 -1]", + accesses: "[]", + remained: "[]", + compareFilters: "", + }, + // Join key and pushed eq filter not continuous. + { + innerKeys: []*expression.Column{dsSchema.Columns[2]}, + pushedDownConds: "a = 1", + otherConds: "", + ranges: "[]", + idxOff2KeyOff: "[]", + accesses: "[]", + remained: "[]", + compareFilters: "", + }, + // Keys are continuous. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "", + ranges: "[[1 NULL,1 NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1)]", + remained: "[]", + compareFilters: "", + }, + // Keys are continuous and there're correlated filters. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "c > g and c < concat(g, \"ab\")", + ranges: "[[1 NULL NULL,1 NULL NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, test.t1.g) lt(test.t.c, concat(test.t1.g, ab))]", + remained: "[]", + compareFilters: "gt(test.t.c, test.t1.g) lt(test.t.c, concat(test.t1.g, ab))", + }, + // cast function won't be involved. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1", + otherConds: "c > g and c < g + 10", + ranges: "[[1 NULL NULL,1 NULL NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, test.t1.g)]", + remained: "[]", + compareFilters: "gt(test.t.c, test.t1.g)", + }, + // Can deal with prefix index correctly. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a = 1 and c > 'a' and c < 'aaaaaa'", + otherConds: "", + ranges: "[(1 NULL \"a\",1 NULL \"[97 97]\"]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[eq(test.t.a, 1) gt(test.t.c, a) lt(test.t.c, aaaaaa)]", + remained: "[gt(test.t.c, a) lt(test.t.c, aaaaaa)]", + compareFilters: "", + }, + // Can generate correct ranges for in functions. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a in (1, 2, 3) and c in ('a', 'b', 'c')", + otherConds: "", + ranges: "[[1 NULL \"a\",1 NULL \"a\"] [2 NULL \"a\",2 NULL \"a\"] [3 NULL \"a\",3 NULL \"a\"] [1 NULL \"b\",1 NULL \"b\"] [2 NULL \"b\",2 NULL \"b\"] [3 NULL \"b\",3 NULL \"b\"] [1 NULL \"c\",1 NULL \"c\"] [2 NULL \"c\",2 NULL \"c\"] [3 NULL \"c\",3 NULL \"c\"]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[in(test.t.a, 1, 2, 3) in(test.t.c, a, b, c)]", + remained: "[in(test.t.c, a, b, c)]", + compareFilters: "", + }, + // Can generate correct ranges for in functions with correlated filters.. + { + innerKeys: []*expression.Column{dsSchema.Columns[1]}, + pushedDownConds: "a in (1, 2, 3) and c in ('a', 'b', 'c')", + otherConds: "d > h and d < h + 100", + ranges: "[[1 NULL \"a\" NULL,1 NULL \"a\" NULL] [2 NULL \"a\" NULL,2 NULL \"a\" NULL] [3 NULL \"a\" NULL,3 NULL \"a\" NULL] [1 NULL \"b\" NULL,1 NULL \"b\" NULL] [2 NULL \"b\" NULL,2 NULL \"b\" NULL] [3 NULL \"b\" NULL,3 NULL \"b\" NULL] [1 NULL \"c\" NULL,1 NULL \"c\" NULL] [2 NULL \"c\" NULL,2 NULL \"c\" NULL] [3 NULL \"c\" NULL,3 NULL \"c\" NULL]]", + idxOff2KeyOff: "[-1 0 -1 -1]", + accesses: "[in(test.t.a, 1, 2, 3) in(test.t.c, a, b, c) gt(test.t.d, test.t1.h) lt(test.t.d, plus(test.t1.h, 100))]", + remained: "[in(test.t.c, a, b, c)]", + compareFilters: "gt(test.t.d, test.t1.h) lt(test.t.d, plus(test.t1.h, 100))", + }, + // Join keys are not continuous and the pushed key connect the key but not eq/in functions. + { + innerKeys: []*expression.Column{dsSchema.Columns[0], dsSchema.Columns[2]}, + pushedDownConds: "b > 1", + otherConds: "", + ranges: "[(NULL 1,NULL +inf]]", + idxOff2KeyOff: "[0 -1 -1 -1]", + accesses: "[gt(test.t.b, 1)]", + remained: "[]", + compareFilters: "", + }, + } + for i, tt := range tests { + pushed, err := s.rewriteSimpleExpr(tt.pushedDownConds, dsSchema) + c.Assert(err, IsNil) + dataSourceNode.pushedDownConds = pushed + others, err := s.rewriteSimpleExpr(tt.otherConds, joinNode.schema) + c.Assert(err, IsNil) + joinNode.OtherConditions = others + helper := &indexJoinBuildHelper{join: joinNode, lastColManager: nil} + err = helper.analyzeLookUpFilters(idxInfo, dataSourceNode, tt.innerKeys) + c.Assert(err, IsNil) + c.Assert(fmt.Sprintf("%v", helper.chosenRanges), Equals, tt.ranges, Commentf("test case: #%v", i)) + c.Assert(fmt.Sprintf("%v", helper.idxOff2KeyOff), Equals, tt.idxOff2KeyOff) + c.Assert(fmt.Sprintf("%v", helper.chosenAccess), Equals, tt.accesses) + c.Assert(fmt.Sprintf("%v", helper.chosenRemained), Equals, tt.remained) + c.Assert(fmt.Sprintf("%v", helper.lastColManager), Equals, tt.compareFilters) + } +} diff --git a/planner/core/explain.go b/planner/core/explain.go index 18fe8f8f998df..ad801dbaecd8f 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -58,8 +58,8 @@ func (p *PhysicalIndexScan) ExplainInfo() string { break } } - if len(p.rangeDecidedBy) > 0 { - fmt.Fprintf(buffer, ", range: decided by %v", p.rangeDecidedBy) + if len(p.rangeInfo) > 0 { + fmt.Fprintf(buffer, ", range: decided by %v", p.rangeInfo) } else if haveCorCol { fmt.Fprintf(buffer, ", range: decided by %v", p.AccessCondition) } else if len(p.Ranges) > 0 { diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 80df8bcf548c8..b71546145175b 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -477,7 +477,6 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid IdxColLens: path.idxColLens, AccessCondition: path.accessConds, Ranges: path.ranges, - filterCondition: path.indexFilters, dataSourceSchema: ds.schema, isPartition: ds.isPartition, physicalTableID: ds.physicalTableID, @@ -734,7 +733,7 @@ func (ds *DataSource) crossEstimateRowCount(path *accessPath, expectedCnt float6 return 0, false, corr } sc := ds.ctx.GetSessionVars().StmtCtx - ranges, err := ranger.BuildColumnRange(accessConds, sc, col.RetType) + ranges, err := ranger.BuildColumnRange(accessConds, sc, col.RetType, types.UnspecifiedLength) if len(ranges) == 0 || err != nil { return 0, err == nil, corr } diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index bfa7598adf72a..81529c780bbd4 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -90,7 +90,6 @@ type PhysicalIndexScan struct { // AccessCondition is used to calculate range. AccessCondition []expression.Expression - filterCondition []expression.Expression Table *model.TableInfo Index *model.IndexInfo @@ -115,7 +114,7 @@ type PhysicalIndexScan struct { // It is used for query feedback. Hist *statistics.Histogram - rangeDecidedBy []*expression.Column + rangeInfo string // The index scan may be on a partition. isPartition bool @@ -235,6 +234,12 @@ type PhysicalIndexJoin struct { Ranges []*ranger.Range // KeyOff2IdxOff maps the offsets in join key to the offsets in the index. KeyOff2IdxOff []int + // CompareFilters stores the filters for last column if those filters need to be evaluated during execution. + // e.g. select * from t where t.a = t1.a and t.b > t1.b and t.b < t1.b+10 + // If there's index(t.a, t.b). All the filters can be used to construct index range but t.b > t1.b and t.b < t1.b=10 + // need to be evaluated after we fetch the data of t1. + // This struct stores them and evaluate them to ranges. + CompareFilters *ColWithCmpFuncManager } // PhysicalMergeJoin represents merge join for inner/ outer join. diff --git a/planner/core/resolve_indices.go b/planner/core/resolve_indices.go index 3cabf66819691..ec415eaf9bef6 100644 --- a/planner/core/resolve_indices.go +++ b/planner/core/resolve_indices.go @@ -193,6 +193,19 @@ func (p *PhysicalIndexJoin) ResolveIndices() (err error) { return err } } + if p.CompareFilters != nil { + err = p.CompareFilters.resolveIndices(p.children[p.OuterIndex].Schema()) + if err != nil { + return err + } + for i := range p.CompareFilters.affectedColSchema.Columns { + resolvedCol, err1 := p.CompareFilters.affectedColSchema.Columns[i].ResolveIndices(p.children[p.OuterIndex].Schema()) + if err1 != nil { + return err1 + } + p.CompareFilters.affectedColSchema.Columns[i] = resolvedCol.(*expression.Column) + } + } return } diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index c5d7ec75cb851..b3a6200151ce8 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -18,6 +18,7 @@ import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/ranger" ) @@ -186,7 +187,7 @@ func (s *partitionProcessor) canBePruned(sctx sessionctx.Context, partCol *expre // handle the null condition, while calculate range can prune something like: // "select * from t where t is null" accessConds := ranger.ExtractAccessConditionsForColumn(conds, partCol.UniqueID) - r, err := ranger.BuildColumnRange(accessConds, sctx.GetSessionVars().StmtCtx, partCol.RetType) + r, err := ranger.BuildColumnRange(accessConds, sctx.GetSessionVars().StmtCtx, partCol.RetType, types.UnspecifiedLength) if err != nil { return false, err } diff --git a/statistics/selectivity.go b/statistics/selectivity.go index 60cd6d80bff98..fef5c54907d25 100644 --- a/statistics/selectivity.go +++ b/statistics/selectivity.go @@ -21,6 +21,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" ) @@ -266,7 +267,7 @@ func getMaskAndRanges(ctx sessionctx.Context, exprs []expression.Expression, ran switch rangeType { case ranger.ColumnRangeType: accessConds = ranger.ExtractAccessConditionsForColumn(exprs, cols[0].UniqueID) - ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType) + ranges, err = ranger.BuildColumnRange(accessConds, sc, cols[0].RetType, types.UnspecifiedLength) case ranger.IndexRangeType: var res *ranger.DetachRangeResult res, err = ranger.DetachCondAndBuildRangeForIndex(ctx, exprs, cols, lengths) diff --git a/util/ranger/detacher.go b/util/ranger/detacher.go index 6aabbfda3f841..671ea328a0c5b 100644 --- a/util/ranger/detacher.go +++ b/util/ranger/detacher.go @@ -146,7 +146,7 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex ) res := &DetachRangeResult{} - accessConds, filterConds, newConditions, emptyRange := extractEqAndInCondition(sctx, conditions, cols, lengths) + accessConds, filterConds, newConditions, emptyRange := ExtractEqAndInCondition(sctx, conditions, cols, lengths) if emptyRange { return res, nil } @@ -156,8 +156,6 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex break } } - // We should remove all accessConds, so that they will not be added to filter conditions. - newConditions = removeAccessConditions(newConditions, accessConds) eqOrInCount := len(accessConds) res.EqCondCount = eqCount res.EqOrInCount = eqOrInCount @@ -197,7 +195,13 @@ func detachCNFCondAndBuildRangeForIndex(sctx sessionctx.Context, conditions []ex return res, err } -func extractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, +// ExtractEqAndInCondition will split the given condition into three parts by the information of index columns and their lengths. +// accesses: The condition will be used to build range. +// filters: filters is the part that some access conditions need to be evaluate again since it's only the prefix part of char column. +// newConditions: We'll simplify the given conditions if there're multiple in conditions or eq conditions on the same column. +// e.g. if there're a in (1, 2, 3) and a in (2, 3, 4). This two will be combined to a in (2, 3) and pushed to newConditions. +// bool: indicate whether there's nil range when merging eq and in conditions. +func ExtractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Expression, cols []*expression.Column, lengths []int) ([]expression.Expression, []expression.Expression, []expression.Expression, bool) { var filters []expression.Expression rb := builder{sc: sctx.GetSessionVars().StmtCtx} @@ -246,6 +250,8 @@ func extractEqAndInCondition(sctx sessionctx.Context, conditions []expression.Ex filters = append(filters, cond) } } + // We should remove all accessConds, so that they will not be added to filter conditions. + newConditions = removeAccessConditions(newConditions, accesses) return accesses, filters, newConditions, false } diff --git a/util/ranger/ranger.go b/util/ranger/ranger.go index 5722c7a17e43e..015f6ea447eb7 100644 --- a/util/ranger/ranger.go +++ b/util/ranger/ranger.go @@ -243,7 +243,7 @@ func points2TableRanges(sc *stmtctx.StatementContext, rangePoints []point, tp *t } // buildColumnRange builds range from CNF conditions. -func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, tableRange bool) (ranges []*Range, err error) { +func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, tableRange bool, colLen int) (ranges []*Range, err error) { rb := builder{sc: sc} rangePoints := fullRange for _, cond := range accessConditions { @@ -261,20 +261,30 @@ func buildColumnRange(accessConditions []expression.Expression, sc *stmtctx.Stat if err != nil { return nil, errors.Trace(err) } + if colLen != types.UnspecifiedLength { + for _, ran := range ranges { + if fixRangeDatum(&ran.LowVal[0], colLen, tp) { + ran.LowExclude = false + } + if fixRangeDatum(&ran.HighVal[0], colLen, tp) { + ran.HighExclude = false + } + } + } return ranges, nil } // BuildTableRange builds range of PK column for PhysicalTableScan. func BuildTableRange(accessConditions []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType) ([]*Range, error) { - return buildColumnRange(accessConditions, sc, tp, true) + return buildColumnRange(accessConditions, sc, tp, true, types.UnspecifiedLength) } // BuildColumnRange builds range from access conditions for general columns. -func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType) ([]*Range, error) { +func BuildColumnRange(conds []expression.Expression, sc *stmtctx.StatementContext, tp *types.FieldType, colLen int) ([]*Range, error) { if len(conds) == 0 { return []*Range{{LowVal: []types.Datum{{}}, HighVal: []types.Datum{types.MaxValueDatum()}}}, nil } - return buildColumnRange(conds, sc, tp, false) + return buildColumnRange(conds, sc, tp, false, colLen) } // buildCNFIndexRange builds the range for index where the top layer is CNF. @@ -485,7 +495,7 @@ func newFieldType(tp *types.FieldType) *types.FieldType { // 1. 'expr' must be either 'EQUAL' or 'IN' function. // 2. 'points' should not be empty. func points2EqOrInCond(ctx sessionctx.Context, points []point, expr expression.Expression) expression.Expression { - // len(points) cannot be 0 here, since we impose early termination in extractEqAndInCondition + // len(points) cannot be 0 here, since we impose early termination in ExtractEqAndInCondition sf, _ := expr.(*expression.ScalarFunction) // Constant and Column args should have same RetType, simply get from first arg retType := sf.GetArgs()[0].GetType() diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index c967f70178f62..5d17fdb91e31e 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mocktikv" + "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/ranger" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -956,7 +957,7 @@ func (s *testRangerSuite) TestColumnRange(c *C) { c.Assert(col, NotNil) conds = ranger.ExtractAccessConditionsForColumn(conds, col.UniqueID) c.Assert(fmt.Sprintf("%s", conds), Equals, tt.accessConds, Commentf("wrong access conditions for expr: %s", tt.exprStr)) - result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType) + result, err := ranger.BuildColumnRange(conds, new(stmtctx.StatementContext), col.RetType, types.UnspecifiedLength) c.Assert(err, IsNil) got := fmt.Sprintf("%v", result) c.Assert(got, Equals, tt.resultStr, Commentf("different for expr %s, col: %v", tt.exprStr, col)) From d8149580bca3a7fe16f5f3a52fb7ed027d155035 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Mon, 29 Apr 2019 11:50:17 +0800 Subject: [PATCH 16/49] session: add the transaction commit time to slow log (#10294) --- executor/adapter.go | 10 ---------- session/session.go | 6 ------ session/tidb.go | 13 +++++++++---- 3 files changed, 9 insertions(+), 20 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 01e005d7dd0f4..07ac3fcfd0fa1 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -290,16 +290,6 @@ func (a *ExecStmt) handleNoDelayExecutor(ctx context.Context, sctx sessionctx.Co var err error defer func() { terror.Log(e.Close()) - txnTS := uint64(0) - // Don't active pending txn here. - if txn, err1 := sctx.Txn(false); err1 != nil { - logutil.Logger(ctx).Error("get current transaction failed", zap.Error(err)) - } else { - if txn.Valid() { - txnTS = txn.StartTS() - } - } - a.LogSlowQuery(txnTS, err == nil) a.logAudit() }() diff --git a/session/session.go b/session/session.go index 9685a4aa9e5c5..35221280cea69 100644 --- a/session/session.go +++ b/session/session.go @@ -460,18 +460,12 @@ func (s *session) CommitTxn(ctx context.Context) error { defer span1.Finish() } - stmt := executor.ExecStmt{ - Text: "commitTxn", - Ctx: s, - StartTime: time.Now(), - } var commitDetail *execdetails.CommitDetails ctx = context.WithValue(ctx, execdetails.CommitDetailCtxKey, &commitDetail) err := s.doCommitWithRetry(ctx) if commitDetail != nil { s.sessionVars.StmtCtx.MergeExecDetails(nil, commitDetail) } - stmt.LogSlowQuery(s.sessionVars.TxnCtx.StartTS, err == nil) s.sessionVars.TxnCtx.Cleanup() s.recordTransactionCounter(err) return err diff --git a/session/tidb.go b/session/tidb.go index 8b8cb6c34ed0b..2be35eb4165b3 100644 --- a/session/tidb.go +++ b/session/tidb.go @@ -196,16 +196,21 @@ func checkStmtLimit(ctx context.Context, sctx sessionctx.Context, se *session, s } // runStmt executes the sqlexec.Statement and commit or rollback the current transaction. -func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement) (sqlexec.RecordSet, error) { +func runStmt(ctx context.Context, sctx sessionctx.Context, s sqlexec.Statement) (rs sqlexec.RecordSet, err error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("session.runStmt", opentracing.ChildOf(span.Context())) span1.LogKV("sql", s.OriginText()) defer span1.Finish() } - - var err error - var rs sqlexec.RecordSet se := sctx.(*session) + defer func() { + // If it is not a select statement, we record its slow log here, + // then it could include the transaction commit time. + if rs == nil { + s.(*executor.ExecStmt).LogSlowQuery(se.GetSessionVars().TxnCtx.StartTS, err != nil) + } + }() + err = se.checkTxnAborted(s) if err != nil { return nil, err From 6a578a8b57199270cf7bf4b3bd864fd7f27a27b7 Mon Sep 17 00:00:00 2001 From: Song Guo Date: Mon, 29 Apr 2019 12:15:59 +0800 Subject: [PATCH 17/49] statistics: Support store CMSketch with Top N (#10255) --- executor/analyze.go | 7 +- executor/analyze_test.go | 8 +- go.mod | 2 +- go.sum | 4 +- session/bootstrap.go | 21 ++++ session/session.go | 2 +- statistics/cmsketch.go | 185 ++++++++++++++++++++++++--------- statistics/cmsketch_test.go | 31 +++++- statistics/estimate.go | 2 - statistics/feedback.go | 6 +- statistics/handle/bootstrap.go | 2 +- statistics/handle/dump_test.go | 42 ++++++++ statistics/handle/gc.go | 5 + statistics/handle/handle.go | 17 ++- 14 files changed, 261 insertions(+), 73 deletions(-) diff --git a/executor/analyze.go b/executor/analyze.go index 5dc307ec1a99b..5c27ba9e17d39 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -322,11 +322,8 @@ func (e *AnalyzeIndexExec) buildStatsFromResult(result distsql.SelectResult, nee if needCMS { if resp.Cms == nil { logutil.Logger(context.TODO()).Warn("nil CMS in response", zap.String("table", e.idxInfo.Table.O), zap.String("index", e.idxInfo.Name.O)) - } else { - err := cms.MergeCMSketch(statistics.CMSketchFromProto(resp.Cms)) - if err != nil { - return nil, nil, err - } + } else if err := cms.MergeCMSketch(statistics.CMSketchFromProto(resp.Cms)); err != nil { + return nil, nil, err } } } diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 2510a1e9a0f09..c33880d4f4b90 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -140,6 +140,8 @@ func (s *testSuite1) TestAnalyzeFastSample(c *C) { ) c.Assert(err, IsNil) var dom *domain.Domain + session.SetStatsLease(0) + session.SetSchemaLease(0) dom, err = session.BootstrapSession(store) c.Assert(err, IsNil) tk := testkit.NewTestKit(c, store) @@ -209,6 +211,8 @@ func (s *testSuite1) TestFastAnalyze(c *C) { ) c.Assert(err, IsNil) var dom *domain.Domain + session.SetStatsLease(0) + session.SetSchemaLease(0) dom, err = session.BootstrapSession(store) c.Assert(err, IsNil) tk := testkit.NewTestKit(c, store) @@ -240,7 +244,7 @@ func (s *testSuite1) TestFastAnalyze(c *C) { tbl := dom.StatsHandle().GetTableStats(tableInfo) sTbl := fmt.Sprintln(tbl) matched := false - if sTbl == "Table:37 Count:3000\n"+ + if sTbl == "Table:39 Count:3000\n"+ "column:1 ndv:3000 totColSize:0\n"+ "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ @@ -259,7 +263,7 @@ func (s *testSuite1) TestFastAnalyze(c *C) { "num: 603 lower_bound: 1207 upper_bound: 1830 repeats: 1\n"+ "num: 603 lower_bound: 1831 upper_bound: 2387 repeats: 1\n"+ "num: 588 lower_bound: 2390 upper_bound: 2997 repeats: 1\n" || - sTbl == "Table:37 Count:3000\n"+ + sTbl == "Table:39 Count:3000\n"+ "column:2 ndv:3000 totColSize:0\n"+ "num: 603 lower_bound: 6 upper_bound: 612 repeats: 1\n"+ "num: 603 lower_bound: 621 upper_bound: 1205 repeats: 1\n"+ diff --git a/go.mod b/go.mod index ff60cdb589e76..bc03df1110c07 100644 --- a/go.mod +++ b/go.mod @@ -42,7 +42,7 @@ require ( github.com/pingcap/parser v0.0.0-20190427000002-f3ecae036b23 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible - github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 + github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 github.com/prometheus/client_golang v0.9.0 github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910 github.com/prometheus/common v0.0.0-20181020173914-7e9e6cabbd39 // indirect diff --git a/go.sum b/go.sum index 4dc2ec528b2a5..d0b3f9a8a1c0b 100644 --- a/go.sum +++ b/go.sum @@ -166,8 +166,8 @@ github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQk github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= -github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7 h1:wnjdQRhybddDesBVBKyOLUPgDaOFdtqA92pduBgWvVQ= -github.com/pingcap/tipb v0.0.0-20190107072121-abbec73437b7/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= +github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 h1:rRMLMjIMFulCX9sGKZ1hoov/iROMsKyC8Snc02nSukw= +github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= diff --git a/session/bootstrap.go b/session/bootstrap.go index 2998f53747a5e..34a0503a051a7 100644 --- a/session/bootstrap.go +++ b/session/bootstrap.go @@ -248,6 +248,16 @@ const ( DEFAULT_ROLE_USER char(32) COLLATE utf8_bin NOT NULL DEFAULT '', PRIMARY KEY (HOST,USER,DEFAULT_ROLE_HOST,DEFAULT_ROLE_USER) )` + + // CreateStatsTopNTable stores topn data of a cmsketch with top n. + CreateStatsTopNTable = `CREATE TABLE if not exists mysql.stats_top_n ( + table_id bigint(64) NOT NULL, + is_index tinyint(2) NOT NULL, + hist_id bigint(64) NOT NULL, + value longblob, + count bigint(64) UNSIGNED NOT NULL, + index tbl(table_id, is_index, hist_id) + );` ) // bootstrap initiates system DB for a store. @@ -317,6 +327,7 @@ const ( version27 = 27 version28 = 28 version29 = 29 + version30 = 30 ) func checkBootstrapped(s Session) (bool, error) { @@ -492,6 +503,10 @@ func upgrade(s Session) { upgradeToVer29(s) } + if ver < version30 { + upgradeToVer30(s) + } + updateBootstrapVer(s) _, err = s.Execute(context.Background(), "COMMIT") @@ -780,6 +795,10 @@ func upgradeToVer29(s Session) { doReentrantDDL(s, "ALTER TABLE mysql.bind_info add index sql_index (original_sql(1024),default_db(1024))", ddl.ErrDupKeyName) } +func upgradeToVer30(s Session) { + mustExecute(s, CreateStatsTopNTable) +} + // updateBootstrapVer updates bootstrap version variable in mysql.TiDB table. func updateBootstrapVer(s Session) { // Update bootstrap version. @@ -836,6 +855,8 @@ func doDDLWorks(s Session) { mustExecute(s, CreateDefaultRolesTable) // Create bind_info table. mustExecute(s, CreateBindInfoTable) + // Create stats_topn_store table. + mustExecute(s, CreateStatsTopNTable) } // doDMLWorks executes DML statements in bootstrap stage. diff --git a/session/session.go b/session/session.go index 35221280cea69..7c57368857127 100644 --- a/session/session.go +++ b/session/session.go @@ -1493,7 +1493,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er const ( notBootstrapped = 0 - currentBootstrapVersion = 29 + currentBootstrapVersion = 30 ) func getStoreBootstrapVersion(store kv.Storage) int64 { diff --git a/statistics/cmsketch.go b/statistics/cmsketch.go index 07e6ebfc6e501..6c58fe47abab9 100644 --- a/statistics/cmsketch.go +++ b/statistics/cmsketch.go @@ -15,6 +15,7 @@ package statistics import ( "bytes" + "fmt" "math" "sort" @@ -25,6 +26,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" + "github.com/pingcap/tidb/util/sqlexec" "github.com/pingcap/tipb/go-tipb" "github.com/spaolacci/murmur3" ) @@ -40,15 +42,14 @@ type CMSketch struct { count uint64 // TopN is not counted in count defaultValue uint64 // In sampled data, if cmsketch returns a small value (less than avg value / 2), then this will returned. table [][]uint32 - topN map[uint64][]topNMeta + topN map[uint64][]*TopNMeta } -// topNMeta is a simple counter used by BuildTopN -type topNMeta struct { - h1 uint64 - h2 uint64 - data []byte - count uint64 +// TopNMeta is a simple counter used by BuildTopN. +type TopNMeta struct { + h2 uint64 // h2 is the second part of `murmur3.Sum128()`, it is always used with the first part `h1`. + Data []byte + Count uint64 } // NewCMSketch returns a new CM sketch. @@ -63,7 +64,6 @@ func NewCMSketch(d, w int32) *CMSketch { // topNHelper wraps some variables used when building cmsketch with top n. type topNHelper struct { sampleSize uint64 - numTop uint32 counter map[hack.MutableString]uint64 sorted []uint64 onlyOnceItems uint64 @@ -106,7 +106,7 @@ func newTopNHelper(sample [][]byte, numTop uint32) *topNHelper { sumTopN += sorted[i] } - return &topNHelper{uint64(len(sample)), numTop, counter, sorted, onlyOnceItems, sumTopN, last} + return &topNHelper{uint64(len(sample)), counter, sorted, onlyOnceItems, sumTopN, last} } // NewCMSketchWithTopN returns a new CM sketch with TopN elements, the estimate NDV and the scale ratio. @@ -122,20 +122,18 @@ func NewCMSketchWithTopN(d, w int32, sample [][]byte, numTop uint32, rowCount ui } func buildCMSWithTopN(helper *topNHelper, d, w int32, scaleRatio uint64) (c *CMSketch) { - c, helper.sumTopN, helper.numTop = NewCMSketch(d, w), 0, 0 + c = NewCMSketch(d, w) enableTopN := helper.sampleSize/topNThreshold <= helper.sumTopN if enableTopN { - c.topN = make(map[uint64][]topNMeta) + c.topN = make(map[uint64][]*TopNMeta) } for counterKey, cnt := range helper.counter { data, scaledCount := hack.Slice(string(counterKey)), cnt*scaleRatio if enableTopN && cnt >= helper.lastVal { h1, h2 := murmur3.Sum128(data) - c.topN[h1] = append(c.topN[h1], topNMeta{h1, h2, data, scaledCount}) - helper.sumTopN += scaledCount - helper.numTop++ + c.topN[h1] = append(c.topN[h1], &TopNMeta{h2, data, scaledCount}) } else { - c.updateBytesWithDelta(data, scaledCount) + c.insertBytesByCount(data, scaledCount) } } return @@ -143,11 +141,7 @@ func buildCMSWithTopN(helper *topNHelper, d, w int32, scaleRatio uint64) (c *CMS func (c *CMSketch) calculateDefaultVal(helper *topNHelper, estimateNDV, scaleRatio, rowCount uint64) { sampleNDV := uint64(len(helper.sorted)) - if rowCount <= helper.sumTopN { - c.defaultValue = 1 - } else if estimateNDV <= uint64(helper.numTop) { - c.defaultValue = 1 - } else if estimateNDV+helper.onlyOnceItems <= uint64(sampleNDV) { + if rowCount <= (helper.sampleSize-uint64(helper.onlyOnceItems))*scaleRatio { c.defaultValue = 1 } else { estimateRemainingCount := rowCount - (helper.sampleSize-uint64(helper.onlyOnceItems))*scaleRatio @@ -155,17 +149,25 @@ func (c *CMSketch) calculateDefaultVal(helper *topNHelper, estimateNDV, scaleRat } } +func (c *CMSketch) findTopNMeta(h1, h2 uint64, d []byte) *TopNMeta { + for _, meta := range c.topN[h1] { + if meta.h2 == h2 && bytes.Equal(d, meta.Data) { + return meta + } + } + return nil +} + // queryAddTopN TopN adds count to CMSketch.topN if exists, and returns the count of such elements after insert. // If such elements does not in topn elements, nothing will happen and false will be returned. func (c *CMSketch) updateTopNWithDelta(h1, h2 uint64, d []byte, delta uint64) bool { if c.topN == nil { return false } - for _, cnt := range c.topN[h1] { - if cnt.h2 == h2 && bytes.Equal(d, cnt.data) { - cnt.count += delta - return true - } + meta := c.findTopNMeta(h1, h2, d) + if meta != nil { + meta.Count += delta + return true } return false } @@ -174,38 +176,47 @@ func (c *CMSketch) queryTopN(h1, h2 uint64, d []byte) (uint64, bool) { if c.topN == nil { return 0, false } - for _, cnt := range c.topN[h1] { - if cnt.h2 == h2 && bytes.Equal(d, cnt.data) { - return cnt.count, true - } + meta := c.findTopNMeta(h1, h2, d) + if meta != nil { + return meta.Count, true } return 0, false } // InsertBytes inserts the bytes value into the CM Sketch. func (c *CMSketch) InsertBytes(bytes []byte) { - c.updateBytesWithDelta(bytes, 1) + c.insertBytesByCount(bytes, 1) } -// updateBytesWithDelta adds the bytes value into the CM Sketch by delta. -func (c *CMSketch) updateBytesWithDelta(bytes []byte, delta uint64) { +// insertBytesByCount adds the bytes value into the TopN (if value already in TopN) or CM Sketch by delta, this does not updates c.defaultValue. +func (c *CMSketch) insertBytesByCount(bytes []byte, count uint64) { h1, h2 := murmur3.Sum128(bytes) - if c.updateTopNWithDelta(h1, h2, bytes, delta) { + if c.updateTopNWithDelta(h1, h2, bytes, count) { return } - c.count += delta + c.count += count for i := range c.table { j := (h1 + h2*uint64(i)) % uint64(c.width) - c.table[i][j] += uint32(delta) + c.table[i][j] += uint32(count) } } func (c *CMSketch) considerDefVal(cnt uint64) bool { - return cnt < 2*(c.count/uint64(c.width)) && c.defaultValue > 0 + return (cnt == 0 || (cnt > c.defaultValue && cnt < 2*(c.count/uint64(c.width)))) && c.defaultValue > 0 +} + +// updateValueBytes updates value of d to count. +func (c *CMSketch) updateValueBytes(d []byte, count uint64) { + h1, h2 := murmur3.Sum128(d) + if oriCount, ok := c.queryTopN(h1, h2, d); ok { + deltaCount := count - oriCount + c.updateTopNWithDelta(h1, h2, d, deltaCount) + } + c.setValue(h1, h2, count) } -// setValue sets the count for value that hashed into (h1, h2). -func (c *CMSketch) setValue(h1, h2 uint64, count uint32) { +// setValue sets the count for value that hashed into (h1, h2), and update defaultValue if necessary. +func (c *CMSketch) setValue(h1, h2 uint64, count uint64) { oriCount := c.queryHashValue(h1, h2) if c.considerDefVal(oriCount) { // We should update c.defaultValue if we used c.defaultValue when getting the estimate count. @@ -217,9 +228,9 @@ func (c *CMSketch) setValue(h1, h2 uint64, count uint32) { } } - c.count += uint64(count) - oriCount + c.count += count - oriCount // let it overflow naturally - deltaCount := count - uint32(oriCount) + deltaCount := uint32(count) - uint32(oriCount) for i := range c.table { j := (h1 + h2*uint64(i)) % uint64(c.width) c.table[i][j] = c.table[i][j] + deltaCount @@ -288,7 +299,6 @@ func (c *CMSketch) MergeCMSketch(rc *CMSketch) error { } // CMSketchToProto converts CMSketch to its protobuf representation. -// TODO: Encode/Decode cmsketch with Top-N func CMSketchToProto(c *CMSketch) *tipb.CMSketch { protoSketch := &tipb.CMSketch{Rows: make([]*tipb.CMSketchRow, c.depth)} for i := range c.table { @@ -297,11 +307,16 @@ func CMSketchToProto(c *CMSketch) *tipb.CMSketch { protoSketch.Rows[i].Counters[j] = c.table[i][j] } } + for _, dataSlice := range c.topN { + for _, dataMeta := range dataSlice { + protoSketch.TopN = append(protoSketch.TopN, &tipb.CMSketchTopN{Data: dataMeta.Data, Count: dataMeta.Count}) + } + } + protoSketch.DefaultValue = c.defaultValue return protoSketch } // CMSketchFromProto converts CMSketch from its protobuf representation. -// TODO: Encode/Decode cmsketch with Top-N func CMSketchFromProto(protoSketch *tipb.CMSketch) *CMSketch { if protoSketch == nil { return nil @@ -314,20 +329,32 @@ func CMSketchFromProto(protoSketch *tipb.CMSketch) *CMSketch { c.count = c.count + uint64(counter) } } + if len(protoSketch.TopN) == 0 { + return c + } + c.defaultValue = protoSketch.DefaultValue + c.topN = make(map[uint64][]*TopNMeta) + for _, e := range protoSketch.TopN { + h1, h2 := murmur3.Sum128(e.Data) + c.topN[h1] = append(c.topN[h1], &TopNMeta{h2, e.Data, e.Count}) + } return c } -// EncodeCMSketch encodes the given CMSketch to byte slice. -func EncodeCMSketch(c *CMSketch) ([]byte, error) { - if c == nil || c.count == 0 { +// EncodeCMSketchWithoutTopN encodes the given CMSketch to byte slice. +// Note that it does not include the topN. +func EncodeCMSketchWithoutTopN(c *CMSketch) ([]byte, error) { + if c == nil { return nil, nil } p := CMSketchToProto(c) - return p.Marshal() + p.TopN = nil + protoData, err := p.Marshal() + return protoData, err } -// DecodeCMSketch decode a CMSketch from the given byte slice. -func DecodeCMSketch(data []byte) (*CMSketch, error) { +// decodeCMSketch decode a CMSketch from the given byte slice. +func decodeCMSketch(data []byte, topN []*TopNMeta) (*CMSketch, error) { if data == nil { return nil, nil } @@ -336,12 +363,29 @@ func DecodeCMSketch(data []byte) (*CMSketch, error) { if err != nil { return nil, errors.Trace(err) } - if len(p.Rows) == 0 { + if len(p.Rows) == 0 && len(topN) == 0 { return nil, nil } + for _, meta := range topN { + p.TopN = append(p.TopN, &tipb.CMSketchTopN{Data: meta.Data, Count: meta.Count}) + } return CMSketchFromProto(p), nil } +// LoadCMSketchWithTopN loads the CM sketch with topN from storage. +func LoadCMSketchWithTopN(exec sqlexec.RestrictedSQLExecutor, tableID, isIndex, histID int64, cms []byte) (*CMSketch, error) { + sql := fmt.Sprintf("select HIGH_PRIORITY value, count from mysql.stats_top_n where table_id = %d and is_index = %d and hist_id = %d", tableID, isIndex, histID) + topNRows, _, err := exec.ExecRestrictedSQL(nil, sql) + if err != nil { + return nil, err + } + topN := make([]*TopNMeta, 0, len(topNRows)) + for _, row := range topNRows { + topN = append(topN, &TopNMeta{Data: row.GetBytes(0), Count: row.GetUint64(1)}) + } + return decodeCMSketch(cms, topN) +} + // TotalCount returns the count, it is only used for test. func (c *CMSketch) TotalCount() uint64 { return c.count @@ -352,7 +396,7 @@ func (c *CMSketch) Equal(rc *CMSketch) bool { if c == nil || rc == nil { return c == nil && rc == nil } - if c.width != rc.width || c.depth != rc.depth || c.count != rc.count { + if c.width != rc.width || c.depth != rc.depth || c.count != rc.count || c.defaultValue != rc.defaultValue { return false } for i := range c.table { @@ -362,6 +406,20 @@ func (c *CMSketch) Equal(rc *CMSketch) bool { } } } + if len(c.topN) != len(rc.topN) { + return false + } + for h1, topNData := range c.topN { + if len(topNData) != len(rc.topN[h1]) { + return false + } + for _, val := range topNData { + meta := rc.findTopNMeta(h1, val.h2, val.Data) + if meta == nil || meta.Count != val.Count { + return false + } + } + } return true } @@ -375,7 +433,32 @@ func (c *CMSketch) Copy() *CMSketch { tbl[i] = make([]uint32, c.width) copy(tbl[i], c.table[i]) } - return &CMSketch{count: c.count, width: c.width, depth: c.depth, table: tbl} + var topN map[uint64][]*TopNMeta + if c.topN != nil { + topN = make(map[uint64][]*TopNMeta) + for h1, vals := range c.topN { + newVals := make([]*TopNMeta, 0, len(vals)) + for _, val := range vals { + newVal := TopNMeta{h2: val.h2, Count: val.Count, Data: make([]byte, len(val.Data))} + copy(newVal.Data, val.Data) + newVals = append(newVals, &newVal) + } + topN[h1] = newVals + } + } + return &CMSketch{count: c.count, width: c.width, depth: c.depth, table: tbl, defaultValue: c.defaultValue, topN: topN} +} + +// TopN gets all the topN meta. +func (c *CMSketch) TopN() []*TopNMeta { + if c == nil { + return nil + } + topN := make([]*TopNMeta, 0, len(c.topN)) + for _, meta := range c.topN { + topN = append(topN, meta...) + } + return topN } // GetWidthAndDepth returns the width and depth of CM Sketch. diff --git a/statistics/cmsketch_test.go b/statistics/cmsketch_test.go index 084fe473f3adf..44ee8a57dfed6 100644 --- a/statistics/cmsketch_test.go +++ b/statistics/cmsketch_test.go @@ -14,6 +14,7 @@ package statistics import ( + "fmt" "math" "math/rand" "time" @@ -23,6 +24,7 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" + "github.com/spaolacci/murmur3" ) func (c *CMSketch) insert(val *types.Datum) error { @@ -148,10 +150,10 @@ func (s *testStatisticsSuite) TestCMSketchCoding(c *C) { lSketch.table[i][j] = math.MaxUint32 } } - bytes, err := EncodeCMSketch(lSketch) + bytes, err := EncodeCMSketchWithoutTopN(lSketch) c.Assert(err, IsNil) - c.Assert(len(bytes), Equals, 61455) - rSketch, err := DecodeCMSketch(bytes) + c.Assert(len(bytes), Equals, 61457) + rSketch, err := decodeCMSketch(bytes, nil) c.Assert(err, IsNil) c.Assert(lSketch.Equal(rSketch), IsTrue) } @@ -213,3 +215,26 @@ func (s *testStatisticsSuite) TestCMSketchTopNUniqueData(c *C) { c.Check(avg, Equals, uint64(0)) c.Check(len(cms.topN), Equals, 0) } + +func (s *testStatisticsSuite) TestCMSketchCodingTopN(c *C) { + lSketch := NewCMSketch(5, 2048) + lSketch.count = 2048 * (math.MaxUint32) + for i := range lSketch.table { + for j := range lSketch.table[i] { + lSketch.table[i][j] = math.MaxUint32 + } + } + lSketch.topN = make(map[uint64][]*TopNMeta) + for i := 0; i < 20; i++ { + tString := []byte(fmt.Sprintf("%20000d", i)) + h1, h2 := murmur3.Sum128(tString) + lSketch.topN[h1] = []*TopNMeta{{h2, tString, math.MaxUint64}} + } + + bytes, err := EncodeCMSketchWithoutTopN(lSketch) + c.Assert(err, IsNil) + c.Assert(len(bytes), Equals, 61457) + rSketch, err := decodeCMSketch(bytes, lSketch.TopN()) + c.Assert(err, IsNil) + c.Assert(lSketch.Equal(rSketch), IsTrue) +} diff --git a/statistics/estimate.go b/statistics/estimate.go index 0878cf1e36298..2112a8fc3767c 100644 --- a/statistics/estimate.go +++ b/statistics/estimate.go @@ -18,8 +18,6 @@ import ( ) // calculateEstimateNDV calculates the estimate ndv of a sampled data from a multisize with size total. -// count[i] stores the count of the i-th element. -// onlyOnceItems is the number of elements that occurred only once. func calculateEstimateNDV(h *topNHelper, rowCount uint64) (ndv uint64, scaleRatio uint64) { sampleSize, sampleNDV, onlyOnceItems := h.sampleSize, uint64(len(h.sorted)), h.onlyOnceItems scaleRatio = rowCount / sampleSize diff --git a/statistics/feedback.go b/statistics/feedback.go index 32e52c6fab27d..8adce8fc4c3dc 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -609,8 +609,7 @@ func UpdateCMSketch(c *CMSketch, eqFeedbacks []Feedback) *CMSketch { } newCMSketch := c.Copy() for _, fb := range eqFeedbacks { - h1, h2 := murmur3.Sum128(fb.Lower.GetBytes()) - newCMSketch.setValue(h1, h2, uint32(fb.Count)) + newCMSketch.updateValueBytes(fb.Lower.GetBytes(), uint64(fb.Count)) } return newCMSketch } @@ -728,7 +727,8 @@ func decodeFeedbackForIndex(q *QueryFeedback, pb *queryFeedback, c *CMSketch) { // decode the index point feedback, just set value count in CM Sketch start := len(pb.IndexRanges) / 2 for i := 0; i < len(pb.HashValues); i += 2 { - c.setValue(pb.HashValues[i], pb.HashValues[i+1], uint32(pb.Counts[start+i/2])) + // TODO: update using raw bytes instead of hash values. + c.setValue(pb.HashValues[i], pb.HashValues[i+1], uint64(pb.Counts[start+i/2])) } } } diff --git a/statistics/handle/bootstrap.go b/statistics/handle/bootstrap.go index 42a19b21d0499..a461bc6d6911d 100644 --- a/statistics/handle/bootstrap.go +++ b/statistics/handle/bootstrap.go @@ -103,7 +103,7 @@ func (h *Handle) initStatsHistograms4Chunk(is infoschema.InfoSchema, tables Stat if idxInfo == nil { continue } - cms, err := statistics.DecodeCMSketch(row.GetBytes(6)) + cms, err := statistics.LoadCMSketchWithTopN(h.restrictedExec, row.GetInt64(0), row.GetInt64(1), row.GetInt64(2), row.GetBytes(6)) if err != nil { cms = nil terror.Log(errors.Trace(err)) diff --git a/statistics/handle/dump_test.go b/statistics/handle/dump_test.go index a56e1163de907..a1371e8e7f925 100644 --- a/statistics/handle/dump_test.go +++ b/statistics/handle/dump_test.go @@ -116,3 +116,45 @@ func (s *testStatsSuite) TestDumpAlteredTable(c *C) { _, err = h.DumpStatsToJSON("test", table.Meta()) c.Assert(err, IsNil) } + +func (s *testStatsSuite) TestDumpCMSketchWithTopN(c *C) { + // Just test if we can store and recover the Top N elements stored in database. + defer cleanEnv(c, s.store, s.do) + testKit := testkit.NewTestKit(c, s.store) + testKit.MustExec("use test") + testKit.MustExec("create table t(a int)") + testKit.MustExec("insert into t values (1),(3),(4),(2),(5)") + testKit.MustExec("analyze table t") + + is := s.do.InfoSchema() + tbl, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + c.Assert(err, IsNil) + tableInfo := tbl.Meta() + h := s.do.StatsHandle() + h.Update(is) + + // Insert 30 fake data + fakeData := make([][]byte, 0, 30) + for i := 0; i < 30; i++ { + fakeData = append(fakeData, []byte(fmt.Sprintf("%01024d", i))) + } + cms, _, _ := statistics.NewCMSketchWithTopN(5, 2048, fakeData, 20, 100) + + stat := h.GetTableStats(tableInfo) + err = h.SaveStatsToStorage(tableInfo.ID, 1, 0, &stat.Columns[tableInfo.Columns[0].ID].Histogram, cms, 1) + c.Assert(err, IsNil) + c.Assert(h.Update(is), IsNil) + + stat = h.GetTableStats(tableInfo) + cmsFromStore := stat.Columns[tableInfo.Columns[0].ID].CMSketch + c.Assert(cmsFromStore, NotNil) + c.Check(cms.Equal(cmsFromStore), IsTrue) + + jsonTable, err := h.DumpStatsToJSON("test", tableInfo) + c.Check(err, IsNil) + err = h.LoadStatsFromJSON(is, jsonTable) + c.Check(err, IsNil) + stat = h.GetTableStats(tableInfo) + cmsFromJSON := stat.Columns[tableInfo.Columns[0].ID].CMSketch.Copy() + c.Check(cms.Equal(cmsFromJSON), IsTrue) +} diff --git a/statistics/handle/gc.go b/statistics/handle/gc.go index bbca79a972013..0cc79e89667e7 100644 --- a/statistics/handle/gc.go +++ b/statistics/handle/gc.go @@ -122,6 +122,11 @@ func (h *Handle) deleteHistStatsFromKV(physicalID int64, histID int64, isIndex i if err != nil { return } + // delete top n data + _, err = exec.Execute(context.Background(), fmt.Sprintf("delete from mysql.stats_top_n where table_id = %d and hist_id = %d and is_index = %d", physicalID, histID, isIndex)) + if err != nil { + return + } // delete all buckets _, err = exec.Execute(context.Background(), fmt.Sprintf("delete from mysql.stats_buckets where table_id = %d and hist_id = %d and is_index = %d", physicalID, histID, isIndex)) return diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index 7545d22e35cdf..b95954b78046e 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -325,7 +325,7 @@ func (h *Handle) cmSketchFromStorage(tblID int64, isIndex, histID int64) (*stati if len(rows) == 0 { return nil, nil } - return statistics.DecodeCMSketch(rows[0].GetBytes(0)) + return statistics.LoadCMSketchWithTopN(h.restrictedExec, tblID, isIndex, histID, rows[0].GetBytes(0)) } func (h *Handle) indexStatsFromStorage(row chunk.Row, table *statistics.Table, tableInfo *model.TableInfo) error { @@ -524,10 +524,23 @@ func (h *Handle) SaveStatsToStorage(tableID int64, count int64, isIndex int, hg if err != nil { return } - data, err := statistics.EncodeCMSketch(cms) + data, err := statistics.EncodeCMSketchWithoutTopN(cms) if err != nil { return } + // Delete outdated data + deleteOutdatedTopNSQL := fmt.Sprintf("delete from mysql.stats_top_n where table_id = %d and is_index = %d and hist_id = %d", tableID, isIndex, hg.ID) + _, err = exec.Execute(ctx, deleteOutdatedTopNSQL) + if err != nil { + return + } + for _, meta := range cms.TopN() { + insertSQL := fmt.Sprintf("insert into mysql.stats_top_n (table_id, is_index, hist_id, value, count) values (%d, %d, %d, X'%X', %d)", tableID, isIndex, hg.ID, meta.Data, meta.Count) + _, err = exec.Execute(ctx, insertSQL) + if err != nil { + return + } + } flag := 0 if isAnalyzed == 1 { flag = statistics.AnalyzeFlag From b05977d47e0be7690df9c78f24268baf59a6e1e0 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 29 Apr 2019 13:22:30 +0800 Subject: [PATCH 18/49] executor: ignore overflow error when construct inner key (#10244) --- executor/index_lookup_join.go | 4 ++++ executor/index_lookup_join_test.go | 9 +++++++++ 2 files changed, 13 insertions(+) diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index ac30d73745219..3d9371d14d396 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -521,6 +521,10 @@ func (iw *innerWorker) constructDatumLookupKey(task *lookUpJoinTask, rowIdx int) innerColType := iw.rowTypes[iw.keyCols[i]] innerValue, err := outerValue.ConvertTo(sc, innerColType) if err != nil { + // If the converted outerValue overflows, we don't need to lookup it. + if terror.ErrorEqual(err, types.ErrOverflow) { + return nil, nil + } return nil, err } cmp, err := outerValue.CompareDatum(sc, &innerValue) diff --git a/executor/index_lookup_join_test.go b/executor/index_lookup_join_test.go index 0df5fa6055edd..60ebb5e1e811b 100644 --- a/executor/index_lookup_join_test.go +++ b/executor/index_lookup_join_test.go @@ -143,3 +143,12 @@ func (s *testSuite1) TestInapplicableIndexJoinHint(c *C) { tk.MustQuery(`select /*+ TIDB_INLJ(t2) */ * from t1 right join t2 on t1.a=t2.a;`).Check(testkit.Rows()) tk.MustQuery(`show warnings;`).Check(testkit.Rows(`Warning 1815 Optimizer Hint /*+ TIDB_INLJ(t2) */ is inapplicable`)) } + +func (s *testSuite) TestIndexJoinOverflow(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec(`drop table if exists t1, t2`) + tk.MustExec(`create table t1(a int)`) + tk.MustExec(`insert into t1 values (-1)`) + tk.MustExec(`create table t2(a int unsigned, index idx(a));`) + tk.MustQuery(`select /*+ TIDB_INLJ(t2) */ * from t1 join t2 on t1.a = t2.a;`).Check(testkit.Rows()) +} From e96190bc309f72a8dab914b3c130c2a0b04c5ea0 Mon Sep 17 00:00:00 2001 From: Yiding Cui Date: Mon, 29 Apr 2019 13:31:29 +0800 Subject: [PATCH 19/49] store/helper, inforschema: add mem table to show region info (#10267) --- infoschema/tables.go | 117 ++++++++++++++++++++++++++++++++++++ store/helper/helper.go | 75 +++++++++++++++++++++++ store/helper/helper_test.go | 56 +++++++++++++++++ util/pdapi/const.go | 5 +- 4 files changed, 251 insertions(+), 2 deletions(-) diff --git a/infoschema/tables.go b/infoschema/tables.go index faa83fd22570d..697a68a7d1997 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/types" binaryJson "github.com/pingcap/tidb/types/json" "github.com/pingcap/tidb/util/pdapi" + "github.com/pingcap/tidb/util/set" "github.com/pingcap/tidb/util/sqlexec" ) @@ -77,6 +78,8 @@ const ( tableTiDBHotRegions = "TIDB_HOT_REGIONS" tableTiKVStoreStatus = "TIKV_STORE_STATUS" tableAnalyzeStatus = "ANALYZE_STATUS" + tableTiKVRegionStatus = "TIKV_REGION_STATUS" + tableTiKVRegionPeers = "TIKV_REGION_PEERS" ) type columnInfo struct { @@ -598,6 +601,114 @@ var tableAnalyzeStatusCols = []columnInfo{ {"STATE", mysql.TypeVarchar, 64, 0, nil, nil}, } +var tableTiKVRegionStatusCols = []columnInfo{ + {"REGION_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"START_KEY", mysql.TypeBlob, types.UnspecifiedLength, 0, nil, nil}, + {"END_KEY", mysql.TypeBlob, types.UnspecifiedLength, 0, nil, nil}, + {"EPOCH_CONF_VER", mysql.TypeLonglong, 21, 0, nil, nil}, + {"EPOCH_VERSION", mysql.TypeLonglong, 21, 0, nil, nil}, + {"WRITTEN_BYTES", mysql.TypeLonglong, 21, 0, nil, nil}, + {"READ_BYTES", mysql.TypeLonglong, 21, 0, nil, nil}, + {"APPROXIMATE_SIZE", mysql.TypeLonglong, 21, 0, nil, nil}, + {"APPROXIMATE_KEYS", mysql.TypeLonglong, 21, 0, nil, nil}, +} + +var tableTiKVRegionPeersCols = []columnInfo{ + {"REGION_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"PEER_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"STORE_ID", mysql.TypeLonglong, 21, 0, nil, nil}, + {"IS_LEARNER", mysql.TypeTiny, 1, mysql.NotNullFlag, 0, nil}, + {"IS_LEADER", mysql.TypeTiny, 1, mysql.NotNullFlag, 0, nil}, + {"STATUS", mysql.TypeVarchar, 10, 0, 0, nil}, + {"DOWN_SECONDS", mysql.TypeLonglong, 21, 0, 0, nil}, +} + +func dataForTiKVRegionStatus(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + regionsStat, err := tikvHelper.GetRegionsInfo() + if err != nil { + return nil, err + } + for _, regionStat := range regionsStat.Regions { + row := make([]types.Datum, len(tableTiKVRegionStatusCols)) + row[0].SetInt64(regionStat.ID) + row[1].SetString(regionStat.StartKey) + row[2].SetString(regionStat.EndKey) + row[3].SetInt64(regionStat.Epoch.ConfVer) + row[4].SetInt64(regionStat.Epoch.Version) + row[5].SetInt64(regionStat.WrittenBytes) + row[6].SetInt64(regionStat.ReadBytes) + row[7].SetInt64(regionStat.ApproximateSize) + row[8].SetInt64(regionStat.ApproximateKeys) + records = append(records, row) + } + return records, nil +} + +const ( + normalPeer = "NORMAL" + pendingPeer = "PENDING" + downPeer = "DOWN" +) + +func dataForTikVRegionPeers(ctx sessionctx.Context) (records [][]types.Datum, err error) { + tikvStore, ok := ctx.GetStore().(tikv.Storage) + if !ok { + return nil, errors.New("Information about TiKV region status can be gotten only when the storage is TiKV") + } + tikvHelper := &helper.Helper{ + Store: tikvStore, + RegionCache: tikvStore.GetRegionCache(), + } + regionsStat, err := tikvHelper.GetRegionsInfo() + if err != nil { + return nil, err + } + for _, regionStat := range regionsStat.Regions { + pendingPeerIDSet := set.NewInt64Set() + for _, peer := range regionStat.PendingPeers { + pendingPeerIDSet.Insert(peer.ID) + } + downPeerMap := make(map[int64]int64) + for _, peerStat := range regionStat.DownPeers { + downPeerMap[peerStat.ID] = peerStat.DownSec + } + for _, peer := range regionStat.Peers { + row := make([]types.Datum, len(tableTiKVRegionPeersCols)) + row[0].SetInt64(regionStat.ID) + row[1].SetInt64(peer.ID) + row[2].SetInt64(peer.StoreID) + if peer.ID == regionStat.Leader.ID { + row[3].SetInt64(1) + } else { + row[3].SetInt64(0) + } + if peer.IsLearner { + row[4].SetInt64(1) + } else { + row[4].SetInt64(0) + } + if pendingPeerIDSet.Exist(peer.ID) { + row[5].SetString(pendingPeer) + } else if downSec, ok := downPeerMap[peer.ID]; ok { + row[5].SetString(downPeer) + row[6].SetInt64(downSec) + } else { + row[5].SetString(normalPeer) + } + records = append(records, row) + } + } + return records, nil +} + func dataForTiKVStoreStatus(ctx sessionctx.Context) (records [][]types.Datum, err error) { tikvStore, ok := ctx.GetStore().(tikv.Storage) if !ok { @@ -1636,6 +1747,8 @@ var tableNameToColumns = map[string][]columnInfo{ tableTiDBHotRegions: tableTiDBHotRegionsCols, tableTiKVStoreStatus: tableTiKVStoreStatusCols, tableAnalyzeStatus: tableAnalyzeStatusCols, + tableTiKVRegionStatus: tableTiKVRegionStatusCols, + tableTiKVRegionPeers: tableTiKVRegionPeersCols, } func createInfoSchemaTable(handle *Handle, meta *model.TableInfo) *infoschemaTable { @@ -1735,6 +1848,10 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) fullRows, err = dataForTiKVStoreStatus(ctx) case tableAnalyzeStatus: fullRows = DataForAnalyzeStatus() + case tableTiKVRegionStatus: + fullRows, err = dataForTiKVRegionStatus(ctx) + case tableTiKVRegionPeers: + fullRows, err = dataForTikVRegionPeers(ctx) } if err != nil { return nil, err diff --git a/store/helper/helper.go b/store/helper/helper.go index d9f05f86c15d2..6e0e250755778 100644 --- a/store/helper/helper.go +++ b/store/helper/helper.go @@ -367,6 +367,81 @@ func (r *RegionFrameRange) GetIndexFrame(tableID, indexID int64, dbName, tableNa return nil } +// RegionPeer stores information of one peer. +type RegionPeer struct { + ID int64 `json:"id"` + StoreID int64 `json:"store_id"` + IsLearner bool `json:"is_learner"` +} + +// RegionEpoch stores the information about its epoch. +type RegionEpoch struct { + ConfVer int64 `json:"conf_ver"` + Version int64 `json:"version"` +} + +// RegionPeerStat stores one field `DownSec` which indicates how long it's down than `RegionPeer`. +type RegionPeerStat struct { + RegionPeer + DownSec int64 `json:"down_seconds"` +} + +// RegionInfo stores the information of one region. +type RegionInfo struct { + ID int64 `json:"id"` + StartKey string `json:"start_key"` + EndKey string `json:"end_key"` + Epoch RegionEpoch `json:"epoch"` + Peers []RegionPeer `json:"peers"` + Leader RegionPeer `json:"leader"` + DownPeers []RegionPeerStat `json:"down_peers"` + PendingPeers []RegionPeer `json:"pending_peers"` + WrittenBytes int64 `json:"written_bytes"` + ReadBytes int64 `json:"read_bytes"` + ApproximateSize int64 `json:"approximate_size"` + ApproximateKeys int64 `json:"approximate_keys"` +} + +// RegionsInfo stores the information of regions. +type RegionsInfo struct { + Count int64 `json:"count"` + Regions []RegionInfo `json:"regions"` +} + +// GetRegionsInfo gets the region information of current store by using PD's api. +func (h *Helper) GetRegionsInfo() (*RegionsInfo, error) { + etcd, ok := h.Store.(tikv.EtcdBackend) + if !ok { + return nil, errors.WithStack(errors.New("not implemented")) + } + pdHosts := etcd.EtcdAddrs() + if len(pdHosts) == 0 { + return nil, errors.New("pd unavailable") + } + req, err := http.NewRequest("GET", protocol+pdHosts[0]+pdapi.Regions, nil) + if err != nil { + return nil, errors.Trace(err) + } + timeout, cancelFunc := context.WithTimeout(context.Background(), 5*time.Second) + resp, err := http.DefaultClient.Do(req.WithContext(timeout)) + defer cancelFunc() + if err != nil { + return nil, errors.Trace(err) + } + defer func() { + err = resp.Body.Close() + if err != nil { + logutil.Logger(context.Background()).Error("close body failed", zap.Error(err)) + } + }() + var regionsInfo RegionsInfo + err = json.NewDecoder(resp.Body).Decode(®ionsInfo) + if err != nil { + return nil, errors.Trace(err) + } + return ®ionsInfo, nil +} + // StoresStat stores all information get from PD's api. type StoresStat struct { Count int `json:"count"` diff --git a/store/helper/helper_test.go b/store/helper/helper_test.go index 7343ce99e5add..30879e50c278e 100644 --- a/store/helper/helper_test.go +++ b/store/helper/helper_test.go @@ -82,6 +82,16 @@ func (s *HelperTestSuite) TestHotRegion(c *C) { c.Assert(fmt.Sprintf("%v", regionMetric), Equals, "map[1:{100 1 0}]") } +func (s *HelperTestSuite) TestTiKVRegionsInfo(c *C) { + h := helper.Helper{ + Store: s.store, + RegionCache: s.store.GetRegionCache(), + } + regionsInfo, err := h.GetRegionsInfo() + c.Assert(err, IsNil, Commentf("err: %+v", err)) + c.Assert(fmt.Sprintf("%v", regionsInfo), Equals, "&{1 [{1 test testtest {1 1} [{2 1 false}] {2 1 false} [] [] 100 1000 500 200}]}") +} + func (s *HelperTestSuite) TestTiKVStoresStat(c *C) { h := helper.Helper{ Store: s.store, @@ -97,6 +107,7 @@ func (s *HelperTestSuite) TestTiKVStoresStat(c *C) { func (s *HelperTestSuite) mockPDHTTPServer(c *C) { router := mux.NewRouter() router.HandleFunc(pdapi.HotRead, s.mockHotRegionResponse) + router.HandleFunc(pdapi.Regions, s.mockTiKVRegionsInfoResponse) router.HandleFunc(pdapi.Stores, s.mockStoreStatResponse) serverMux := http.NewServeMux() serverMux.Handle("/", router) @@ -132,6 +143,51 @@ func (s *HelperTestSuite) mockHotRegionResponse(w http.ResponseWriter, req *http } +func (s *HelperTestSuite) mockTiKVRegionsInfoResponse(w http.ResponseWriter, req *http.Request) { + w.Header().Set("Content-Type", "application/json") + w.WriteHeader(http.StatusOK) + resp := helper.RegionsInfo{ + Count: 1, + Regions: []helper.RegionInfo{ + { + ID: 1, + StartKey: "test", + EndKey: "testtest", + Epoch: helper.RegionEpoch{ + ConfVer: 1, + Version: 1, + }, + Peers: []helper.RegionPeer{ + { + ID: 2, + StoreID: 1, + IsLearner: false, + }, + }, + Leader: helper.RegionPeer{ + ID: 2, + StoreID: 1, + IsLearner: false, + }, + DownPeers: nil, + PendingPeers: nil, + WrittenBytes: 100, + ReadBytes: 1000, + ApproximateKeys: 200, + ApproximateSize: 500, + }, + }, + } + data, err := json.MarshalIndent(resp, "", " ") + if err != nil { + log.Panic("json marshal failed", zap.Error(err)) + } + _, err = w.Write(data) + if err != nil { + log.Panic("write http response failed", zap.Error(err)) + } +} + func (s *HelperTestSuite) mockStoreStatResponse(w http.ResponseWriter, req *http.Request) { w.Header().Set("Content-Type", "application/json") w.WriteHeader(http.StatusOK) diff --git a/util/pdapi/const.go b/util/pdapi/const.go index 086d55f99a781..436784f627245 100644 --- a/util/pdapi/const.go +++ b/util/pdapi/const.go @@ -13,9 +13,10 @@ package pdapi -// HotRead / HotWrite is the pd apis to get the corresponding hot region information. +// The following constants are the APIs of PD server. const ( HotRead = "/pd/api/v1/hotspot/regions/read" - HotWrite = "/pd/api/v1/hotspot/regions/read" + HotWrite = "/pd/api/v1/hotspot/regions/write" + Regions = "/pd/api/v1/regions" Stores = "/pd/api/v1/stores" ) From 69b02a3736d51369c169b1569824bca3abc1d4f6 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 29 Apr 2019 16:38:46 +0800 Subject: [PATCH 20/49] *: make config items `Performance.*`, `OOMAction` and `MemQuotaQuery` support hot-reloading (#10295) --- config/config.go | 141 +++++++++++++++++++++++++- config/config_test.go | 25 +++++ executor/seqtest/prepared_test.go | 22 ++-- executor/seqtest/seq_executor_test.go | 4 +- go.mod | 1 + planner/core/cache.go | 3 +- planner/core/cbo_test.go | 8 +- planner/core/optimizer.go | 5 +- planner/core/point_get_plan_test.go | 2 +- planner/core/prepare_test.go | 10 +- server/http_handler.go | 13 +++ server/http_status.go | 1 + session/session.go | 4 +- statistics/feedback.go | 5 +- statistics/handle/handle.go | 5 +- statistics/handle/update.go | 4 +- statistics/handle/update_test.go | 26 ++--- statistics/table.go | 5 +- tidb-server/main.go | 107 +++++++++---------- 19 files changed, 279 insertions(+), 112 deletions(-) diff --git a/config/config.go b/config/config.go index f5112120320d7..cada69fe3edbd 100644 --- a/config/config.go +++ b/config/config.go @@ -14,17 +14,23 @@ package config import ( + "bytes" + "context" "crypto/tls" "crypto/x509" "fmt" "io/ioutil" + "os" + "reflect" "strings" + "sync" "time" "github.com/BurntSushi/toml" "github.com/pingcap/errors" "github.com/pingcap/tidb/util/logutil" tracing "github.com/uber/jaeger-client-go/config" + "go.uber.org/atomic" ) // Config number limitations @@ -370,7 +376,14 @@ var defaultConf = Config{ }, } -var globalConf = defaultConf +var ( + globalConf = atomic.Value{} + reloadConfPath = "" + confReloader func(nc, c *Config) + confReloadLock sync.Mutex + supportedReloadConfigs = make(map[string]struct{}, 32) + supportedReloadConfList = make([]string, 0, 32) +) // NewConfig creates a new config instance with default value. func NewConfig() *Config { @@ -378,11 +391,89 @@ func NewConfig() *Config { return &conf } +// SetConfReloader sets reload config path and a reloader. +// It should be called only once at start time. +func SetConfReloader(cpath string, reloader func(nc, c *Config), confItems ...string) { + reloadConfPath = cpath + confReloader = reloader + for _, item := range confItems { + supportedReloadConfigs[item] = struct{}{} + supportedReloadConfList = append(supportedReloadConfList, item) + } +} + // GetGlobalConfig returns the global configuration for this server. // It should store configuration from command line and configuration file. // Other parts of the system can read the global configuration use this function. func GetGlobalConfig() *Config { - return &globalConf + return globalConf.Load().(*Config) +} + +// ReloadGlobalConfig reloads global configuration for this server. +func ReloadGlobalConfig() error { + confReloadLock.Lock() + defer confReloadLock.Unlock() + + nc := NewConfig() + if err := nc.Load(reloadConfPath); err != nil { + return err + } + if err := nc.Valid(); err != nil { + return err + } + c := GetGlobalConfig() + + diffs := collectsDiff(*nc, *c, "") + if len(diffs) == 0 { + return nil + } + var formattedDiff bytes.Buffer + for k, vs := range diffs { + formattedDiff.WriteString(fmt.Sprintf(", %v:%v->%v", k, vs[1], vs[0])) + } + unsupported := make([]string, 0, 2) + for k := range diffs { + if _, ok := supportedReloadConfigs[k]; !ok { + unsupported = append(unsupported, k) + } + } + if len(unsupported) > 0 { + return fmt.Errorf("reloading config %v is not supported, only %v are supported now, "+ + "your changes%s", unsupported, supportedReloadConfList, formattedDiff.String()) + } + + confReloader(nc, c) + globalConf.Store(nc) + logutil.Logger(context.Background()).Info("reload config changes" + formattedDiff.String()) + return nil +} + +// collectsDiff collects different config items. +// map[string][]string -> map[field path][]{new value, old value} +func collectsDiff(i1, i2 interface{}, fieldPath string) map[string][]interface{} { + diff := make(map[string][]interface{}) + t := reflect.TypeOf(i1) + if t.Kind() != reflect.Struct { + if reflect.DeepEqual(i1, i2) { + return diff + } + diff[fieldPath] = []interface{}{i1, i2} + return diff + } + + v1 := reflect.ValueOf(i1) + v2 := reflect.ValueOf(i2) + for i := 0; i < v1.NumField(); i++ { + p := t.Field(i).Name + if fieldPath != "" { + p = fieldPath + "." + p + } + m := collectsDiff(v1.Field(i).Interface(), v2.Field(i).Interface(), p) + for k, v := range m { + diff[k] = v + } + } + return diff } // Load loads config options from a toml file. @@ -406,6 +497,51 @@ func (c *Config) Load(confFile string) error { return err } +// Valid checks if this config is valid. +func (c *Config) Valid() error { + if c.Security.SkipGrantTable && !hasRootPrivilege() { + return fmt.Errorf("TiDB run with skip-grant-table need root privilege") + } + if _, ok := ValidStorage[c.Store]; !ok { + nameList := make([]string, 0, len(ValidStorage)) + for k, v := range ValidStorage { + if v { + nameList = append(nameList, k) + } + } + return fmt.Errorf("invalid store=%s, valid storages=%v", c.Store, nameList) + } + if c.Store == "mocktikv" && !c.RunDDL { + return fmt.Errorf("can't disable DDL on mocktikv") + } + if c.Log.File.MaxSize > MaxLogFileSize { + return fmt.Errorf("invalid max log file size=%v which is larger than max=%v", c.Log.File.MaxSize, MaxLogFileSize) + } + c.OOMAction = strings.ToLower(c.OOMAction) + + // lower_case_table_names is allowed to be 0, 1, 2 + if c.LowerCaseTableNames < 0 || c.LowerCaseTableNames > 2 { + return fmt.Errorf("lower-case-table-names should be 0 or 1 or 2") + } + + if c.TxnLocalLatches.Enabled && c.TxnLocalLatches.Capacity == 0 { + return fmt.Errorf("txn-local-latches.capacity can not be 0") + } + + // For tikvclient. + if c.TiKVClient.GrpcConnectionCount == 0 { + return fmt.Errorf("grpc-connection-count should be greater than 0") + } + if c.TiKVClient.MaxTxnTimeUse == 0 { + return fmt.Errorf("max-txn-time-use should be greater than 0") + } + return nil +} + +func hasRootPrivilege() bool { + return os.Geteuid() == 0 +} + // ToLogConfig converts *Log to *logutil.LogConfig. func (l *Log) ToLogConfig() *logutil.LogConfig { return logutil.NewLogConfig(l.Level, l.Format, l.SlowQueryFile, l.File, l.DisableTimestamp) @@ -433,6 +569,7 @@ func (t *OpenTracing) ToTracingConfig() *tracing.Configuration { } func init() { + globalConf.Store(&defaultConf) if checkBeforeDropLDFlag == "1" { CheckTableBeforeDrop = true } diff --git a/config/config_test.go b/config/config_test.go index 06110c2da9cc6..237bf1a1b85ba 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -169,3 +169,28 @@ xkNuJ2BlEGkwWLiRbKy1lNBBFUXKuhh3L/EIY10WTnr3TQzeL6H1 c.Assert(os.Remove(certFile), IsNil) c.Assert(os.Remove(keyFile), IsNil) } + +func (s *testConfigSuite) TestConfigDiff(c *C) { + c1 := NewConfig() + c2 := &Config{} + *c2 = *c1 + c1.OOMAction = "c1" + c2.OOMAction = "c2" + c1.MemQuotaQuery = 2333 + c2.MemQuotaQuery = 3222 + c1.Performance.CrossJoin = true + c2.Performance.CrossJoin = false + c1.Performance.FeedbackProbability = 2333 + c2.Performance.FeedbackProbability = 23.33 + + diffs := collectsDiff(*c1, *c2, "") + c.Assert(len(diffs), Equals, 4) + c.Assert(diffs["OOMAction"][0], Equals, "c1") + c.Assert(diffs["OOMAction"][1], Equals, "c2") + c.Assert(diffs["MemQuotaQuery"][0], Equals, int64(2333)) + c.Assert(diffs["MemQuotaQuery"][1], Equals, int64(3222)) + c.Assert(diffs["Performance.CrossJoin"][0], Equals, true) + c.Assert(diffs["Performance.CrossJoin"][1], Equals, false) + c.Assert(diffs["Performance.FeedbackProbability"][0], Equals, float64(2333)) + c.Assert(diffs["Performance.FeedbackProbability"][1], Equals, float64(23.33)) +} diff --git a/executor/seqtest/prepared_test.go b/executor/seqtest/prepared_test.go index cbc4d95475b03..46f4f64a5a41d 100644 --- a/executor/seqtest/prepared_test.go +++ b/executor/seqtest/prepared_test.go @@ -45,7 +45,7 @@ func (s *seqTestSuite) TestPrepared(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -262,7 +262,7 @@ func (s *seqTestSuite) TestPreparedLimitOffset(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -305,7 +305,7 @@ func (s *seqTestSuite) TestPreparedNullParam(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -349,7 +349,7 @@ func (s *seqTestSuite) TestPrepareWithAggregation(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -384,7 +384,7 @@ func (s *seqTestSuite) TestPreparedIssue7579(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -439,7 +439,7 @@ func (s *seqTestSuite) TestPreparedInsert(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -522,7 +522,7 @@ func (s *seqTestSuite) TestPreparedUpdate(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -582,7 +582,7 @@ func (s *seqTestSuite) TestPreparedDelete(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists prepare_test") @@ -636,7 +636,7 @@ func (s *seqTestSuite) TestPrepareDealloc(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -681,7 +681,7 @@ func (s *seqTestSuite) TestPreparedIssue8153(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -739,7 +739,7 @@ func (s *seqTestSuite) TestPreparedIssue8644(c *C) { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - plannercore.PreparedPlanCacheMaxMemory = math.MaxUint64 + plannercore.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 7bc877411f3de..e6b3cf3727647 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -777,14 +777,14 @@ func (s *seqTestSuite) TestCartesianProduct(c *C) { tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(c1 int)") - plannercore.AllowCartesianProduct = false + plannercore.AllowCartesianProduct.Store(false) err := tk.ExecToErr("select * from t t1, t t2") c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue) err = tk.ExecToErr("select * from t t1 left join t t2 on 1") c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue) err = tk.ExecToErr("select * from t t1 right join t t2 on 1") c.Check(plannercore.ErrCartesianProductUnsupported.Equal(err), IsTrue) - plannercore.AllowCartesianProduct = true + plannercore.AllowCartesianProduct.Store(true) } type checkPrioClient struct { diff --git a/go.mod b/go.mod index bc03df1110c07..7be334aa05bc4 100644 --- a/go.mod +++ b/go.mod @@ -61,6 +61,7 @@ require ( github.com/uber/jaeger-client-go v2.15.0+incompatible github.com/uber/jaeger-lib v1.5.0 // indirect github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d // indirect + go.uber.org/atomic v1.3.2 go.uber.org/zap v1.9.1 golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e golang.org/x/sys v0.0.0-20190109145017-48ac38b7c8cb // indirect diff --git a/planner/core/cache.go b/planner/core/cache.go index b8fcc669ceecf..c4bd455ff1d9d 100644 --- a/planner/core/cache.go +++ b/planner/core/cache.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/hack" "github.com/pingcap/tidb/util/kvcache" + atomic2 "go.uber.org/atomic" ) var ( @@ -34,7 +35,7 @@ var ( // PreparedPlanCacheMemoryGuardRatio stores the global config "prepared-plan-cache-memory-guard-ratio". PreparedPlanCacheMemoryGuardRatio float64 // PreparedPlanCacheMaxMemory stores the max memory size defined in the global config "performance-max-memory". - PreparedPlanCacheMaxMemory uint64 + PreparedPlanCacheMaxMemory atomic2.Uint64 ) const ( diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index 3a739fb9028c5..f817600b4e1aa 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -216,9 +216,9 @@ func (s *testAnalyzeSuite) TestEstimation(c *C) { defer func() { dom.Close() store.Close() - statistics.RatioOfPseudoEstimate = 0.7 + statistics.RatioOfPseudoEstimate.Store(0.7) }() - statistics.RatioOfPseudoEstimate = 10.0 + statistics.RatioOfPseudoEstimate.Store(10.0) testKit.MustExec("use test") testKit.MustExec("create table t (a int)") testKit.MustExec("insert into t values (1), (2), (3), (4), (5), (6), (7), (8), (9), (10)") @@ -575,13 +575,13 @@ func (s *testAnalyzeSuite) TestOutdatedAnalyze(c *C) { testKit.MustExec("insert into t select * from t") h.DumpStatsDeltaToKV(handle.DumpAll) c.Assert(h.Update(dom.InfoSchema()), IsNil) - statistics.RatioOfPseudoEstimate = 10.0 + statistics.RatioOfPseudoEstimate.Store(10.0) testKit.MustQuery("explain select * from t where a <= 5 and b <= 5").Check(testkit.Rows( "TableReader_7 35.91 root data:Selection_6", "└─Selection_6 35.91 cop le(test.t.a, 5), le(test.t.b, 5)", " └─TableScan_5 80.00 cop table:t, range:[-inf,+inf], keep order:false", )) - statistics.RatioOfPseudoEstimate = 0.7 + statistics.RatioOfPseudoEstimate.Store(0.7) testKit.MustQuery("explain select * from t where a <= 5 and b <= 5").Check(testkit.Rows( "IndexLookUp_11 8.84 root ", "├─IndexScan_8 26.59 cop table:t, index:a, range:[-inf,5], keep order:false, stats:pseudo", diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index cdee37b93fef2..e0fd6c507c304 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -24,13 +24,14 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" + "go.uber.org/atomic" ) // OptimizeAstNode optimizes the query to a physical plan directly. var OptimizeAstNode func(ctx sessionctx.Context, node ast.Node, is infoschema.InfoSchema) (Plan, error) // AllowCartesianProduct means whether tidb allows cartesian join without equal conditions. -var AllowCartesianProduct = true +var AllowCartesianProduct = atomic.NewBool(true) const ( flagPrunColumns uint64 = 1 << iota @@ -102,7 +103,7 @@ func DoOptimize(flag uint64, logic LogicalPlan) (PhysicalPlan, error) { if err != nil { return nil, err } - if !AllowCartesianProduct && existsCartesianProduct(logic) { + if !AllowCartesianProduct.Load() && existsCartesianProduct(logic) { return nil, errors.Trace(ErrCartesianProductUnsupported) } physical, err := physicalOptimize(logic) diff --git a/planner/core/point_get_plan_test.go b/planner/core/point_get_plan_test.go index b2f8858b36b5b..6a548bca53cdc 100644 --- a/planner/core/point_get_plan_test.go +++ b/planner/core/point_get_plan_test.go @@ -51,7 +51,7 @@ func (s *testPointGetSuite) TestPointGetPlanCache(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, c int, key idx_bc(b,c))") diff --git a/planner/core/prepare_test.go b/planner/core/prepare_test.go index f5af8954eec94..b681b19f4a64a 100644 --- a/planner/core/prepare_test.go +++ b/planner/core/prepare_test.go @@ -58,7 +58,7 @@ func (s *testPrepareSuite) TestPrepareCache(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int primary key, b int, c int, index idx1(b, a), index idx2(b))") @@ -108,7 +108,7 @@ func (s *testPrepareSuite) TestPrepareCacheIndexScan(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int, b int, c int, primary key (a, b))") @@ -142,7 +142,7 @@ func (s *testPlanSuite) TestPrepareCacheDeferredFunction(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -202,7 +202,7 @@ func (s *testPrepareSuite) TestPrepareCacheNow(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec(`prepare stmt1 from "select now(), current_timestamp(), utc_timestamp(), unix_timestamp(), sleep(0.1), now(), current_timestamp(), utc_timestamp(), unix_timestamp()"`) // When executing one statement at the first time, we don't usTestPrepareCacheDeferredFunctione cache, so we need to execute it at least twice to test the cache. @@ -286,7 +286,7 @@ func (s *testPrepareSuite) TestPrepareTableAsNameOnGroupByWithCache(c *C) { core.PreparedPlanCacheMemoryGuardRatio = 0.1 // PreparedPlanCacheMaxMemory is set to MAX_UINT64 to make sure the cache // behavior would not be effected by the uncertain memory utilization. - core.PreparedPlanCacheMaxMemory = math.MaxUint64 + core.PreparedPlanCacheMaxMemory.Store(math.MaxUint64) tk.MustExec("use test") tk.MustExec("drop table if exists t1") tk.MustExec(`create table t1 ( diff --git a/server/http_handler.go b/server/http_handler.go index 3a2db8174d8e4..2e3814ab24656 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -590,6 +590,19 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { } } +// configReloadHandler is the handler for reloading config online. +type configReloadHandler struct { +} + +// ServeHTTP handles request of reloading config for this server. +func (h configReloadHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) { + if err := config.ReloadGlobalConfig(); err != nil { + writeError(w, err) + } else { + writeData(w, "success!") + } +} + // ServeHTTP recovers binlog service. func (h binlogRecover) ServeHTTP(w http.ResponseWriter, req *http.Request) { binloginfo.DisableSkipBinlogFlag() diff --git a/server/http_status.go b/server/http_status.go index 62bd51c548c4f..af90a49e54e00 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -60,6 +60,7 @@ func (s *Server) startHTTPServer() { router.Handle("/stats/dump/{db}/{table}", s.newStatsHandler()).Name("StatsDump") router.Handle("/settings", settingsHandler{}).Name("Settings") + router.Handle("/reload-config", configReloadHandler{}).Name("ConfigReload") router.Handle("/binlog/recover", binlogRecover{}).Name("BinlogRecover") tikvHandlerTool := s.newTikvHandlerTool() diff --git a/session/session.go b/session/session.go index 7c57368857127..5d297a7476367 100644 --- a/session/session.go +++ b/session/session.go @@ -1458,7 +1458,7 @@ func createSession(store kv.Storage) (*session, error) { } if plannercore.PreparedPlanCacheEnabled() { s.preparedPlanCache = kvcache.NewSimpleLRUCache(plannercore.PreparedPlanCacheCapacity, - plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory) + plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory.Load()) } s.mu.values = make(map[fmt.Stringer]interface{}) domain.BindDomain(s, dom) @@ -1481,7 +1481,7 @@ func createSessionWithDomain(store kv.Storage, dom *domain.Domain) (*session, er } if plannercore.PreparedPlanCacheEnabled() { s.preparedPlanCache = kvcache.NewSimpleLRUCache(plannercore.PreparedPlanCacheCapacity, - plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory) + plannercore.PreparedPlanCacheMemoryGuardRatio, plannercore.PreparedPlanCacheMaxMemory.Load()) } s.mu.values = make(map[fmt.Stringer]interface{}) domain.BindDomain(s, dom) diff --git a/statistics/feedback.go b/statistics/feedback.go index 8adce8fc4c3dc..95abf21dfc0ce 100644 --- a/statistics/feedback.go +++ b/statistics/feedback.go @@ -36,6 +36,7 @@ import ( "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/ranger" "github.com/spaolacci/murmur3" + "go.uber.org/atomic" "go.uber.org/zap" ) @@ -83,7 +84,7 @@ var ( // MaxNumberOfRanges is the max number of ranges before split to collect feedback. MaxNumberOfRanges = 20 // FeedbackProbability is the probability to collect the feedback. - FeedbackProbability = 0.0 + FeedbackProbability = atomic.NewFloat64(0) ) // CalcErrorRate calculates the error rate the current QueryFeedback. @@ -106,7 +107,7 @@ func (q *QueryFeedback) CollectFeedback(numOfRanges int) bool { if q.Hist == nil || q.Hist.Len() == 0 { return false } - if numOfRanges > MaxNumberOfRanges || rand.Float64() > FeedbackProbability { + if numOfRanges > MaxNumberOfRanges || rand.Float64() > FeedbackProbability.Load() { return false } return true diff --git a/statistics/handle/handle.go b/statistics/handle/handle.go index b95954b78046e..8432fdfba2535 100644 --- a/statistics/handle/handle.go +++ b/statistics/handle/handle.go @@ -34,6 +34,7 @@ import ( "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/sqlexec" + atomic2 "go.uber.org/atomic" "go.uber.org/zap" ) @@ -89,7 +90,7 @@ func (h *Handle) Clear() { } // MaxQueryFeedbackCount is the max number of feedback that cache in memory. -var MaxQueryFeedbackCount = 1 << 10 +var MaxQueryFeedbackCount = atomic2.NewInt64(1 << 10) // NewHandle creates a Handle for update stats. func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { @@ -98,7 +99,7 @@ func NewHandle(ctx sessionctx.Context, lease time.Duration) *Handle { listHead: &SessionStatsCollector{mapper: make(tableDeltaMap), rateMap: make(errorRateDeltaMap)}, globalMap: make(tableDeltaMap), Lease: lease, - feedback: make([]*statistics.QueryFeedback, 0, MaxQueryFeedbackCount), + feedback: make([]*statistics.QueryFeedback, 0, MaxQueryFeedbackCount.Load()), } // It is safe to use it concurrently because the exec won't touch the ctx. if exec, ok := ctx.(sqlexec.RestrictedSQLExecutor); ok { diff --git a/statistics/handle/update.go b/statistics/handle/update.go index bfff6364aece3..24ad2fe7e1c85 100644 --- a/statistics/handle/update.go +++ b/statistics/handle/update.go @@ -160,7 +160,7 @@ func (s *SessionStatsCollector) Update(id int64, delta int64, count int64, colSi func mergeQueryFeedback(lq []*statistics.QueryFeedback, rq []*statistics.QueryFeedback) []*statistics.QueryFeedback { for _, q := range rq { - if len(lq) >= MaxQueryFeedbackCount { + if len(lq) >= int(MaxQueryFeedbackCount.Load()) { break } lq = append(lq, q) @@ -198,7 +198,7 @@ func (s *SessionStatsCollector) StoreQueryFeedback(feedback interface{}, h *Hand defer s.Unlock() isIndex := q.Tp == statistics.IndexType s.rateMap.update(q.PhysicalID, q.Hist.ID, rate, isIndex) - if len(s.feedback) < MaxQueryFeedbackCount { + if len(s.feedback) < int(MaxQueryFeedbackCount.Load()) { s.feedback = append(s.feedback, q) } return nil diff --git a/statistics/handle/update_test.go b/statistics/handle/update_test.go index 8748e7857b3af..f91b9c77587d6 100644 --- a/statistics/handle/update_test.go +++ b/statistics/handle/update_test.go @@ -545,7 +545,7 @@ func (s *testStatsSuite) TestUpdateErrorRate(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") @@ -615,7 +615,7 @@ func (s *testStatsSuite) TestUpdatePartitionErrorRate(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit := testkit.NewTestKit(c, s.store) testKit.MustExec("use test") @@ -733,7 +733,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) tests := []struct { sql string hist string @@ -799,7 +799,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { } // Test collect feedback by probability. - statistics.FeedbackProbability = 0 + statistics.FeedbackProbability.Store(0) statistics.MaxNumberOfRanges = oriNumber for _, t := range tests { testKit.MustQuery(t.sql) @@ -809,7 +809,7 @@ func (s *testStatsSuite) TestQueryFeedback(c *C) { } // Test that after drop stats, the feedback won't cause panic. - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) for _, t := range tests { testKit.MustQuery(t.sql) } @@ -845,7 +845,7 @@ func (s *testStatsSuite) TestQueryFeedbackForPartition(c *C) { statistics.FeedbackProbability = oriProbability }() h := s.do.StatsHandle() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) tests := []struct { sql string hist string @@ -969,7 +969,7 @@ func (s *testStatsSuite) TestUpdateStatsByLocalFeedback(c *C) { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -1022,7 +1022,7 @@ func (s *testStatsSuite) TestUpdatePartitionStatsByLocalFeedback(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) is := s.do.InfoSchema() table, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) @@ -1094,7 +1094,7 @@ func (s *testStatsSuite) TestLogDetailedInfo(c *C) { s.do.StatsHandle().Lease = oriLease log.SetLevel(oriLevel) }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) handle.MinLogScanCount = 0 handle.MinLogErrorRate = 0 s.do.StatsHandle().Lease = 1 @@ -1264,7 +1264,7 @@ func (s *testStatsSuite) TestIndexQueryFeedback(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), c bigint(64), index idx_ab(a,b), index idx_ac(a,c), index idx_b(b))") @@ -1341,7 +1341,7 @@ func (s *testStatsSuite) TestAbnormalIndexFeedback(c *C) { defer func() { statistics.FeedbackProbability = oriProbability }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit.MustExec("use test") testKit.MustExec("create table t (a bigint(64), b bigint(64), index idx_ab(a,b))") @@ -1410,7 +1410,7 @@ func (s *testStatsSuite) TestFeedbackRanges(c *C) { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit.MustExec("use test") testKit.MustExec("create table t (a tinyint, b tinyint, primary key(a), index idx(a, b))") @@ -1479,7 +1479,7 @@ func (s *testStatsSuite) TestUnsignedFeedbackRanges(c *C) { statistics.FeedbackProbability = oriProbability statistics.MaxNumberOfRanges = oriNumber }() - statistics.FeedbackProbability = 1 + statistics.FeedbackProbability.Store(1) testKit.MustExec("use test") testKit.MustExec("create table t (a tinyint unsigned, primary key(a))") diff --git a/statistics/table.go b/statistics/table.go index ce666369d845f..9fc6964020fe3 100644 --- a/statistics/table.go +++ b/statistics/table.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/ranger" + "go.uber.org/atomic" ) const ( @@ -160,11 +161,11 @@ func (n *neededColumnMap) Delete(col tableColumnID) { // RatioOfPseudoEstimate means if modifyCount / statsTblCount is greater than this ratio, we think the stats is invalid // and use pseudo estimation. -var RatioOfPseudoEstimate = 0.7 +var RatioOfPseudoEstimate = atomic.NewFloat64(0.7) // IsOutdated returns true if the table stats is outdated. func (t *Table) IsOutdated() bool { - if t.Count > 0 && float64(t.ModifyCount)/float64(t.Count) > RatioOfPseudoEstimate { + if t.Count > 0 && float64(t.ModifyCount)/float64(t.Count) > RatioOfPseudoEstimate.Load() { return true } return false diff --git a/tidb-server/main.go b/tidb-server/main.go index 73ee8610c0fca..98c9463b901df 100644 --- a/tidb-server/main.go +++ b/tidb-server/main.go @@ -20,7 +20,6 @@ import ( "os" "runtime" "strconv" - "strings" "sync/atomic" "time" @@ -29,7 +28,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" - pd "github.com/pingcap/pd/client" + "github.com/pingcap/pd/client" pumpcli "github.com/pingcap/tidb-tools/tidb-binlog/pump_client" "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/ddl" @@ -148,7 +147,10 @@ func main() { registerMetrics() configWarning := loadConfig() overrideConfig() - validateConfig() + if err := cfg.Valid(); err != nil { + fmt.Fprintln(os.Stderr, "invalid config", err) + os.Exit(1) + } if *configCheck { fmt.Println("config check successful") os.Exit(0) @@ -289,10 +291,6 @@ func parseDuration(lease string) time.Duration { return dur } -func hasRootPrivilege() bool { - return os.Geteuid() == 0 -} - func flagBoolean(name string, defaultVal bool, usage string) *bool { if !defaultVal { // Fix #4125, golang do not print default false value in usage, so we append it. @@ -305,6 +303,9 @@ func flagBoolean(name string, defaultVal bool, usage string) *bool { func loadConfig() string { cfg = config.GetGlobalConfig() if *configPath != "" { + // Not all config items are supported now. + config.SetConfReloader(*configPath, reloadConfig, hotReloadConfigItems...) + err := cfg.Load(*configPath) // This block is to accommodate an interim situation where strict config checking // is not the default behavior of TiDB. The warning message must be deferred until @@ -318,6 +319,37 @@ func loadConfig() string { return "" } +// hotReloadConfigItems lists all config items which support hot-reload. +var hotReloadConfigItems = []string{"Performance.MaxProcs", "Performance.MaxMemory", "Performance.CrossJoin", + "Performance.FeedbackProbability", "Performance.QueryFeedbackLimit", "Performance.PseudoEstimateRatio", + "OOMAction", "MemQuotaQuery"} + +func reloadConfig(nc, c *config.Config) { + // Just a part of config items need to be reload explicitly. + // Some of them like OOMAction are always used by getting from global config directly + // like config.GetGlobalConfig().OOMAction. + // These config items will become available naturally after the global config pointer + // is updated in function ReloadGlobalConfig. + if nc.Performance.MaxProcs != c.Performance.MaxProcs { + runtime.GOMAXPROCS(int(nc.Performance.MaxProcs)) + } + if nc.Performance.MaxMemory != c.Performance.MaxMemory { + plannercore.PreparedPlanCacheMaxMemory.Store(nc.Performance.MaxMemory) + } + if nc.Performance.CrossJoin != c.Performance.CrossJoin { + plannercore.AllowCartesianProduct.Store(nc.Performance.CrossJoin) + } + if nc.Performance.FeedbackProbability != c.Performance.FeedbackProbability { + statistics.FeedbackProbability.Store(nc.Performance.FeedbackProbability) + } + if nc.Performance.QueryFeedbackLimit != c.Performance.QueryFeedbackLimit { + handle.MaxQueryFeedbackCount.Store(int64(nc.Performance.QueryFeedbackLimit)) + } + if nc.Performance.PseudoEstimateRatio != c.Performance.PseudoEstimateRatio { + statistics.RatioOfPseudoEstimate.Store(nc.Performance.PseudoEstimateRatio) + } +} + func overrideConfig() { actualFlags := make(map[string]bool) flag.Visit(func(f *flag.Flag) { @@ -410,53 +442,6 @@ func overrideConfig() { } } -func validateConfig() { - if cfg.Security.SkipGrantTable && !hasRootPrivilege() { - log.Error("TiDB run with skip-grant-table need root privilege.") - os.Exit(-1) - } - if _, ok := config.ValidStorage[cfg.Store]; !ok { - nameList := make([]string, 0, len(config.ValidStorage)) - for k, v := range config.ValidStorage { - if v { - nameList = append(nameList, k) - } - } - log.Error("validate config", zap.Strings("valid storages", nameList)) - os.Exit(-1) - } - if cfg.Store == "mocktikv" && !cfg.RunDDL { - log.Error("can't disable DDL on mocktikv") - os.Exit(-1) - } - if cfg.Log.File.MaxSize > config.MaxLogFileSize { - log.Error("validate config", zap.Int("log max-size should not be larger than", config.MaxLogFileSize)) - os.Exit(-1) - } - cfg.OOMAction = strings.ToLower(cfg.OOMAction) - - // lower_case_table_names is allowed to be 0, 1, 2 - if cfg.LowerCaseTableNames < 0 || cfg.LowerCaseTableNames > 2 { - log.Error("lower-case-table-names should be 0 or 1 or 2.") - os.Exit(-1) - } - - if cfg.TxnLocalLatches.Enabled && cfg.TxnLocalLatches.Capacity == 0 { - log.Error("txn-local-latches.capacity can not be 0") - os.Exit(-1) - } - - // For tikvclient. - if cfg.TiKVClient.GrpcConnectionCount == 0 { - log.Error("grpc-connection-count should be greater than 0") - os.Exit(-1) - } - if cfg.TiKVClient.MaxTxnTimeUse == 0 { - log.Error("max-txn-time-use should be greater than 0") - os.Exit(-1) - } -} - func setGlobalVars() { ddlLeaseDuration := parseDuration(cfg.Lease) session.SetSchemaLease(ddlLeaseDuration) @@ -464,14 +449,14 @@ func setGlobalVars() { statsLeaseDuration := parseDuration(cfg.Performance.StatsLease) session.SetStatsLease(statsLeaseDuration) domain.RunAutoAnalyze = cfg.Performance.RunAutoAnalyze - statistics.FeedbackProbability = cfg.Performance.FeedbackProbability - handle.MaxQueryFeedbackCount = int(cfg.Performance.QueryFeedbackLimit) - statistics.RatioOfPseudoEstimate = cfg.Performance.PseudoEstimateRatio + statistics.FeedbackProbability.Store(cfg.Performance.FeedbackProbability) + handle.MaxQueryFeedbackCount.Store(int64(cfg.Performance.QueryFeedbackLimit)) + statistics.RatioOfPseudoEstimate.Store(cfg.Performance.PseudoEstimateRatio) ddl.RunWorker = cfg.RunDDL if cfg.SplitTable { atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1) } - plannercore.AllowCartesianProduct = cfg.Performance.CrossJoin + plannercore.AllowCartesianProduct.Store(cfg.Performance.CrossJoin) privileges.SkipWithGrant = cfg.Security.SkipGrantTable priority := mysql.Str2Priority(cfg.Performance.ForcePriority) @@ -495,11 +480,11 @@ func setGlobalVars() { if plannercore.PreparedPlanCacheMemoryGuardRatio < 0.0 || plannercore.PreparedPlanCacheMemoryGuardRatio > 1.0 { plannercore.PreparedPlanCacheMemoryGuardRatio = 0.1 } - plannercore.PreparedPlanCacheMaxMemory = cfg.Performance.MaxMemory + plannercore.PreparedPlanCacheMaxMemory.Store(cfg.Performance.MaxMemory) total, err := memory.MemTotal() terror.MustNil(err) - if plannercore.PreparedPlanCacheMaxMemory > total || plannercore.PreparedPlanCacheMaxMemory <= 0 { - plannercore.PreparedPlanCacheMaxMemory = total + if plannercore.PreparedPlanCacheMaxMemory.Load() > total || plannercore.PreparedPlanCacheMaxMemory.Load() <= 0 { + plannercore.PreparedPlanCacheMaxMemory.Store(total) } } From cd10bca6660937beb5d6de11d49ec50e149fe083 Mon Sep 17 00:00:00 2001 From: liyuzhou <2541781827@qq.com> Date: Mon, 29 Apr 2019 16:47:11 +0800 Subject: [PATCH 21/49] *: support `select`/ `explain select` using bind info (#10284) --- bindinfo/bind.go | 169 +++++++++++++++++++++++++++++++++++++ bindinfo/bind_test.go | 144 +++++++++++++++++++++++++++++++ bindinfo/cache.go | 8 +- bindinfo/handle.go | 66 ++++++++++++++- bindinfo/session_handle.go | 3 +- domain/domain.go | 26 +++++- executor/bind.go | 1 + executor/compiler.go | 60 +++++++++++++ session/session.go | 77 +++++++++++++++-- 9 files changed, 538 insertions(+), 16 deletions(-) create mode 100644 bindinfo/bind.go diff --git a/bindinfo/bind.go b/bindinfo/bind.go new file mode 100644 index 0000000000000..9d70aadf198a4 --- /dev/null +++ b/bindinfo/bind.go @@ -0,0 +1,169 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package bindinfo + +import "github.com/pingcap/parser/ast" + +// BindHint will add hints for originStmt according to hintedStmt' hints. +func BindHint(originStmt, hintedStmt ast.StmtNode) ast.StmtNode { + switch x := originStmt.(type) { + case *ast.SelectStmt: + return selectBind(x, hintedStmt.(*ast.SelectStmt)) + default: + return originStmt + } +} + +func selectBind(originalNode, hintedNode *ast.SelectStmt) *ast.SelectStmt { + if hintedNode.TableHints != nil { + originalNode.TableHints = hintedNode.TableHints + } + if originalNode.From != nil { + originalNode.From.TableRefs = resultSetNodeBind(originalNode.From.TableRefs, hintedNode.From.TableRefs).(*ast.Join) + } + if originalNode.Where != nil { + originalNode.Where = exprBind(originalNode.Where, hintedNode.Where).(ast.ExprNode) + } + + if originalNode.Having != nil { + originalNode.Having.Expr = exprBind(originalNode.Having.Expr, hintedNode.Having.Expr) + } + + if originalNode.OrderBy != nil { + originalNode.OrderBy = orderByBind(originalNode.OrderBy, hintedNode.OrderBy) + } + + if originalNode.Fields != nil { + origFields := originalNode.Fields.Fields + hintFields := hintedNode.Fields.Fields + for idx := range origFields { + origFields[idx].Expr = exprBind(origFields[idx].Expr, hintFields[idx].Expr) + } + } + return originalNode +} + +func orderByBind(originalNode, hintedNode *ast.OrderByClause) *ast.OrderByClause { + for idx := 0; idx < len(originalNode.Items); idx++ { + originalNode.Items[idx].Expr = exprBind(originalNode.Items[idx].Expr, hintedNode.Items[idx].Expr) + } + return originalNode +} + +func exprBind(originalNode, hintedNode ast.ExprNode) ast.ExprNode { + switch v := originalNode.(type) { + case *ast.SubqueryExpr: + if v.Query != nil { + v.Query = resultSetNodeBind(v.Query, hintedNode.(*ast.SubqueryExpr).Query) + } + case *ast.ExistsSubqueryExpr: + if v.Sel != nil { + v.Sel.(*ast.SubqueryExpr).Query = resultSetNodeBind(v.Sel.(*ast.SubqueryExpr).Query, hintedNode.(*ast.ExistsSubqueryExpr).Sel.(*ast.SubqueryExpr).Query) + } + case *ast.PatternInExpr: + if v.Sel != nil { + v.Sel.(*ast.SubqueryExpr).Query = resultSetNodeBind(v.Sel.(*ast.SubqueryExpr).Query, hintedNode.(*ast.PatternInExpr).Sel.(*ast.SubqueryExpr).Query) + } + case *ast.BinaryOperationExpr: + if v.L != nil { + v.L = exprBind(v.L, hintedNode.(*ast.BinaryOperationExpr).L) + } + if v.R != nil { + v.R = exprBind(v.R, hintedNode.(*ast.BinaryOperationExpr).R) + } + case *ast.IsNullExpr: + if v.Expr != nil { + v.Expr = exprBind(v.Expr, hintedNode.(*ast.IsNullExpr).Expr) + } + case *ast.IsTruthExpr: + if v.Expr != nil { + v.Expr = exprBind(v.Expr, hintedNode.(*ast.IsTruthExpr).Expr) + } + case *ast.PatternLikeExpr: + if v.Pattern != nil { + v.Pattern = exprBind(v.Pattern, hintedNode.(*ast.PatternLikeExpr).Pattern) + } + case *ast.CompareSubqueryExpr: + if v.L != nil { + v.L = exprBind(v.L, hintedNode.(*ast.CompareSubqueryExpr).L) + } + if v.R != nil { + v.R = exprBind(v.R, hintedNode.(*ast.CompareSubqueryExpr).R) + } + case *ast.BetweenExpr: + if v.Left != nil { + v.Left = exprBind(v.Left, hintedNode.(*ast.BetweenExpr).Left) + } + if v.Right != nil { + v.Right = exprBind(v.Right, hintedNode.(*ast.BetweenExpr).Right) + } + case *ast.UnaryOperationExpr: + if v.V != nil { + v.V = exprBind(v.V, hintedNode.(*ast.UnaryOperationExpr).V) + } + case *ast.CaseExpr: + if v.Value != nil { + v.Value = exprBind(v.Value, hintedNode.(*ast.CaseExpr).Value) + } + if v.ElseClause != nil { + v.ElseClause = exprBind(v.ElseClause, hintedNode.(*ast.CaseExpr).ElseClause) + } + } + return originalNode +} + +func resultSetNodeBind(originalNode, hintedNode ast.ResultSetNode) ast.ResultSetNode { + switch x := originalNode.(type) { + case *ast.Join: + return joinBind(x, hintedNode.(*ast.Join)) + case *ast.TableSource: + ts, _ := hintedNode.(*ast.TableSource) + switch v := x.Source.(type) { + case *ast.SelectStmt: + x.Source = selectBind(v, ts.Source.(*ast.SelectStmt)) + case *ast.UnionStmt: + x.Source = unionSelectBind(v, hintedNode.(*ast.TableSource).Source.(*ast.UnionStmt)) + case *ast.TableName: + x.Source.(*ast.TableName).IndexHints = ts.Source.(*ast.TableName).IndexHints + } + return x + case *ast.SelectStmt: + return selectBind(x, hintedNode.(*ast.SelectStmt)) + case *ast.UnionStmt: + return unionSelectBind(x, hintedNode.(*ast.UnionStmt)) + default: + return x + } +} + +func joinBind(originalNode, hintedNode *ast.Join) *ast.Join { + if originalNode.Left != nil { + originalNode.Left = resultSetNodeBind(originalNode.Left, hintedNode.Left) + } + + if hintedNode.Right != nil { + originalNode.Right = resultSetNodeBind(originalNode.Right, hintedNode.Right) + } + + return originalNode +} + +func unionSelectBind(originalNode, hintedNode *ast.UnionStmt) ast.ResultSetNode { + selects := originalNode.SelectList.Selects + for i := len(selects) - 1; i >= 0; i-- { + originalNode.SelectList.Selects[i] = selectBind(selects[i], hintedNode.SelectList.Selects[i]) + } + + return originalNode +} diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 9b764b9c6f12f..e37ae0f45eff3 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -280,3 +280,147 @@ func (s *testSuite) TestSessionBinding(c *C) { c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") c.Check(bindData.Status, Equals, "deleted") } + +func (s *testSuite) TestGlobalAndSessionBindingBothExist(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1(id int)") + tk.MustExec("create table t2(id int)") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "HashLeftJoin_8 12487.50 root inner join, inner:TableReader_15, equal:[eq(test.t1.id, test.t2.id)]", + "├─TableReader_12 9990.00 root data:Selection_11", + "│ └─Selection_11 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_15 9990.00 root data:Selection_14", + " └─Selection_14 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_13 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustQuery("explain SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id", + "├─Sort_11 9990.00 root test.t1.id:asc", + "│ └─TableReader_10 9990.00 root data:Selection_9", + "│ └─Selection_9 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_8 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─Sort_15 9990.00 root test.t2.id:asc", + " └─TableReader_14 9990.00 root data:Selection_13", + " └─Selection_13 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustExec("create global binding for SELECT * from t1,t2 where t1.id = t2.id using SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id", + "├─Sort_11 9990.00 root test.t1.id:asc", + "│ └─TableReader_10 9990.00 root data:Selection_9", + "│ └─Selection_9 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_8 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─Sort_15 9990.00 root test.t2.id:asc", + " └─TableReader_14 9990.00 root data:Selection_13", + " └─Selection_13 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustExec("drop global binding for SELECT * from t1,t2 where t1.id = t2.id") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "HashLeftJoin_8 12487.50 root inner join, inner:TableReader_15, equal:[eq(test.t1.id, test.t2.id)]", + "├─TableReader_12 9990.00 root data:Selection_11", + "│ └─Selection_11 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_15 9990.00 root data:Selection_14", + " └─Selection_14 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_13 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) +} + +func (s *testSuite) TestExplain(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("drop table if exists t2") + tk.MustExec("create table t1(id int)") + tk.MustExec("create table t2(id int)") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "HashLeftJoin_8 12487.50 root inner join, inner:TableReader_15, equal:[eq(test.t1.id, test.t2.id)]", + "├─TableReader_12 9990.00 root data:Selection_11", + "│ └─Selection_11 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─TableReader_15 9990.00 root data:Selection_14", + " └─Selection_14 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_13 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustQuery("explain SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id", + "├─Sort_11 9990.00 root test.t1.id:asc", + "│ └─TableReader_10 9990.00 root data:Selection_9", + "│ └─Selection_9 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_8 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─Sort_15 9990.00 root test.t2.id:asc", + " └─TableReader_14 9990.00 root data:Selection_13", + " └─Selection_13 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustExec("create global binding for SELECT * from t1,t2 where t1.id = t2.id using SELECT /*+ TIDB_SMJ(t1, t2) */ * from t1,t2 where t1.id = t2.id") + + tk.MustQuery("explain SELECT * from t1,t2 where t1.id = t2.id").Check(testkit.Rows( + "MergeJoin_7 12487.50 root inner join, left key:test.t1.id, right key:test.t2.id", + "├─Sort_11 9990.00 root test.t1.id:asc", + "│ └─TableReader_10 9990.00 root data:Selection_9", + "│ └─Selection_9 9990.00 cop not(isnull(test.t1.id))", + "│ └─TableScan_8 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", + "└─Sort_15 9990.00 root test.t2.id:asc", + " └─TableReader_14 9990.00 root data:Selection_13", + " └─Selection_13 9990.00 cop not(isnull(test.t2.id))", + " └─TableScan_12 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + + tk.MustExec("drop global binding for SELECT * from t1,t2 where t1.id = t2.id") +} + +func (s *testSuite) TestErrorBind(c *C) { + tk := testkit.NewTestKit(c, s.store) + s.cleanBindingEnv(tk) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t(i int, s varchar(20))") + tk.MustExec("create table t1(i int, s varchar(20))") + tk.MustExec("create index index_t on t(i,s)") + + _, err := tk.Exec("create global binding for select * from t where i>100 using select * from t use index(index_t) where i>100") + c.Assert(err, IsNil, Commentf("err %v", err)) + + bindData := s.domain.BindHandle().GetBindRecord("select * from t where i > ?", "test") + c.Check(bindData, NotNil) + c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") + c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>100") + c.Check(bindData.Db, Equals, "test") + c.Check(bindData.Status, Equals, "using") + c.Check(bindData.Charset, NotNil) + c.Check(bindData.Collation, NotNil) + c.Check(bindData.CreateTime, NotNil) + c.Check(bindData.UpdateTime, NotNil) + + tk.MustExec("drop index index_t on t") + _, err = tk.Exec("select * from t where i > 10") + c.Check(err, IsNil) + + s.domain.BindHandle().DropInvalidBindRecord() + + rs, err := tk.Exec("show global bindings") + c.Assert(err, IsNil) + chk := rs.NewRecordBatch() + err = rs.Next(context.TODO(), chk) + c.Check(err, IsNil) + c.Check(chk.NumRows(), Equals, 0) +} diff --git a/bindinfo/cache.go b/bindinfo/cache.go index b8ef2583c0f69..a4c2785eb9c64 100644 --- a/bindinfo/cache.go +++ b/bindinfo/cache.go @@ -20,16 +20,18 @@ import ( ) const ( - // using is the bind info's in use status. - using = "using" + // Using is the bind info's in use status. + Using = "using" // deleted is the bind info's deleted status. deleted = "deleted" + // Invalid is the bind info's invalid status. + Invalid = "invalid" ) // BindMeta stores the basic bind info and bindSql astNode. type BindMeta struct { *BindRecord - ast ast.StmtNode //ast will be used to do query sql bind check + Ast ast.StmtNode //ast will be used to do query sql bind check } // cache is a k-v map, key is original sql, value is a slice of BindMeta. diff --git a/bindinfo/handle.go b/bindinfo/handle.go index f0ba865a779eb..fc33f2cd3442c 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -20,6 +20,7 @@ import ( "go.uber.org/zap" "sync" "sync/atomic" + "time" "github.com/pingcap/parser" "github.com/pingcap/parser/mysql" @@ -60,15 +61,28 @@ type BindHandle struct { atomic.Value } + // invalidBindRecordMap indicates the invalid bind records found during querying. + // A record will be deleted from this map, after 2 bind-lease, after it is dropped from the kv. + invalidBindRecordMap struct { + sync.Mutex + atomic.Value + } + parser *parser.Parser lastUpdateTime types.Time } +type invalidBindRecordMap struct { + bindRecord *BindRecord + droppedTime time.Time +} + // NewBindHandle creates a new BindHandle. func NewBindHandle(ctx sessionctx.Context, parser *parser.Parser) *BindHandle { handle := &BindHandle{parser: parser} handle.sctx.Context = ctx handle.bindInfo.Value.Store(make(cache, 32)) + handle.invalidBindRecordMap.Value.Store(make(map[string]*invalidBindRecordMap)) return handle } @@ -106,7 +120,7 @@ func (h *BindHandle) Update(fullLoad bool) (err error) { } newCache.removeStaleBindMetas(hash, meta) - if meta.Status == using { + if meta.Status == Using { newCache[hash] = append(newCache[hash], meta) } } @@ -163,7 +177,7 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { Fsp: 3, } record.UpdateTime = record.CreateTime - record.Status = using + record.Status = Using record.BindSQL = h.getEscapeCharacter(record.BindSQL) // insert the BindRecord to the storage. @@ -217,6 +231,44 @@ func (h *BindHandle) DropBindRecord(record *BindRecord) (err error) { return err } +// DropInvalidBindRecord execute the drop bindRecord task. +func (h *BindHandle) DropInvalidBindRecord() { + invalidBindRecordMap := copyInvalidBindRecordMap(h.invalidBindRecordMap.Load().(map[string]*invalidBindRecordMap)) + for key, invalidBindRecord := range invalidBindRecordMap { + if invalidBindRecord.droppedTime.IsZero() { + err := h.DropBindRecord(invalidBindRecord.bindRecord) + if err != nil { + logutil.Logger(context.Background()).Error("DropInvalidBindRecord failed", zap.Error(err)) + } + invalidBindRecord.droppedTime = time.Now() + continue + } + + if time.Since(invalidBindRecord.droppedTime) > 6*time.Second { + delete(invalidBindRecordMap, key) + } + } + h.invalidBindRecordMap.Store(invalidBindRecordMap) +} + +// AddDropInvalidBindTask add bindRecord to invalidBindRecordMap when the bindRecord need to be deleted. +func (h *BindHandle) AddDropInvalidBindTask(invalidBindRecord *BindRecord) { + key := invalidBindRecord.OriginalSQL + ":" + invalidBindRecord.Db + if _, ok := h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)[key]; ok { + return + } + h.invalidBindRecordMap.Lock() + defer h.invalidBindRecordMap.Unlock() + if _, ok := h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)[key]; ok { + return + } + newMap := copyInvalidBindRecordMap(h.invalidBindRecordMap.Value.Load().(map[string]*invalidBindRecordMap)) + newMap[key] = &invalidBindRecordMap{ + bindRecord: invalidBindRecord, + } + h.invalidBindRecordMap.Store(newMap) +} + // Size return the size of bind info cache. func (h *BindHandle) Size() int { size := 0 @@ -246,7 +298,7 @@ func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMet if err != nil { return "", nil, err } - meta = &BindMeta{BindRecord: record, ast: stmtNodes[0]} + meta = &BindMeta{BindRecord: record, Ast: stmtNodes[0]} return hash, meta, nil } @@ -328,6 +380,14 @@ func (c cache) copy() cache { return newCache } +func copyInvalidBindRecordMap(oldMap map[string]*invalidBindRecordMap) map[string]*invalidBindRecordMap { + newMap := make(map[string]*invalidBindRecordMap, len(oldMap)) + for k, v := range oldMap { + newMap[k] = v + } + return newMap +} + func (c cache) getBindRecord(normdOrigSQL, db string) *BindMeta { hash := parser.DigestHash(normdOrigSQL) bindRecords := c[hash] diff --git a/bindinfo/session_handle.go b/bindinfo/session_handle.go index 90b4d8ac3c457..f343b3ca8e24d 100644 --- a/bindinfo/session_handle.go +++ b/bindinfo/session_handle.go @@ -48,13 +48,12 @@ func (h *SessionHandle) newBindMeta(record *BindRecord) (hash string, meta *Bind if err != nil { return "", nil, err } - meta = &BindMeta{BindRecord: record, ast: stmtNodes[0]} + meta = &BindMeta{BindRecord: record, Ast: stmtNodes[0]} return hash, meta, nil } // AddBindRecord new a BindRecord with BindMeta, add it to the cache. func (h *SessionHandle) AddBindRecord(record *BindRecord) error { - record.Status = using record.CreateTime = types.Time{ Time: types.FromGoTime(time.Now()), Type: mysql.TypeDatetime, diff --git a/domain/domain.go b/domain/domain.go index 45b8a1d5cec5a..008348cee7eff 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -791,6 +791,12 @@ func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser return err } + do.loadBindInfoLoop() + do.handleInvalidBindTaskLoop() + return nil +} + +func (do *Domain) loadBindInfoLoop() { duration := 3 * time.Second do.wg.Add(1) go func() { @@ -802,13 +808,29 @@ func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser return case <-time.After(duration): } - err = do.bindHandle.Update(false) + err := do.bindHandle.Update(false) if err != nil { logutil.Logger(context.Background()).Error("update bindinfo failed", zap.Error(err)) } } }() - return nil +} + +func (do *Domain) handleInvalidBindTaskLoop() { + handleInvalidTaskDuration := 3 * time.Second + do.wg.Add(1) + go func() { + defer do.wg.Done() + defer recoverInDomain("loadBindInfoLoop-dropInvalidBindInfo", false) + for { + select { + case <-do.exit: + return + case <-time.After(handleInvalidTaskDuration): + } + do.bindHandle.DropInvalidBindRecord() + } + }() } // StatsHandle returns the statistic handle. diff --git a/executor/bind.go b/executor/bind.go index 2e3ad0f8aa326..d2c2034851bde 100644 --- a/executor/bind.go +++ b/executor/bind.go @@ -76,6 +76,7 @@ func (e *SQLBindExec) createSQLBind() error { Db: e.ctx.GetSessionVars().CurrentDB, Charset: e.charset, Collation: e.collation, + Status: bindinfo.Using, } if !e.isGlobal { handle := e.ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) diff --git a/executor/compiler.go b/executor/compiler.go index 23b2239475ca9..f233bdfe833cd 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -16,10 +16,14 @@ package executor import ( "context" "fmt" + "strings" "github.com/opentracing/opentracing-go" + "github.com/pingcap/parser" "github.com/pingcap/parser/ast" + "github.com/pingcap/tidb/bindinfo" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/planner" @@ -49,11 +53,24 @@ type Compiler struct { // Compile compiles an ast.StmtNode to a physical plan. func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStmt, error) { + return c.compile(ctx, stmtNode, false) +} + +// SkipBindCompile compiles an ast.StmtNode to a physical plan without SQL bind. +func (c *Compiler) SkipBindCompile(ctx context.Context, node ast.StmtNode) (*ExecStmt, error) { + return c.compile(ctx, node, true) +} + +func (c *Compiler) compile(ctx context.Context, stmtNode ast.StmtNode, skipBind bool) (*ExecStmt, error) { if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("executor.Compile", opentracing.ChildOf(span.Context())) defer span1.Finish() } + if !skipBind { + stmtNode = addHint(c.Ctx, stmtNode) + } + infoSchema := GetInfoSchema(c.Ctx) if err := plannercore.Preprocess(c.Ctx, stmtNode, infoSchema); err != nil { return nil, err @@ -367,3 +384,46 @@ func GetInfoSchema(ctx sessionctx.Context) infoschema.InfoSchema { } return is } + +func addHint(ctx sessionctx.Context, stmtNode ast.StmtNode) ast.StmtNode { + if ctx.Value(bindinfo.SessionBindInfoKeyType) == nil { //when the domain is initializing, the bind will be nil. + return stmtNode + } + switch x := stmtNode.(type) { + case *ast.ExplainStmt: + switch x.Stmt.(type) { + case *ast.SelectStmt: + normalizeExplainSQL := parser.Normalize(x.Text()) + idx := strings.Index(normalizeExplainSQL, "select") + normalizeSQL := normalizeExplainSQL[idx:] + x.Stmt = addHintForSelect(normalizeSQL, ctx, x.Stmt) + } + return x + case *ast.SelectStmt: + return addHintForSelect(parser.Normalize(x.Text()), ctx, x) + default: + return stmtNode + } +} + +func addHintForSelect(normdOrigSQL string, ctx sessionctx.Context, stmt ast.StmtNode) ast.StmtNode { + sessionHandle := ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + bindRecord := sessionHandle.GetBindRecord(normdOrigSQL, ctx.GetSessionVars().CurrentDB) + if bindRecord != nil { + if bindRecord.Status == bindinfo.Invalid { + return stmt + } + if bindRecord.Status == bindinfo.Using { + return bindinfo.BindHint(stmt, bindRecord.Ast) + } + } + globalHandle := domain.GetDomain(ctx).BindHandle() + bindRecord = globalHandle.GetBindRecord(normdOrigSQL, ctx.GetSessionVars().CurrentDB) + if bindRecord == nil { + bindRecord = globalHandle.GetBindRecord(normdOrigSQL, "") + } + if bindRecord != nil { + return bindinfo.BindHint(stmt, bindRecord.Ast) + } + return stmt +} diff --git a/session/session.go b/session/session.go index 5d297a7476367..1f504b9dd4ddd 100644 --- a/session/session.go +++ b/session/session.go @@ -991,8 +991,9 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec sessionExecuteParseDurationGeneral.Observe(time.Since(startTS).Seconds()) } + var tempStmtNodes []ast.StmtNode compiler := executor.Compiler{Ctx: s} - for _, stmtNode := range stmtNodes { + for idx, stmtNode := range stmtNodes { s.PrepareTxnCtx(ctx) // Step2: Transform abstract syntax tree to a physical plan(stored in executor.ExecStmt). @@ -1003,11 +1004,22 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec } stmt, err := compiler.Compile(ctx, stmtNode) if err != nil { - s.rollbackOnError(ctx) - logutil.Logger(ctx).Warn("compile sql error", - zap.Error(err), - zap.String("sql", sql)) - return nil, err + if tempStmtNodes == nil { + tempStmtNodes, warns, err = s.ParseSQL(ctx, sql, charsetInfo, collation) + if err != nil || warns != nil { + //just skip errcheck, because parse will not return an error. + } + } + stmtNode = tempStmtNodes[idx] + stmt, err = compiler.SkipBindCompile(ctx, stmtNode) + if err != nil { + s.rollbackOnError(ctx) + logutil.Logger(ctx).Warn("compile sql error", + zap.Error(err), + zap.String("sql", sql)) + return nil, err + } + s.handleInvalidBindRecord(ctx, stmtNode) } if isInternal { sessionExecuteCompileDurationInternal.Observe(time.Since(startTS).Seconds()) @@ -1033,6 +1045,59 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec return recordSets, nil } +func (s *session) handleInvalidBindRecord(ctx context.Context, stmtNode ast.StmtNode) { + var normdOrigSQL string + switch x := stmtNode.(type) { + case *ast.ExplainStmt: + switch x.Stmt.(type) { + case *ast.SelectStmt: + normalizeExplainSQL := parser.Normalize(x.Text()) + idx := strings.Index(normalizeExplainSQL, "select") + normdOrigSQL = normalizeExplainSQL[idx:] + default: + return + } + case *ast.SelectStmt: + normdOrigSQL = parser.Normalize(x.Text()) + default: + return + } + sessionHandle := s.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) + bindMeta := sessionHandle.GetBindRecord(normdOrigSQL, s.GetSessionVars().CurrentDB) + if bindMeta != nil { + bindMeta.Status = bindinfo.Invalid + return + } + + globalHandle := domain.GetDomain(s).BindHandle() + bindMeta = globalHandle.GetBindRecord(normdOrigSQL, s.GetSessionVars().CurrentDB) + if bindMeta == nil { + bindMeta = globalHandle.GetBindRecord(normdOrigSQL, "") + } + if bindMeta != nil { + record := &bindinfo.BindRecord{ + OriginalSQL: bindMeta.OriginalSQL, + BindSQL: bindMeta.BindSQL, + Db: s.GetSessionVars().CurrentDB, + Charset: bindMeta.Charset, + Collation: bindMeta.Collation, + Status: bindinfo.Invalid, + } + + err := sessionHandle.AddBindRecord(record) + if err != nil { + logutil.Logger(ctx).Warn("handleInvalidBindRecord failed", zap.Error(err)) + } + + globalHandle := domain.GetDomain(s).BindHandle() + dropBindRecord := &bindinfo.BindRecord{ + OriginalSQL: bindMeta.OriginalSQL, + Db: bindMeta.Db, + } + globalHandle.AddDropInvalidBindTask(dropBindRecord) + } +} + // rollbackOnError makes sure the next statement starts a new transaction with the latest InfoSchema. func (s *session) rollbackOnError(ctx context.Context) { if !s.sessionVars.InTxn() { From 7cc688c036782ae16238123ad2b2fa17e07bb4d2 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Mon, 29 Apr 2019 19:13:39 +0800 Subject: [PATCH 22/49] infoschema, domain, ddl: fix upper cased charset names (#10272) --- ddl/db_integration_test.go | 45 ++++++++++++++++++++++++++++++++++++++ ddl/db_test.go | 2 +- domain/domain.go | 1 + go.mod | 2 +- go.sum | 5 ++--- infoschema/builder.go | 16 ++++++++++++++ 6 files changed, 66 insertions(+), 5 deletions(-) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 81e35dc563745..5ee6b31999c74 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -751,6 +751,51 @@ func (s *testIntegrationSuite7) TestCaseInsensitiveCharsetAndCollate(c *C) { tk.MustExec("create table t2(id int) ENGINE=InnoDB DEFAULT CHARSET=Utf8 COLLATE=utf8_BIN;") tk.MustExec("create table t3(id int) ENGINE=InnoDB DEFAULT CHARSET=Utf8mb4 COLLATE=utf8MB4_BIN;") tk.MustExec("create table t4(id int) ENGINE=InnoDB DEFAULT CHARSET=Utf8mb4 COLLATE=utf8MB4_general_ci;") + + tk.MustExec("create table t5(a varchar(20)) ENGINE=InnoDB DEFAULT CHARSET=UTF8MB4 COLLATE=UTF8MB4_GENERAL_CI;") + tk.MustExec("insert into t5 values ('特克斯和凯科斯群岛')") + + db, ok := domain.GetDomain(s.ctx).InfoSchema().SchemaByName(model.NewCIStr("test_charset_collate")) + c.Assert(ok, IsTrue) + tbl := testGetTableByName(c, s.ctx, "test_charset_collate", "t5") + tblInfo := tbl.Meta().Clone() + c.Assert(tblInfo.Charset, Equals, "utf8mb4") + c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") + + tblInfo.Version = model.TableInfoVersion2 + tblInfo.Charset = "UTF8MB4" + + updateTableInfo := func(tblInfo *model.TableInfo) { + mockCtx := mock.NewContext() + mockCtx.Store = s.store + err := mockCtx.NewTxn(context.Background()) + c.Assert(err, IsNil) + txn, err := mockCtx.Txn(true) + c.Assert(err, IsNil) + mt := meta.NewMeta(txn) + c.Assert(ok, IsTrue) + err = mt.UpdateTable(db.ID, tblInfo) + c.Assert(err, IsNil) + err = txn.Commit(context.Background()) + c.Assert(err, IsNil) + } + updateTableInfo(tblInfo) + tk.MustExec("alter table t5 add column b varchar(10);") // load latest schema. + + tblInfo = testGetTableByName(c, s.ctx, "test_charset_collate", "t5").Meta() + c.Assert(tblInfo.Charset, Equals, "utf8mb4") + c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") + + // For model.TableInfoVersion3, it is believed that all charsets / collations are lower-cased, do not do case-convert + tblInfo = tblInfo.Clone() + tblInfo.Version = model.TableInfoVersion3 + tblInfo.Charset = "UTF8MB4" + updateTableInfo(tblInfo) + tk.MustExec("alter table t5 add column c varchar(10);") // load latest schema. + + tblInfo = testGetTableByName(c, s.ctx, "test_charset_collate", "t5").Meta() + c.Assert(tblInfo.Charset, Equals, "UTF8MB4") + c.Assert(tblInfo.Columns[0].Charset, Equals, "utf8mb4") } func (s *testIntegrationSuite3) TestZeroFillCreateTable(c *C) { diff --git a/ddl/db_test.go b/ddl/db_test.go index c5867b6648576..5c10247ba7f76 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -2292,7 +2292,7 @@ func (s *testDBSuite4) TestCheckTooBigFieldLength(c *C) { assertErrorCode(c, s.tk, "alter table tr_04 add column b varchar(20000) charset utf8mb4;", tmysql.ErrTooBigFieldlength) assertErrorCode(c, s.tk, "alter table tr_04 convert to character set utf8mb4;", tmysql.ErrTooBigFieldlength) assertErrorCode(c, s.tk, "create table tr (id int, name varchar(30000), purchased date ) default charset=utf8 collate=utf8_bin;", tmysql.ErrTooBigFieldlength) - assertErrorCode(c, s.tk, "create table tr (id int, name varchar(20000) charset utf8mb4, purchased date ) default charset=utf8 collate=utf8;", tmysql.ErrTooBigFieldlength) + assertErrorCode(c, s.tk, "create table tr (id int, name varchar(20000) charset utf8mb4, purchased date ) default charset=utf8 collate=utf8_bin;", tmysql.ErrTooBigFieldlength) assertErrorCode(c, s.tk, "create table tr (id int, name varchar(65536), purchased date ) default charset=latin1;", tmysql.ErrTooBigFieldlength) s.tk.MustExec("drop table if exists tr_05;") diff --git a/domain/domain.go b/domain/domain.go index 008348cee7eff..814452ea9870a 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -194,6 +194,7 @@ func (do *Domain) fetchSchemasWithTables(schemas []*model.DBInfo, m *meta.Meta, // schema is not public, can't be used outside. continue } + infoschema.ConvertCharsetCollateToLowerCaseIfNeed(tbl) di.Tables = append(di.Tables, tbl) } } diff --git a/go.mod b/go.mod index 7be334aa05bc4..19a9d81e6ca3c 100644 --- a/go.mod +++ b/go.mod @@ -39,7 +39,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190427000002-f3ecae036b23 + github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 diff --git a/go.sum b/go.sum index d0b3f9a8a1c0b..7cddadde9083e 100644 --- a/go.sum +++ b/go.sum @@ -160,8 +160,8 @@ github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190427000002-f3ecae036b23 h1:9RUvFHNfDplW4KZ90FMOC7QEDLORa15AlozhX0P8cCg= -github.com/pingcap/parser v0.0.0-20190427000002-f3ecae036b23/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6 h1:m35H8yhrMcATKp8yUVUHAmvuzIT8xjL+ialXyBrBWHU= +github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= @@ -292,7 +292,6 @@ google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3 gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4K+aSYdM/U= gopkg.in/alecthomas/gometalinter.v2 v2.0.12/go.mod h1:NDRytsqEZyolNuAgTzJkZMkSQM7FIKyzVzGhjB/qfYo= gopkg.in/alecthomas/kingpin.v3-unstable v3.0.0-20180810215634-df19058c872c/go.mod h1:3HH7i1SgMqlzxCcBmUHW657sD4Kvv9sC3HpL3YukzwA= -gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/infoschema/builder.go b/infoschema/builder.go index 65eba8279c1b0..5b0446648f46f 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -16,6 +16,7 @@ package infoschema import ( "fmt" "sort" + "strings" "github.com/pingcap/errors" "github.com/pingcap/parser/charset" @@ -172,6 +173,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i fmt.Sprintf("(Table ID %d)", tableID), ) } + ConvertCharsetCollateToLowerCaseIfNeed(tblInfo) ConvertOldVersionUTF8ToUTF8MB4IfNeed(tblInfo) if alloc == nil { @@ -197,6 +199,20 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i return nil } +// ConvertCharsetCollateToLowerCaseIfNeed convert the charset / collation of table and its columns to lower case, +// if the table's version is prior to TableInfoVersion3. +func ConvertCharsetCollateToLowerCaseIfNeed(tbInfo *model.TableInfo) { + if tbInfo.Version >= model.TableInfoVersion3 { + return + } + tbInfo.Charset = strings.ToLower(tbInfo.Charset) + tbInfo.Collate = strings.ToLower(tbInfo.Collate) + for _, col := range tbInfo.Columns { + col.Charset = strings.ToLower(col.Charset) + col.Collate = strings.ToLower(col.Collate) + } +} + // ConvertOldVersionUTF8ToUTF8MB4IfNeed convert old version UTF8 to UTF8MB4 if config.TreatOldVersionUTF8AsUTF8MB4 is enable. func ConvertOldVersionUTF8ToUTF8MB4IfNeed(tbInfo *model.TableInfo) { if tbInfo.Version >= model.TableInfoVersion2 || !config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 { From 36bb1ae89a908375ff941fbe1c07259e5978b957 Mon Sep 17 00:00:00 2001 From: crazycs Date: Tue, 30 Apr 2019 09:35:52 +0800 Subject: [PATCH 23/49] log: change slow log time format to compatible with pt-query-digest (#10290) --- infoschema/slow_log.go | 6 +++++- infoschema/slow_log_test.go | 32 +++++++++++++++++++++++++------- infoschema/tables_test.go | 2 +- sessionctx/variable/session.go | 2 +- util/logutil/log.go | 5 ++++- 5 files changed, 36 insertions(+), 11 deletions(-) diff --git a/infoschema/slow_log.go b/infoschema/slow_log.go index 93d86525e9207..6afa6dab3ea47 100644 --- a/infoschema/slow_log.go +++ b/infoschema/slow_log.go @@ -330,7 +330,11 @@ func (st *slowQueryTuple) convertToDatumRow() []types.Datum { func ParseTime(s string) (time.Time, error) { t, err := time.Parse(logutil.SlowLogTimeFormat, s) if err != nil { - err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\", err: %v", s, logutil.SlowLogTimeFormat, err) + // This is for compatibility. + t, err = time.Parse(logutil.OldSlowLogTimeFormat, s) + if err != nil { + err = errors.Errorf("string \"%v\" doesn't has a prefix that matches format \"%v\", err: %v", s, logutil.SlowLogTimeFormat, err) + } } return t, err } diff --git a/infoschema/slow_log_test.go b/infoschema/slow_log_test.go index 8a839ae89cf75..3c75daecd5c2c 100644 --- a/infoschema/slow_log_test.go +++ b/infoschema/slow_log_test.go @@ -25,7 +25,7 @@ import ( func (s *testSuite) TestParseSlowLogFile(c *C) { slowLog := bytes.NewBufferString( - `# Time: 2019-01-24-22:32:29.313255 +0800 + `# Time: 2019-04-28T15:24:04.309074+08:00 # Txn_start_ts: 405888132465033227 # Query_time: 0.216905 # Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 @@ -51,14 +51,14 @@ select * from t;`) } recordString += str } - expectRecordString := "2019-01-24 22:32:29.313255,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,select * from t;" + expectRecordString := "2019-04-28 15:24:04.309074,405888132465033227,,0,0.216905,0.021,0,0,1,637,0,,,1,42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772,t1:1,t2:2,0.1,0.2,0.03,127.0.0.1:20160,0.05,0.6,0.8,0.0.0.0:20160,70724,select * from t;" c.Assert(expectRecordString, Equals, recordString) // fix sql contain '# ' bug slowLog = bytes.NewBufferString( - `# Time: 2019-01-24-22:32:29.313255 +0800 + `# Time: 2019-04-28T15:24:04.309074+08:00 select a# from t; -# Time: 2019-01-24-22:32:29.313255 +0800 +# Time: 2019-01-24T22:32:29.313255+08:00 # Txn_start_ts: 405888132465033227 # Query_time: 0.216905 # Process_time: 0.021 Request_count: 1 Total_keys: 637 Processed_keys: 436 @@ -70,16 +70,34 @@ select * from t; scanner = bufio.NewScanner(slowLog) _, err = infoschema.ParseSlowLog(loc, scanner) c.Assert(err, IsNil) + + // test for time format compatibility. + slowLog = bytes.NewBufferString( + `# Time: 2019-04-28T15:24:04.309074+08:00 +select * from t; +# Time: 2019-04-24-19:41:21.716221 +0800 +select * from t; +`) + scanner = bufio.NewScanner(slowLog) + rows, err = infoschema.ParseSlowLog(loc, scanner) + c.Assert(err, IsNil) + c.Assert(len(rows) == 2, IsTrue) + t0Str, err := rows[0][0].ToString() + c.Assert(err, IsNil) + c.Assert(t0Str, Equals, "2019-04-28 15:24:04.309074") + t1Str, err := rows[1][0].ToString() + c.Assert(err, IsNil) + c.Assert(t1Str, Equals, "2019-04-24 19:41:21.716221") } func (s *testSuite) TestSlowLogParseTime(c *C) { - t1Str := "2019-01-24-22:32:29.313255 +0800" - t2Str := "2019-01-24-22:32:29.313255" + t1Str := "2019-01-24T22:32:29.313255+08:00" + t2Str := "2019-01-24T22:32:29.313255" t1, err := infoschema.ParseTime(t1Str) c.Assert(err, IsNil) loc, err := time.LoadLocation("Asia/Shanghai") c.Assert(err, IsNil) - t2, err := time.ParseInLocation("2006-01-02-15:04:05.999999999", t2Str, loc) + t2, err := time.ParseInLocation("2006-01-02T15:04:05.999999999", t2Str, loc) c.Assert(err, IsNil) c.Assert(t1.Unix(), Equals, t2.Unix()) t1Format := t1.In(loc).Format(logutil.SlowLogTimeFormat) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index b0e0f2f52ad6d..31020e99ed846 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -329,7 +329,7 @@ func (s *testTableSuite) TestSlowQuery(c *C) { f, err := os.OpenFile(slowLogFileName, os.O_CREATE|os.O_WRONLY, 0644) c.Assert(err, IsNil) defer os.Remove(slowLogFileName) - _, err = f.Write([]byte(`# Time: 2019-02-12-19:33:56.571953 +0800 + _, err = f.Write([]byte(`# Time: 2019-02-12T19:33:56.571953+08:00 # Txn_start_ts: 406315658548871171 # User: root@127.0.0.1 # Conn_ID: 6 diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index da2a76c05d413..81a58c6243489 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -945,7 +945,7 @@ const ( // SlowLogFormat uses for formatting slow log. // The slow log output is like below: -// # Time: 2019-02-12-19:33:56.571953 +0800 +// # Time: 2019-04-28T15:24:04.309074+08:00 // # Txn_start_ts: 406315658548871171 // # User: root@127.0.0.1 // # Conn_ID: 6 diff --git a/util/logutil/log.go b/util/logutil/log.go index 691b662eea63f..40061f20be11a 100644 --- a/util/logutil/log.go +++ b/util/logutil/log.go @@ -22,6 +22,7 @@ import ( "runtime" "sort" "strings" + "time" "github.com/pingcap/errors" zaplog "github.com/pingcap/log" @@ -208,7 +209,9 @@ func (f *textFormatter) Format(entry *log.Entry) ([]byte, error) { const ( // SlowLogTimeFormat is the time format for slow log. - SlowLogTimeFormat = "2006-01-02-15:04:05.999999999 -0700" + SlowLogTimeFormat = time.RFC3339Nano + // OldSlowLogTimeFormat is the first version of the the time format for slow log, This is use for compatibility. + OldSlowLogTimeFormat = "2006-01-02-15:04:05.999999999 -0700" ) type slowLogFormatter struct{} From fb6bbeed398861c2a7e9e15904e43fb542bcd74b Mon Sep 17 00:00:00 2001 From: maiyang Date: Tue, 30 Apr 2019 11:39:28 +0800 Subject: [PATCH 24/49] hooks: add pre_commit comment (#9308) --- hooks/pre-commit | 2 ++ 1 file changed, 2 insertions(+) diff --git a/hooks/pre-commit b/hooks/pre-commit index e3746039b5fd4..8bc330c9af495 100755 --- a/hooks/pre-commit +++ b/hooks/pre-commit @@ -1,6 +1,8 @@ #!/bin/bash # This file modified from k8s # https://github.com/kubernetes/kubernetes/blob/master/hooks/pre-commit +# Now It's removed, The Reason is https://github.com/kubernetes/community/issues/729 +# The PR is https://github.com/kubernetes/kubernetes/pull/47673 # How to use this hook? # ln -s hooks/pre-commit .git/hooks/ From 4cf3630eec03ba9ba1df4cc64bd75b18bc14986d Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 30 Apr 2019 12:49:00 +0800 Subject: [PATCH 25/49] ddl: pre-split region for partitioned table (#10221) --- ddl/ddl_api.go | 16 ++++++++++++++-- ddl/table.go | 12 +++++++----- ddl/table_split_test.go | 28 ++++++++++++++++++++++++---- 3 files changed, 45 insertions(+), 11 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d8e96e056d206..16650d6dbe9d6 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1202,14 +1202,26 @@ func (d *ddl) CreateTable(ctx sessionctx.Context, s *ast.CreateTableStmt) (err e err = d.doDDLJob(ctx, job) if err == nil { + var preSplitAndScatter func() // do pre-split and scatter. if tbInfo.ShardRowIDBits > 0 && tbInfo.PreSplitRegions > 0 { + preSplitAndScatter = func() { preSplitTableRegion(d.store, tbInfo, ctx.GetSessionVars().WaitTableSplitFinish) } + } else if atomic.LoadUint32(&EnableSplitTableRegion) != 0 { + pi := tbInfo.GetPartitionInfo() + if pi != nil { + preSplitAndScatter = func() { splitPartitionTableRegion(d.store, pi) } + } else { + preSplitAndScatter = func() { splitTableRegion(d.store, tbInfo.ID) } + } + } + if preSplitAndScatter != nil { if ctx.GetSessionVars().WaitTableSplitFinish { - preSplitTableRegion(d.store, tbInfo, true) + preSplitAndScatter() } else { - go preSplitTableRegion(d.store, tbInfo, false) + go preSplitAndScatter() } } + if tbInfo.AutoIncID > 1 { // Default tableAutoIncID base is 0. // If the first ID is expected to greater than 1, we need to do rebase. diff --git a/ddl/table.go b/ddl/table.go index cec8c7ca07a8e..dd604a3d8c7c3 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -17,7 +17,6 @@ import ( "fmt" "strconv" "strings" - "sync/atomic" "github.com/pingcap/errors" "github.com/pingcap/failpoint" @@ -74,10 +73,6 @@ func onCreateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) if err != nil { return ver, errors.Trace(err) } - if atomic.LoadUint32(&EnableSplitTableRegion) != 0 { - // TODO: Add restrictions to this operation. - go splitTableRegion(d.store, tbInfo.ID) - } // Finish this job. job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tbInfo) asyncNotifyEvent(d, &util.Event{Tp: model.ActionCreateTable, TableInfo: tbInfo}) @@ -340,6 +335,13 @@ type splitableStore interface { WaitScatterRegionFinish(regionID uint64) error } +func splitPartitionTableRegion(store kv.Storage, pi *model.PartitionInfo) { + // Max partition count is 4096, should we sample and just choose some of the partition to split? + for _, def := range pi.Definitions { + splitTableRegion(store, def.ID) + } +} + func splitTableRegion(store kv.Storage, tableID int64) { s, ok := store.(splitableStore) if !ok { diff --git a/ddl/table_split_test.go b/ddl/table_split_test.go index ca6aa9b97d236..132255a986dc6 100644 --- a/ddl/table_split_test.go +++ b/ddl/table_split_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/util/testkit" ) type testDDLTableSplitSuite struct{} @@ -41,20 +42,39 @@ func (s *testDDLTableSplitSuite) TestTableSplit(c *C) { atomic.StoreUint32(&ddl.EnableSplitTableRegion, 1) dom, err := session.BootstrapSession(store) c.Assert(err, IsNil) + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test") + tk.MustExec(`create table t_part (a int key) partition by range(a) ( + partition p0 values less than (10), + partition p1 values less than (20) + )`) defer dom.Close() atomic.StoreUint32(&ddl.EnableSplitTableRegion, 0) infoSchema := dom.InfoSchema() c.Assert(infoSchema, NotNil) t, err := infoSchema.TableByName(model.NewCIStr("mysql"), model.NewCIStr("tidb")) c.Assert(err, IsNil) - regionStartKey := tablecodec.EncodeTablePrefix(t.Meta().ID) + checkRegionStartWithTableID(c, t.Meta().ID, store.(kvStore)) - type kvStore interface { - GetRegionCache() *tikv.RegionCache + t, err = infoSchema.TableByName(model.NewCIStr("test"), model.NewCIStr("t_part")) + c.Assert(err, IsNil) + pi := t.Meta().GetPartitionInfo() + c.Assert(pi, NotNil) + for _, def := range pi.Definitions { + checkRegionStartWithTableID(c, def.ID, store.(kvStore)) } +} + +type kvStore interface { + GetRegionCache() *tikv.RegionCache +} + +func checkRegionStartWithTableID(c *C, id int64, store kvStore) { + regionStartKey := tablecodec.EncodeTablePrefix(id) var loc *tikv.KeyLocation + var err error for i := 0; i < 10; i++ { - cache := store.(kvStore).GetRegionCache() + cache := store.GetRegionCache() loc, err = cache.LocateKey(tikv.NewBackoffer(context.Background(), 5000), regionStartKey) c.Assert(err, IsNil) From efe9b6ab2ad8606c2d7bd94d6f1527637d53a793 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 30 Apr 2019 13:06:54 +0800 Subject: [PATCH 26/49] planner/core: fix privilege check for update (#10281) --- cmd/explaintest/r/explain_complex.result | 30 +- .../r/explain_complex_stats.result | 30 +- cmd/explaintest/r/explain_easy.result | 104 +++---- cmd/explaintest/r/explain_easy_stats.result | 10 +- cmd/explaintest/r/select.result | 22 +- cmd/explaintest/r/subquery.result | 6 +- cmd/explaintest/r/topn_push_down.result | 46 +-- cmd/explaintest/r/tpch.result | 52 ++-- executor/merge_join_test.go | 2 +- planner/core/cbo_test.go | 28 +- planner/core/logical_plan_builder.go | 1 - planner/core/logical_plan_test.go | 277 +++++++++--------- planner/core/physical_plan_test.go | 132 ++++----- session/session_test.go | 23 ++ util/ranger/ranger_test.go | 6 +- 15 files changed, 395 insertions(+), 374 deletions(-) diff --git a/cmd/explaintest/r/explain_complex.result b/cmd/explaintest/r/explain_complex.result index e23a148310563..64da664d9adb6 100644 --- a/cmd/explaintest/r/explain_complex.result +++ b/cmd/explaintest/r/explain_complex.result @@ -115,30 +115,30 @@ Projection_7 53.00 root test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt. └─TableScan_14 2650.00 cop table:dt, keep order:false, stats:pseudo explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext, gad.t as gtime from st gad join (select id, aid, pt, dic, ip, t from dd where pt = 'android' and bm = 0 and t > 1478143908) sdk on gad.aid = sdk.aid and gad.ip = sdk.ip and sdk.t > gad.t where gad.t > 1478143908 and gad.pt = 'android' group by gad.aid, sdk.dic limit 2500; id count task operator info -Projection_13 1.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t +Projection_13 1.00 root test.gad.id, test.dd.id, test.gad.aid, test.gad.cm, test.dd.dic, test.dd.ip, test.dd.t, test.gad.p1, test.gad.p2, test.gad.p3, test.gad.p4, test.gad.p5, test.gad.p6_md5, test.gad.p7_md5, test.gad.ext, test.gad.t └─Limit_16 1.00 root offset:0, count:2500 - └─HashAgg_19 1.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) - └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(test.dd.ip, gad.ip), gt(test.dd.t, gad.t) + └─HashAgg_19 1.00 root group by:test.dd.dic, test.gad.aid, funcs:firstrow(test.gad.id), firstrow(test.gad.aid), firstrow(test.gad.cm), firstrow(test.gad.p1), firstrow(test.gad.p2), firstrow(test.gad.p3), firstrow(test.gad.p4), firstrow(test.gad.p5), firstrow(test.gad.p6_md5), firstrow(test.gad.p7_md5), firstrow(test.gad.ext), firstrow(test.gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) + └─IndexJoin_24 0.00 root inner join, inner:IndexLookUp_23, outer key:test.gad.aid, inner key:test.dd.aid, other cond:eq(test.dd.ip, test.gad.ip), gt(test.dd.t, test.gad.t) ├─IndexLookUp_23 0.00 root - │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false, stats:pseudo + │ ├─IndexScan_20 10.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, test.gad.aid)], keep order:false, stats:pseudo │ └─Selection_22 0.00 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) │ └─TableScan_21 10.00 cop table:dd, keep order:false, stats:pseudo └─IndexLookUp_33 3.33 root ├─IndexScan_30 3333.33 cop table:gad, index:t, range:(1478143908,+inf], keep order:false, stats:pseudo - └─Selection_32 3.33 cop eq(gad.pt, "android"), not(isnull(gad.ip)) + └─Selection_32 3.33 cop eq(test.gad.pt, "android"), not(isnull(test.gad.ip)) └─TableScan_31 3333.33 cop table:st, keep order:false, stats:pseudo explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; id count task operator info -Projection_10 0.00 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext +Projection_10 0.00 root test.gad.id, test.sdk.id, test.gad.aid, test.gad.cm, test.sdk.dic, test.sdk.ip, test.sdk.t, test.gad.p1, test.gad.p2, test.gad.p3, test.gad.p4, test.gad.p5, test.gad.p6_md5, test.gad.p7_md5, test.gad.ext └─Limit_13 0.00 root offset:0, count:3000 - └─IndexJoin_18 0.00 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac), lt(gad.t, sdk.t) + └─IndexJoin_18 0.00 root inner join, inner:IndexLookUp_17, outer key:test.gad.aid, inner key:test.sdk.aid, other cond:eq(test.gad.dic, test.sdk.mac), lt(test.gad.t, test.sdk.t) ├─IndexLookUp_27 0.00 root │ ├─IndexScan_24 3333.33 cop table:gad, index:t, range:(1477971479,+inf], keep order:false, stats:pseudo - │ └─Selection_26 0.00 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), not(isnull(gad.dic)) + │ └─Selection_26 0.00 cop eq(test.gad.bm, 0), eq(test.gad.dit, "mac"), eq(test.gad.pt, "ios"), not(isnull(test.gad.dic)) │ └─TableScan_25 3333.33 cop table:st, keep order:false, stats:pseudo └─IndexLookUp_17 0.00 root - ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false, stats:pseudo - └─Selection_16 0.00 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) + ├─IndexScan_14 10.00 cop table:sdk, index:aid, dic, range: decided by [eq(test.sdk.aid, test.gad.aid)], keep order:false, stats:pseudo + └─Selection_16 0.00 cop eq(test.sdk.bm, 0), eq(test.sdk.pt, "ios"), gt(test.sdk.t, 1477971479), not(isnull(test.sdk.mac)), not(isnull(test.sdk.t)) └─TableScan_15 10.00 cop table:dd, keep order:false, stats:pseudo explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; id count task operator info @@ -150,15 +150,15 @@ Projection_5 1.00 root test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st.p └─TableScan_13 250.00 cop table:st, keep order:false, stats:pseudo explain select dt.id as id, dt.aid as aid, dt.pt as pt, dt.dic as dic, dt.cm as cm, rr.gid as gid, rr.acd as acd, rr.t as t,dt.p1 as p1, dt.p2 as p2, dt.p3 as p3, dt.p4 as p4, dt.p5 as p5, dt.p6_md5 as p6, dt.p7_md5 as p7 from dt dt join rr rr on (rr.pt = 'ios' and rr.t > 1478185592 and dt.aid = rr.aid and dt.dic = rr.dic) where dt.pt = 'ios' and dt.t > 1478185592 and dt.bm = 0 limit 2000; id count task operator info -Projection_10 0.00 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr.t, dt.p1, dt.p2, dt.p3, dt.p4, dt.p5, dt.p6_md5, dt.p7_md5 +Projection_10 0.00 root test.dt.id, test.dt.aid, test.dt.pt, test.dt.dic, test.dt.cm, test.rr.gid, test.rr.acd, test.rr.t, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5 └─Limit_13 0.00 root offset:0, count:2000 - └─IndexJoin_19 0.00 root inner join, inner:IndexLookUp_18, outer key:dt.aid, dt.dic, inner key:rr.aid, rr.dic + └─IndexJoin_19 0.00 root inner join, inner:IndexLookUp_18, outer key:test.dt.aid, test.dt.dic, inner key:test.rr.aid, test.rr.dic ├─TableReader_41 0.00 root data:Selection_40 - │ └─Selection_40 0.00 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) + │ └─Selection_40 0.00 cop eq(test.dt.bm, 0), eq(test.dt.pt, "ios"), gt(test.dt.t, 1478185592), not(isnull(test.dt.dic)) │ └─TableScan_39 10000.00 cop table:dt, range:[0,+inf], keep order:false, stats:pseudo └─IndexLookUp_18 3.33 root - ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false, stats:pseudo - └─Selection_17 3.33 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) + ├─IndexScan_15 10.00 cop table:rr, index:aid, dic, range: decided by [eq(test.rr.aid, test.dt.aid) eq(test.rr.dic, test.dt.dic)], keep order:false, stats:pseudo + └─Selection_17 3.33 cop eq(test.rr.pt, "ios"), gt(test.rr.t, 1478185592) └─TableScan_16 10.00 cop table:rr, keep order:false, stats:pseudo explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; id count task operator info diff --git a/cmd/explaintest/r/explain_complex_stats.result b/cmd/explaintest/r/explain_complex_stats.result index 5db97abb7f63e..32e2eb653c670 100644 --- a/cmd/explaintest/r/explain_complex_stats.result +++ b/cmd/explaintest/r/explain_complex_stats.result @@ -125,28 +125,28 @@ Projection_7 21.40 root test.dt.ds, test.dt.p1, test.dt.p2, test.dt.p3, test.dt. └─TableScan_14 128.32 cop table:dt, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext, gad.t as gtime from st gad join (select id, aid, pt, dic, ip, t from dd where pt = 'android' and bm = 0 and t > 1478143908) sdk on gad.aid = sdk.aid and gad.ip = sdk.ip and sdk.t > gad.t where gad.t > 1478143908 and gad.bm = 0 and gad.pt = 'android' group by gad.aid, sdk.dic limit 2500; id count task operator info -Projection_13 424.00 root gad.id, test.dd.id, gad.aid, gad.cm, test.dd.dic, test.dd.ip, test.dd.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext, gad.t +Projection_13 424.00 root test.gad.id, test.dd.id, test.gad.aid, test.gad.cm, test.dd.dic, test.dd.ip, test.dd.t, test.gad.p1, test.gad.p2, test.gad.p3, test.gad.p4, test.gad.p5, test.gad.p6_md5, test.gad.p7_md5, test.gad.ext, test.gad.t └─Limit_16 424.00 root offset:0, count:2500 - └─HashAgg_19 424.00 root group by:gad.aid, test.dd.dic, funcs:firstrow(gad.id), firstrow(gad.aid), firstrow(gad.cm), firstrow(gad.p1), firstrow(gad.p2), firstrow(gad.p3), firstrow(gad.p4), firstrow(gad.p5), firstrow(gad.p6_md5), firstrow(gad.p7_md5), firstrow(gad.ext), firstrow(gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) - └─IndexJoin_24 424.00 root inner join, inner:IndexLookUp_23, outer key:gad.aid, inner key:test.dd.aid, other cond:eq(gad.ip, test.dd.ip), gt(test.dd.t, gad.t) + └─HashAgg_19 424.00 root group by:test.dd.dic, test.gad.aid, funcs:firstrow(test.gad.id), firstrow(test.gad.aid), firstrow(test.gad.cm), firstrow(test.gad.p1), firstrow(test.gad.p2), firstrow(test.gad.p3), firstrow(test.gad.p4), firstrow(test.gad.p5), firstrow(test.gad.p6_md5), firstrow(test.gad.p7_md5), firstrow(test.gad.ext), firstrow(test.gad.t), firstrow(test.dd.id), firstrow(test.dd.dic), firstrow(test.dd.ip), firstrow(test.dd.t) + └─IndexJoin_24 424.00 root inner join, inner:IndexLookUp_23, outer key:test.gad.aid, inner key:test.dd.aid, other cond:eq(test.gad.ip, test.dd.ip), gt(test.dd.t, test.gad.t) ├─TableReader_29 424.00 root data:Selection_28 - │ └─Selection_28 424.00 cop eq(gad.bm, 0), eq(gad.pt, "android"), gt(gad.t, 1478143908), not(isnull(gad.ip)) + │ └─Selection_28 424.00 cop eq(test.gad.bm, 0), eq(test.gad.pt, "android"), gt(test.gad.t, 1478143908), not(isnull(test.gad.ip)) │ └─TableScan_27 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_23 455.80 root - ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, gad.aid)], keep order:false + ├─IndexScan_20 1.00 cop table:dd, index:aid, dic, range: decided by [eq(test.dd.aid, test.gad.aid)], keep order:false └─Selection_22 455.80 cop eq(test.dd.bm, 0), eq(test.dd.pt, "android"), gt(test.dd.t, 1478143908), not(isnull(test.dd.ip)), not(isnull(test.dd.t)) └─TableScan_21 1.00 cop table:dd, keep order:false explain select gad.id as gid,sdk.id as sid,gad.aid as aid,gad.cm as cm,sdk.dic as dic,sdk.ip as ip, sdk.t as t, gad.p1 as p1, gad.p2 as p2, gad.p3 as p3, gad.p4 as p4, gad.p5 as p5, gad.p6_md5 as p6, gad.p7_md5 as p7, gad.ext as ext from st gad join dd sdk on gad.aid = sdk.aid and gad.dic = sdk.mac and gad.t < sdk.t where gad.t > 1477971479 and gad.bm = 0 and gad.pt = 'ios' and gad.dit = 'mac' and sdk.t > 1477971479 and sdk.bm = 0 and sdk.pt = 'ios' limit 3000; id count task operator info -Projection_10 170.34 root gad.id, sdk.id, gad.aid, gad.cm, sdk.dic, sdk.ip, sdk.t, gad.p1, gad.p2, gad.p3, gad.p4, gad.p5, gad.p6_md5, gad.p7_md5, gad.ext +Projection_10 170.34 root test.gad.id, test.sdk.id, test.gad.aid, test.gad.cm, test.sdk.dic, test.sdk.ip, test.sdk.t, test.gad.p1, test.gad.p2, test.gad.p3, test.gad.p4, test.gad.p5, test.gad.p6_md5, test.gad.p7_md5, test.gad.ext └─Limit_13 170.34 root offset:0, count:3000 - └─IndexJoin_18 170.34 root inner join, inner:IndexLookUp_17, outer key:gad.aid, inner key:sdk.aid, other cond:eq(gad.dic, sdk.mac), lt(gad.t, sdk.t) + └─IndexJoin_18 170.34 root inner join, inner:IndexLookUp_17, outer key:test.gad.aid, inner key:test.sdk.aid, other cond:eq(test.gad.dic, test.sdk.mac), lt(test.gad.t, test.sdk.t) ├─TableReader_23 170.34 root data:Selection_22 - │ └─Selection_22 170.34 cop eq(gad.bm, 0), eq(gad.dit, "mac"), eq(gad.pt, "ios"), gt(gad.t, 1477971479), not(isnull(gad.dic)) + │ └─Selection_22 170.34 cop eq(test.gad.bm, 0), eq(test.gad.dit, "mac"), eq(test.gad.pt, "ios"), gt(test.gad.t, 1477971479), not(isnull(test.gad.dic)) │ └─TableScan_21 1999.00 cop table:gad, range:[0,+inf], keep order:false └─IndexLookUp_17 509.04 root - ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [eq(sdk.aid, gad.aid)], keep order:false - └─Selection_16 509.04 cop eq(sdk.bm, 0), eq(sdk.pt, "ios"), gt(sdk.t, 1477971479), not(isnull(sdk.mac)), not(isnull(sdk.t)) + ├─IndexScan_14 1.00 cop table:sdk, index:aid, dic, range: decided by [eq(test.sdk.aid, test.gad.aid)], keep order:false + └─Selection_16 509.04 cop eq(test.sdk.bm, 0), eq(test.sdk.pt, "ios"), gt(test.sdk.t, 1477971479), not(isnull(test.sdk.mac)), not(isnull(test.sdk.t)) └─TableScan_15 1.00 cop table:dd, keep order:false explain SELECT cm, p1, p2, p3, p4, p5, p6_md5, p7_md5, count(1) as click_pv, count(DISTINCT ip) as click_ip FROM st WHERE (t between 1478188800 and 1478275200) and aid='cn.sbkcq' and pt='android' GROUP BY cm, p1, p2, p3, p4, p5, p6_md5, p7_md5; id count task operator info @@ -158,15 +158,15 @@ Projection_5 39.28 root test.st.cm, test.st.p1, test.st.p2, test.st.p3, test.st. └─TableScan_13 160.23 cop table:st, keep order:false explain select dt.id as id, dt.aid as aid, dt.pt as pt, dt.dic as dic, dt.cm as cm, rr.gid as gid, rr.acd as acd, rr.t as t,dt.p1 as p1, dt.p2 as p2, dt.p3 as p3, dt.p4 as p4, dt.p5 as p5, dt.p6_md5 as p6, dt.p7_md5 as p7 from dt dt join rr rr on (rr.pt = 'ios' and rr.t > 1478185592 and dt.aid = rr.aid and dt.dic = rr.dic) where dt.pt = 'ios' and dt.t > 1478185592 and dt.bm = 0 limit 2000; id count task operator info -Projection_10 428.32 root dt.id, dt.aid, dt.pt, dt.dic, dt.cm, rr.gid, rr.acd, rr.t, dt.p1, dt.p2, dt.p3, dt.p4, dt.p5, dt.p6_md5, dt.p7_md5 +Projection_10 428.32 root test.dt.id, test.dt.aid, test.dt.pt, test.dt.dic, test.dt.cm, test.rr.gid, test.rr.acd, test.rr.t, test.dt.p1, test.dt.p2, test.dt.p3, test.dt.p4, test.dt.p5, test.dt.p6_md5, test.dt.p7_md5 └─Limit_13 428.32 root offset:0, count:2000 - └─IndexJoin_19 428.32 root inner join, inner:IndexLookUp_18, outer key:dt.aid, dt.dic, inner key:rr.aid, rr.dic + └─IndexJoin_19 428.32 root inner join, inner:IndexLookUp_18, outer key:test.dt.aid, test.dt.dic, inner key:test.rr.aid, test.rr.dic ├─TableReader_41 428.32 root data:Selection_40 - │ └─Selection_40 428.32 cop eq(dt.bm, 0), eq(dt.pt, "ios"), gt(dt.t, 1478185592), not(isnull(dt.dic)) + │ └─Selection_40 428.32 cop eq(test.dt.bm, 0), eq(test.dt.pt, "ios"), gt(test.dt.t, 1478185592), not(isnull(test.dt.dic)) │ └─TableScan_39 2000.00 cop table:dt, range:[0,+inf], keep order:false └─IndexLookUp_18 970.00 root - ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [eq(rr.aid, dt.aid) eq(rr.dic, dt.dic)], keep order:false - └─Selection_17 970.00 cop eq(rr.pt, "ios"), gt(rr.t, 1478185592) + ├─IndexScan_15 1.00 cop table:rr, index:aid, dic, range: decided by [eq(test.rr.aid, test.dt.aid) eq(test.rr.dic, test.dt.dic)], keep order:false + └─Selection_17 970.00 cop eq(test.rr.pt, "ios"), gt(test.rr.t, 1478185592) └─TableScan_16 1.00 cop table:rr, keep order:false explain select pc,cr,count(DISTINCT uid) as pay_users,count(oid) as pay_times,sum(am) as am from pp where ps=2 and ppt>=1478188800 and ppt<1478275200 and pi in ('510017','520017') and uid in ('18089709','18090780') group by pc,cr; id count task operator info diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 1497fb03490de..5255df34cea89 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -16,7 +16,7 @@ Projection_12 8000.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d │ └─TableScan_15 10000.00 cop table:t3, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_29 1.00 root funcs:sum(col_0) └─TableReader_30 1.00 root data:StreamAgg_21 - └─StreamAgg_21 1.00 cop funcs:sum(s.a) + └─StreamAgg_21 1.00 cop funcs:sum(test.s.a) └─TableScan_28 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t1; id count task operator info @@ -60,13 +60,13 @@ IndexLookUp_9 10.00 root explain select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id count task operator info Projection_11 9990.00 root cast(join_agg_0) -└─IndexJoin_14 9990.00 root inner join, inner:TableReader_13, outer key:b.c2, inner key:a.c1 +└─IndexJoin_14 9990.00 root inner join, inner:TableReader_13, outer key:test.b.c2, inner key:test.a.c1 ├─TableReader_13 1.00 root data:TableScan_12 - │ └─TableScan_12 1.00 cop table:a, range: decided by [b.c2], keep order:false, stats:pseudo + │ └─TableScan_12 1.00 cop table:a, range: decided by [test.b.c2], keep order:false, stats:pseudo └─HashAgg_22 7992.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) └─TableReader_23 7992.00 root data:HashAgg_17 - └─HashAgg_17 7992.00 cop group by:b.c2, funcs:count(b.c2), firstrow(b.c2) - └─Selection_21 9990.00 cop not(isnull(b.c2)) + └─HashAgg_17 7992.00 cop group by:test.b.c2, funcs:count(test.b.c2), firstrow(test.b.c2) + └─Selection_21 9990.00 cop not(isnull(test.b.c2)) └─TableScan_20 10000.00 cop table:b, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t2 order by t2.c2 limit 0, 1; id count task operator info @@ -110,10 +110,10 @@ explain select (select count(1) k from t1 s where s.c1 = t1.c1 having k != 0) fr id count task operator info Projection_12 10000.00 root k └─Projection_13 10000.00 root test.t1.c1, ifnull(5_col_0, 0) - └─MergeJoin_14 10000.00 root left outer join, left key:test.t1.c1, right key:s.c1 + └─MergeJoin_14 10000.00 root left outer join, left key:test.t1.c1, right key:test.s.c1 ├─TableReader_17 10000.00 root data:TableScan_16 │ └─TableScan_16 10000.00 cop table:t1, range:[-inf,+inf], keep order:true, stats:pseudo - └─Projection_18 8000.00 root 1, s.c1 + └─Projection_18 8000.00 root 1, test.s.c1 └─TableReader_20 10000.00 root data:TableScan_19 └─TableScan_19 10000.00 cop table:s, range:[-inf,+inf], keep order:true, stats:pseudo explain select * from information_schema.columns; @@ -288,12 +288,12 @@ Projection_11 10000.00 root 9_aux_0 ├─TableReader_15 10000.00 root data:TableScan_14 │ └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_20 1.00 root funcs:count(1) - └─IndexJoin_44 12.50 root inner join, inner:TableReader_43, outer key:s.a, inner key:t1.a + └─IndexJoin_44 12.50 root inner join, inner:TableReader_43, outer key:test.s.a, inner key:test.t1.a ├─TableReader_37 1.00 root data:TableScan_36 - │ └─TableScan_36 1.00 cop table:s, range: decided by [eq(s.a, test.t.a)], keep order:false, stats:pseudo + │ └─TableScan_36 1.00 cop table:s, range: decided by [eq(test.s.a, test.t.a)], keep order:false, stats:pseudo └─TableReader_43 0.80 root data:Selection_42 - └─Selection_42 0.80 cop eq(t1.a, test.t.a) - └─TableScan_41 1.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo + └─Selection_42 0.80 cop eq(test.t1.a, test.t.a) + └─TableScan_41 1.00 cop table:t1, range: decided by [test.s.a], keep order:false, stats:pseudo explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.a = t1.a) from t; id count task operator info Projection_11 10000.00 root 9_aux_0 @@ -301,11 +301,11 @@ Projection_11 10000.00 root 9_aux_0 ├─TableReader_15 10000.00 root data:TableScan_14 │ └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_20 1.00 root funcs:count(1) - └─IndexJoin_32 12.50 root inner join, inner:TableReader_31, outer key:s.a, inner key:t1.a + └─IndexJoin_32 12.50 root inner join, inner:TableReader_31, outer key:test.s.a, inner key:test.t1.a ├─IndexReader_27 10.00 root index:IndexScan_26 - │ └─IndexScan_26 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo + │ └─IndexScan_26 10.00 cop table:s, index:b, range: decided by [eq(test.s.b, test.t.a)], keep order:false, stats:pseudo └─TableReader_31 1.00 root data:TableScan_30 - └─TableScan_30 1.00 cop table:t1, range: decided by [s.a], keep order:false, stats:pseudo + └─TableScan_30 1.00 cop table:t1, range: decided by [test.s.a], keep order:false, stats:pseudo explain select t.c in (select count(*) from t s use index(idx), t t1 where s.b = t.a and s.c = t1.a) from t; id count task operator info Projection_11 10000.00 root 9_aux_0 @@ -313,13 +313,13 @@ Projection_11 10000.00 root 9_aux_0 ├─TableReader_15 10000.00 root data:TableScan_14 │ └─TableScan_14 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_20 1.00 root funcs:count(1) - └─IndexJoin_34 12.49 root inner join, inner:TableReader_33, outer key:s.c, inner key:t1.a + └─IndexJoin_34 12.49 root inner join, inner:TableReader_33, outer key:test.s.c, inner key:test.t1.a ├─IndexLookUp_29 9.99 root - │ ├─IndexScan_26 10.00 cop table:s, index:b, range: decided by [eq(s.b, test.t.a)], keep order:false, stats:pseudo - │ └─Selection_28 9.99 cop not(isnull(s.c)) + │ ├─IndexScan_26 10.00 cop table:s, index:b, range: decided by [eq(test.s.b, test.t.a)], keep order:false, stats:pseudo + │ └─Selection_28 9.99 cop not(isnull(test.s.c)) │ └─TableScan_27 10.00 cop table:t, keep order:false, stats:pseudo └─TableReader_33 1.00 root data:TableScan_32 - └─TableScan_32 1.00 cop table:t1, range: decided by [s.c], keep order:false, stats:pseudo + └─TableScan_32 1.00 cop table:t1, range: decided by [test.s.c], keep order:false, stats:pseudo insert into t values(1, 1, 1), (2, 2 ,2), (3, 3, 3), (4, 3, 4),(5,3,5); analyze table t; explain select t.c in (select count(*) from t s, t t1 where s.b = t.a and s.b = 3 and s.a = t1.a) from t; @@ -329,13 +329,13 @@ Projection_11 5.00 root 9_aux_0 ├─TableReader_15 5.00 root data:TableScan_14 │ └─TableScan_14 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_20 1.00 root funcs:count(1) - └─IndexJoin_48 2.40 root inner join, inner:TableReader_47, outer key:s.a, inner key:t1.a + └─IndexJoin_48 2.40 root inner join, inner:TableReader_47, outer key:test.s.a, inner key:test.t1.a ├─IndexReader_40 2.40 root index:Selection_39 │ └─Selection_39 2.40 cop eq(3, test.t.a) │ └─IndexScan_38 3.00 cop table:s, index:b, range:[3,3], keep order:false └─TableReader_47 0.80 root data:Selection_46 └─Selection_46 0.80 cop eq(3, test.t.a) - └─TableScan_45 1.00 cop table:t1, range: decided by [s.a], keep order:false + └─TableScan_45 1.00 cop table:t1, range: decided by [test.s.a], keep order:false explain select t.c in (select count(*) from t s left join t t1 on s.a = t1.a where 3 = t.a and s.b = 3) from t; id count task operator info Projection_10 5.00 root 9_aux_0 @@ -343,13 +343,13 @@ Projection_10 5.00 root 9_aux_0 ├─TableReader_14 5.00 root data:TableScan_13 │ └─TableScan_13 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_19 1.00 root funcs:count(1) - └─IndexJoin_42 2.40 root left outer join, inner:TableReader_41, outer key:s.a, inner key:t1.a + └─IndexJoin_42 2.40 root left outer join, inner:TableReader_41, outer key:test.s.a, inner key:test.t1.a ├─IndexReader_34 2.40 root index:Selection_33 │ └─Selection_33 2.40 cop eq(3, test.t.a) │ └─IndexScan_32 3.00 cop table:s, index:b, range:[3,3], keep order:false └─TableReader_41 0.80 root data:Selection_40 └─Selection_40 0.80 cop eq(3, test.t.a) - └─TableScan_39 1.00 cop table:t1, range: decided by [s.a], keep order:false + └─TableScan_39 1.00 cop table:t1, range: decided by [test.s.a], keep order:false explain select t.c in (select count(*) from t s right join t t1 on s.a = t1.a where 3 = t.a and t1.b = 3) from t; id count task operator info Projection_10 5.00 root 9_aux_0 @@ -357,10 +357,10 @@ Projection_10 5.00 root 9_aux_0 ├─TableReader_14 5.00 root data:TableScan_13 │ └─TableScan_13 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_19 1.00 root funcs:count(1) - └─IndexJoin_42 2.40 root right outer join, inner:TableReader_41, outer key:t1.a, inner key:s.a + └─IndexJoin_42 2.40 root right outer join, inner:TableReader_41, outer key:test.t1.a, inner key:test.s.a ├─TableReader_41 0.80 root data:Selection_40 │ └─Selection_40 0.80 cop eq(3, test.t.a) - │ └─TableScan_39 1.00 cop table:s, range: decided by [t1.a], keep order:false + │ └─TableScan_39 1.00 cop table:s, range: decided by [test.t1.a], keep order:false └─IndexReader_34 2.40 root index:Selection_33 └─Selection_33 2.40 cop eq(3, test.t.a) └─IndexScan_32 3.00 cop table:t1, index:b, range:[3,3], keep order:false @@ -409,17 +409,17 @@ id count task operator info TableDual_5 0.00 root rows:0 explain select * from t t1 join t t2 where t1.b = t2.b and t2.b is null; id count task operator info -Projection_7 0.00 root t1.a, t1.b, t2.a, t2.b -└─HashRightJoin_9 0.00 root inner join, inner:TableReader_12, equal:[eq(t2.b, t1.b)] +Projection_7 0.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b +└─HashRightJoin_9 0.00 root inner join, inner:TableReader_12, equal:[eq(test.t2.b, test.t1.b)] ├─TableReader_12 0.00 root data:Selection_11 - │ └─Selection_11 0.00 cop isnull(t2.b), not(isnull(t2.b)) + │ └─Selection_11 0.00 cop isnull(test.t2.b), not(isnull(test.t2.b)) │ └─TableScan_10 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_15 9990.00 root data:Selection_14 - └─Selection_14 9990.00 cop not(isnull(t1.b)) + └─Selection_14 9990.00 cop not(isnull(test.t1.b)) └─TableScan_13 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t t1 where not exists (select * from t t2 where t1.b = t2.b); id count task operator info -HashLeftJoin_9 8000.00 root anti semi join, inner:TableReader_13, equal:[eq(t1.b, t2.b)] +HashLeftJoin_9 8000.00 root anti semi join, inner:TableReader_13, equal:[eq(test.t1.b, test.t2.b)] ├─TableReader_11 10000.00 root data:TableScan_10 │ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_13 10000.00 root data:TableScan_12 @@ -503,29 +503,29 @@ Projection_4 10.00 root plus(1, test.t.nb) └─TableScan_5 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t ta left outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(ta.nb, 1) or ta.nb is null; id count task operator info -HashLeftJoin_7 8320.83 root left outer join, inner:TableReader_12, equal:[eq(ta.nb, tb.nb)], left cond:[gt(ta.a, 1)] +HashLeftJoin_7 8320.83 root left outer join, inner:TableReader_12, equal:[eq(test.ta.nb, test.tb.nb)], left cond:[gt(test.ta.a, 1)] ├─TableReader_10 6656.67 root data:Selection_9 -│ └─Selection_9 6656.67 cop or(ta.nb, isnull(ta.nb)) +│ └─Selection_9 6656.67 cop or(test.ta.nb, isnull(test.ta.nb)) │ └─TableScan_8 10000.00 cop table:ta, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_12 10000.00 root data:TableScan_11 └─TableScan_11 10000.00 cop table:tb, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t ta right outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.nb, 1) or tb.nb is null; id count task operator info -HashRightJoin_7 6656.67 root right outer join, inner:TableReader_10, equal:[eq(ta.nb, tb.nb)] +HashRightJoin_7 6656.67 root right outer join, inner:TableReader_10, equal:[eq(test.ta.nb, test.tb.nb)] ├─TableReader_10 3333.33 root data:Selection_9 -│ └─Selection_9 3333.33 cop gt(ta.a, 1) +│ └─Selection_9 3333.33 cop gt(test.ta.a, 1) │ └─TableScan_8 10000.00 cop table:ta, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_13 6656.67 root data:Selection_12 - └─Selection_12 6656.67 cop or(tb.nb, isnull(tb.nb)) + └─Selection_12 6656.67 cop or(test.tb.nb, isnull(test.tb.nb)) └─TableScan_11 10000.00 cop table:tb, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t ta inner join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.nb, 1) or tb.nb is null; id count task operator info -HashRightJoin_9 4166.67 root inner join, inner:TableReader_12, equal:[eq(ta.nb, tb.nb)] +HashRightJoin_9 4166.67 root inner join, inner:TableReader_12, equal:[eq(test.ta.nb, test.tb.nb)] ├─TableReader_12 3333.33 root data:Selection_11 -│ └─Selection_11 3333.33 cop gt(ta.a, 1) +│ └─Selection_11 3333.33 cop gt(test.ta.a, 1) │ └─TableScan_10 10000.00 cop table:ta, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_15 6656.67 root data:Selection_14 - └─Selection_14 6656.67 cop or(tb.nb, isnull(tb.nb)) + └─Selection_14 6656.67 cop or(test.tb.nb, isnull(test.tb.nb)) └─TableScan_13 10000.00 cop table:tb, range:[-inf,+inf], keep order:false, stats:pseudo explain select ifnull(t.nc, 1) in (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t; id count task operator info @@ -534,31 +534,31 @@ Projection_12 10000.00 root 9_aux_0 ├─TableReader_16 10000.00 root data:TableScan_15 │ └─TableScan_15 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─HashAgg_19 1.00 root funcs:count(join_agg_0) - └─HashLeftJoin_20 9.99 root inner join, inner:HashAgg_30, equal:[eq(s.a, t1.a)] + └─HashLeftJoin_20 9.99 root inner join, inner:HashAgg_30, equal:[eq(test.s.a, test.t1.a)] ├─TableReader_24 9.99 root data:Selection_23 - │ └─Selection_23 9.99 cop eq(s.a, test.t.a), not(isnull(s.a)) + │ └─Selection_23 9.99 cop eq(test.s.a, test.t.a), not(isnull(test.s.a)) │ └─TableScan_22 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo └─HashAgg_30 7.99 root group by:col_2, funcs:count(col_0), firstrow(col_1) └─TableReader_31 7.99 root data:HashAgg_25 - └─HashAgg_25 7.99 cop group by:t1.a, funcs:count(1), firstrow(t1.a) - └─Selection_29 9.99 cop eq(t1.a, test.t.a), not(isnull(t1.a)) + └─HashAgg_25 7.99 cop group by:test.t1.a, funcs:count(1), firstrow(test.t1.a) + └─Selection_29 9.99 cop eq(test.t1.a, test.t.a), not(isnull(test.t1.a)) └─TableScan_28 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t ta left outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.a, 1) or tb.a is null; id count task operator info -Selection_7 10000.00 root or(ifnull(tb.a, 1), isnull(tb.a)) -└─HashLeftJoin_8 12500.00 root left outer join, inner:TableReader_12, equal:[eq(ta.nb, tb.nb)], left cond:[gt(ta.a, 1)] +Selection_7 10000.00 root or(ifnull(test.tb.a, 1), isnull(test.tb.a)) +└─HashLeftJoin_8 12500.00 root left outer join, inner:TableReader_12, equal:[eq(test.ta.nb, test.tb.nb)], left cond:[gt(test.ta.a, 1)] ├─TableReader_10 10000.00 root data:TableScan_9 │ └─TableScan_9 10000.00 cop table:ta, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_12 10000.00 root data:TableScan_11 └─TableScan_11 10000.00 cop table:tb, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from t ta right outer join t tb on ta.nb = tb.nb and ta.a > 1 where ifnull(tb.a, 1) or tb.a is null; id count task operator info -HashRightJoin_7 8000.00 root right outer join, inner:TableReader_10, equal:[eq(ta.nb, tb.nb)] +HashRightJoin_7 8000.00 root right outer join, inner:TableReader_10, equal:[eq(test.ta.nb, test.tb.nb)] ├─TableReader_10 3333.33 root data:Selection_9 -│ └─Selection_9 3333.33 cop gt(ta.a, 1) +│ └─Selection_9 3333.33 cop gt(test.ta.a, 1) │ └─TableScan_8 10000.00 cop table:ta, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_13 8000.00 root data:Selection_12 - └─Selection_12 8000.00 cop or(ifnull(tb.a, 1), isnull(tb.a)) + └─Selection_12 8000.00 cop or(ifnull(test.tb.a, 1), isnull(test.tb.a)) └─TableScan_11 10000.00 cop table:tb, range:[-inf,+inf], keep order:false, stats:pseudo explain select ifnull(t.a, 1) in (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t; id count task operator info @@ -567,14 +567,14 @@ Projection_12 10000.00 root 9_aux_0 ├─TableReader_16 10000.00 root data:TableScan_15 │ └─TableScan_15 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo └─HashAgg_19 1.00 root funcs:count(join_agg_0) - └─HashLeftJoin_20 9.99 root inner join, inner:HashAgg_30, equal:[eq(s.a, t1.a)] + └─HashLeftJoin_20 9.99 root inner join, inner:HashAgg_30, equal:[eq(test.s.a, test.t1.a)] ├─TableReader_24 9.99 root data:Selection_23 - │ └─Selection_23 9.99 cop eq(s.a, test.t.a), not(isnull(s.a)) + │ └─Selection_23 9.99 cop eq(test.s.a, test.t.a), not(isnull(test.s.a)) │ └─TableScan_22 10000.00 cop table:s, range:[-inf,+inf], keep order:false, stats:pseudo └─HashAgg_30 7.99 root group by:col_2, funcs:count(col_0), firstrow(col_1) └─TableReader_31 7.99 root data:HashAgg_25 - └─HashAgg_25 7.99 cop group by:t1.a, funcs:count(1), firstrow(t1.a) - └─Selection_29 9.99 cop eq(t1.a, test.t.a), not(isnull(t1.a)) + └─HashAgg_25 7.99 cop group by:test.t1.a, funcs:count(1), firstrow(test.t1.a) + └─Selection_29 9.99 cop eq(test.t1.a, test.t.a), not(isnull(test.t1.a)) └─TableScan_28 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo drop table if exists t; create table t(a int); @@ -603,12 +603,12 @@ drop table if exists t; create table t(a int, b int, c int); explain select * from (select * from t order by (select 2)) t order by a, b; id count task operator info -Sort_12 10000.00 root t.a:asc, t.b:asc +Sort_12 10000.00 root test.t.a:asc, test.t.b:asc └─TableReader_18 10000.00 root data:TableScan_17 └─TableScan_17 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo explain select * from (select * from t order by c) t order by a, b; id count task operator info -Sort_6 10000.00 root t.a:asc, t.b:asc +Sort_6 10000.00 root test.t.a:asc, test.t.b:asc └─Sort_9 10000.00 root test.t.c:asc └─TableReader_12 10000.00 root data:TableScan_11 └─TableScan_11 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index 1f41bb9e626a7..44f693e962177 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -19,7 +19,7 @@ Projection_12 1600.00 root test.t3.a, test.t3.b, test.t3.c, test.t3.d │ └─TableScan_15 2000.00 cop table:t3, range:[-inf,+inf], keep order:false └─StreamAgg_29 1.00 root funcs:sum(col_0) └─TableReader_30 1.00 root data:StreamAgg_21 - └─StreamAgg_21 1.00 cop funcs:sum(s.a) + └─StreamAgg_21 1.00 cop funcs:sum(test.s.a) └─TableScan_28 2000.00 cop table:s, range:[-inf,+inf], keep order:false explain select * from t1; id count task operator info @@ -65,13 +65,13 @@ IndexLookUp_9 0.00 root explain select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id count task operator info Projection_11 1985.00 root cast(join_agg_0) -└─IndexJoin_14 1985.00 root inner join, inner:TableReader_13, outer key:b.c2, inner key:a.c1 +└─IndexJoin_14 1985.00 root inner join, inner:TableReader_13, outer key:test.b.c2, inner key:test.a.c1 ├─TableReader_13 1.00 root data:TableScan_12 - │ └─TableScan_12 1.00 cop table:a, range: decided by [b.c2], keep order:false + │ └─TableScan_12 1.00 cop table:a, range: decided by [test.b.c2], keep order:false └─HashAgg_22 1985.00 root group by:col_2, funcs:count(col_0), firstrow(col_1) └─TableReader_23 1985.00 root data:HashAgg_17 - └─HashAgg_17 1985.00 cop group by:b.c2, funcs:count(b.c2), firstrow(b.c2) - └─Selection_21 1985.00 cop not(isnull(b.c2)) + └─HashAgg_17 1985.00 cop group by:test.b.c2, funcs:count(test.b.c2), firstrow(test.b.c2) + └─Selection_21 1985.00 cop not(isnull(test.b.c2)) └─TableScan_20 1985.00 cop table:b, range:[-inf,+inf], keep order:false explain select * from t2 order by t2.c2 limit 0, 1; id count task operator info diff --git a/cmd/explaintest/r/select.result b/cmd/explaintest/r/select.result index 3e396148fa057..5f56ff0e3c775 100644 --- a/cmd/explaintest/r/select.result +++ b/cmd/explaintest/r/select.result @@ -325,18 +325,18 @@ drop table if exists t; create table t (id int primary key, a int, b int); explain select * from (t t1 left join t t2 on t1.a = t2.a) left join (t t3 left join t t4 on t3.a = t4.a) on t2.b = 1; id count task operator info -HashLeftJoin_10 155937656.25 root left outer join, inner:HashLeftJoin_17, left cond:[eq(t2.b, 1)] -├─HashLeftJoin_11 12487.50 root left outer join, inner:TableReader_16, equal:[eq(t1.a, t2.a)] +HashLeftJoin_10 155937656.25 root left outer join, inner:HashLeftJoin_17, left cond:[eq(test.t2.b, 1)] +├─HashLeftJoin_11 12487.50 root left outer join, inner:TableReader_16, equal:[eq(test.t1.a, test.t2.a)] │ ├─TableReader_13 10000.00 root data:TableScan_12 │ │ └─TableScan_12 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo │ └─TableReader_16 9990.00 root data:Selection_15 -│ └─Selection_15 9990.00 cop not(isnull(t2.a)) +│ └─Selection_15 9990.00 cop not(isnull(test.t2.a)) │ └─TableScan_14 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo -└─HashLeftJoin_17 12487.50 root left outer join, inner:TableReader_22, equal:[eq(t3.a, t4.a)] +└─HashLeftJoin_17 12487.50 root left outer join, inner:TableReader_22, equal:[eq(test.t3.a, test.t4.a)] ├─TableReader_19 10000.00 root data:TableScan_18 │ └─TableScan_18 10000.00 cop table:t3, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_22 9990.00 root data:Selection_21 - └─Selection_21 9990.00 cop not(isnull(t4.a)) + └─Selection_21 9990.00 cop not(isnull(test.t4.a)) └─TableScan_20 10000.00 cop table:t4, range:[-inf,+inf], keep order:false, stats:pseudo drop table if exists t; create table t(a bigint primary key, b bigint); @@ -380,22 +380,22 @@ drop table if exists t; create table t(a int, b int); explain select a != any (select a from t t2) from t t1; id count task operator info -Projection_9 10000.00 root and(or(or(gt(col_count, 1), ne(t1.a, col_firstrow)), if(ne(agg_col_sum, 0), NULL, 0)), and(ne(agg_col_cnt, 0), if(isnull(t1.a), NULL, 1))) +Projection_9 10000.00 root and(or(or(gt(col_count, 1), ne(test.t1.a, col_firstrow)), if(ne(agg_col_sum, 0), NULL, 0)), and(ne(agg_col_cnt, 0), if(isnull(test.t1.a), NULL, 1))) └─HashLeftJoin_10 10000.00 root inner join, inner:StreamAgg_17 ├─TableReader_13 10000.00 root data:TableScan_12 │ └─TableScan_12 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_17 1.00 root funcs:firstrow(col_0), count(distinct col_1), sum(col_2), count(1) - └─Projection_27 10000.00 root t2.a, t2.a, cast(isnull(t2.a)) + └─Projection_27 10000.00 root test.t2.a, test.t2.a, cast(isnull(test.t2.a)) └─TableReader_24 10000.00 root data:TableScan_23 └─TableScan_23 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo explain select a = all (select a from t t2) from t t1; id count task operator info -Projection_9 10000.00 root or(and(and(le(col_count, 1), eq(t1.a, col_firstrow)), if(ne(agg_col_sum, 0), NULL, 1)), or(eq(agg_col_cnt, 0), if(isnull(t1.a), NULL, 0))) +Projection_9 10000.00 root or(and(and(le(col_count, 1), eq(test.t1.a, col_firstrow)), if(ne(agg_col_sum, 0), NULL, 1)), or(eq(agg_col_cnt, 0), if(isnull(test.t1.a), NULL, 0))) └─HashLeftJoin_10 10000.00 root inner join, inner:StreamAgg_17 ├─TableReader_13 10000.00 root data:TableScan_12 │ └─TableScan_12 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─StreamAgg_17 1.00 root funcs:firstrow(col_0), count(distinct col_1), sum(col_2), count(1) - └─Projection_27 10000.00 root t2.a, t2.a, cast(isnull(t2.a)) + └─Projection_27 10000.00 root test.t2.a, test.t2.a, cast(isnull(test.t2.a)) └─TableReader_24 10000.00 root data:TableScan_23 └─TableScan_23 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo drop table if exists t; @@ -415,7 +415,7 @@ Projection_9 10000.00 root 6_aux_0 explain select a in (select a+b from t t2 where t2.b = t1.b) from t t1; id count task operator info Projection_7 10000.00 root 6_aux_0 -└─HashLeftJoin_8 10000.00 root left outer semi join, inner:TableReader_12, equal:[eq(t1.b, t2.b)], other cond:eq(t1.a, plus(t2.a, t2.b)) +└─HashLeftJoin_8 10000.00 root left outer semi join, inner:TableReader_12, equal:[eq(test.t1.b, test.t2.b)], other cond:eq(test.t1.a, plus(test.t2.a, test.t2.b)) ├─TableReader_10 10000.00 root data:TableScan_9 │ └─TableScan_9 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_12 10000.00 root data:TableScan_11 @@ -425,7 +425,7 @@ create table t(a int not null, b int); explain select a in (select a from t t2 where t2.b = t1.b) from t t1; id count task operator info Projection_7 10000.00 root 6_aux_0 -└─HashLeftJoin_8 10000.00 root left outer semi join, inner:TableReader_12, equal:[eq(t1.b, t2.b) eq(t1.a, t2.a)] +└─HashLeftJoin_8 10000.00 root left outer semi join, inner:TableReader_12, equal:[eq(test.t1.b, test.t2.b) eq(test.t1.a, test.t2.a)] ├─TableReader_10 10000.00 root data:TableScan_9 │ └─TableScan_9 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_12 10000.00 root data:TableScan_11 diff --git a/cmd/explaintest/r/subquery.result b/cmd/explaintest/r/subquery.result index 39eed9b1b6344..f0bad21a8d2e0 100644 --- a/cmd/explaintest/r/subquery.result +++ b/cmd/explaintest/r/subquery.result @@ -20,8 +20,8 @@ Projection_11 5.00 root 9_aux_0 ├─TableReader_15 5.00 root data:TableScan_14 │ └─TableScan_14 5.00 cop table:t, range:[-inf,+inf], keep order:false └─StreamAgg_20 1.00 root funcs:count(1) - └─IndexJoin_23 0.50 root inner join, inner:TableReader_22, outer key:s.a, inner key:t1.a + └─IndexJoin_23 0.50 root inner join, inner:TableReader_22, outer key:test.s.a, inner key:test.t1.a ├─IndexReader_27 1.00 root index:IndexScan_26 - │ └─IndexScan_26 1.00 cop table:s, index:b, c, d, range: decided by [eq(s.b, 1) eq(s.c, 1) eq(s.d, test.t.a)], keep order:false + │ └─IndexScan_26 1.00 cop table:s, index:b, c, d, range: decided by [eq(test.s.b, 1) eq(test.s.c, 1) eq(test.s.d, test.t.a)], keep order:false └─TableReader_22 1.00 root data:TableScan_21 - └─TableScan_21 1.00 cop table:t1, range: decided by [s.a], keep order:false + └─TableScan_21 1.00 cop table:t1, range: decided by [test.s.a], keep order:false diff --git a/cmd/explaintest/r/topn_push_down.result b/cmd/explaintest/r/topn_push_down.result index e8ba90678635a..b6080a8720f82 100644 --- a/cmd/explaintest/r/topn_push_down.result +++ b/cmd/explaintest/r/topn_push_down.result @@ -167,23 +167,23 @@ te.expect_time BETWEEN '2018-04-23 00:00:00.0' AND '2018-04-23 23:59:59.0' ORDER BY te.expect_time asc LIMIT 0, 5; id count task operator info -Projection_13 0.00 root te.expect_time +Projection_13 0.00 root test.te.expect_time └─Limit_19 0.00 root offset:0, count:5 - └─IndexJoin_92 0.00 root left outer join, inner:IndexReader_91, outer key:tr.id, inner key:p.relate_id - ├─TopN_95 0.00 root te.expect_time:asc, offset:0, count:5 - │ └─IndexJoin_36 0.00 root inner join, inner:IndexLookUp_35, outer key:tr.id, inner key:te.trade_id + └─IndexJoin_92 0.00 root left outer join, inner:IndexReader_91, outer key:test.tr.id, inner key:test.p.relate_id + ├─TopN_95 0.00 root test.te.expect_time:asc, offset:0, count:5 + │ └─IndexJoin_36 0.00 root inner join, inner:IndexLookUp_35, outer key:test.tr.id, inner key:test.te.trade_id │ ├─IndexLookUp_74 0.00 root - │ │ ├─Selection_72 0.00 cop eq(tr.business_type, 18), in(tr.trade_type, 1) + │ │ ├─Selection_72 0.00 cop eq(test.tr.business_type, 18), in(test.tr.trade_type, 1) │ │ │ └─IndexScan_70 10.00 cop table:tr, index:shop_identy, trade_status, business_type, trade_pay_status, trade_type, delivery_type, source, biz_date, range:[810094178,810094178], keep order:false, stats:pseudo - │ │ └─Selection_73 0.00 cop eq(tr.brand_identy, 32314), eq(tr.domain_type, 2) + │ │ └─Selection_73 0.00 cop eq(test.tr.brand_identy, 32314), eq(test.tr.domain_type, 2) │ │ └─TableScan_71 0.00 cop table:tr, keep order:false, stats:pseudo │ └─IndexLookUp_35 250.00 root - │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [eq(te.trade_id, tr.id)], keep order:false, stats:pseudo - │ └─Selection_34 250.00 cop ge(te.expect_time, 2018-04-23 00:00:00.000000), le(te.expect_time, 2018-04-23 23:59:59.000000) + │ ├─IndexScan_32 10.00 cop table:te, index:trade_id, range: decided by [eq(test.te.trade_id, test.tr.id)], keep order:false, stats:pseudo + │ └─Selection_34 250.00 cop ge(test.te.expect_time, 2018-04-23 00:00:00.000000), le(test.te.expect_time, 2018-04-23 23:59:59.000000) │ └─TableScan_33 10.00 cop table:te, keep order:false, stats:pseudo └─IndexReader_91 0.00 root index:Selection_90 - └─Selection_90 0.00 cop not(isnull(p.relate_id)) - └─IndexScan_89 10.00 cop table:p, index:relate_id, range: decided by [eq(p.relate_id, tr.id)], keep order:false, stats:pseudo + └─Selection_90 0.00 cop not(isnull(test.p.relate_id)) + └─IndexScan_89 10.00 cop table:p, index:relate_id, range: decided by [eq(test.p.relate_id, test.tr.id)], keep order:false, stats:pseudo desc select 1 as a from dual order by a limit 1; id count task operator info Projection_6 1.00 root 1 @@ -206,11 +206,11 @@ Apply_15 9990.00 root semi join, inner:Selection_19, equal:[eq(test.t1.a, test.t └─TableScan_23 1.25 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo desc select * from t1 where t1.a in (select a from (select t2.a as a, t1.b as b from t2 where t2.b > t1.b) x order by b limit 1); id count task operator info -Apply_17 9990.00 root semi join, inner:Selection_21, equal:[eq(test.t1.a, x.a)] +Apply_17 9990.00 root semi join, inner:Selection_21, equal:[eq(test.t1.a, test.x.a)] ├─TableReader_20 9990.00 root data:Selection_19 │ └─Selection_19 9990.00 cop not(isnull(test.t1.a)) │ └─TableScan_18 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo -└─Selection_21 0.80 root not(isnull(x.a)) +└─Selection_21 0.80 root not(isnull(test.x.a)) └─Projection_22 1.00 root test.t2.a, test.t1.b └─Limit_23 1.00 root offset:0, count:1 └─TableReader_29 1.00 root data:Limit_28 @@ -222,24 +222,24 @@ create table t(a int not null, index idx(a)); explain select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 on t1.a = t2.a limit 5; id count task operator info Limit_11 5.00 root offset:0, count:5 -└─IndexJoin_15 5.00 root inner join, inner:IndexReader_14, outer key:t1.a, inner key:t2.a +└─IndexJoin_15 5.00 root inner join, inner:IndexReader_14, outer key:test.t1.a, inner key:test.t2.a ├─TableReader_17 4.00 root data:TableScan_16 │ └─TableScan_16 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_14 10.00 root index:IndexScan_13 - └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo + └─IndexScan_13 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo explain select /*+ TIDB_INLJ(t2) */ * from t t1 left join t t2 on t1.a = t2.a where t2.a is null limit 5; id count task operator info Limit_12 5.00 root offset:0, count:5 -└─Selection_13 5.00 root isnull(t2.a) - └─IndexJoin_17 5.00 root left outer join, inner:IndexReader_16, outer key:t1.a, inner key:t2.a +└─Selection_13 5.00 root isnull(test.t2.a) + └─IndexJoin_17 5.00 root left outer join, inner:IndexReader_16, outer key:test.t1.a, inner key:test.t2.a ├─TableReader_19 4.00 root data:TableScan_18 │ └─TableScan_18 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─IndexReader_16 10.00 root index:IndexScan_15 - └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [eq(t2.a, t1.a)], keep order:false, stats:pseudo + └─IndexScan_15 10.00 cop table:t2, index:a, range: decided by [eq(test.t2.a, test.t1.a)], keep order:false, stats:pseudo explain select /*+ TIDB_SMJ(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a limit 5; id count task operator info Limit_11 5.00 root offset:0, count:5 -└─MergeJoin_12 5.00 root inner join, left key:t1.a, right key:t2.a +└─MergeJoin_12 5.00 root inner join, left key:test.t1.a, right key:test.t2.a ├─IndexReader_14 4.00 root index:IndexScan_13 │ └─IndexScan_13 4.00 cop table:t1, index:a, range:[NULL,+inf], keep order:true, stats:pseudo └─IndexReader_16 4.00 root index:IndexScan_15 @@ -247,8 +247,8 @@ Limit_11 5.00 root offset:0, count:5 explain select /*+ TIDB_SMJ(t1, t2) */ * from t t1 left join t t2 on t1.a = t2.a where t2.a is null limit 5; id count task operator info Limit_12 5.00 root offset:0, count:5 -└─Selection_13 5.00 root isnull(t2.a) - └─MergeJoin_14 5.00 root left outer join, left key:t1.a, right key:t2.a +└─Selection_13 5.00 root isnull(test.t2.a) + └─MergeJoin_14 5.00 root left outer join, left key:test.t1.a, right key:test.t2.a ├─IndexReader_16 4.00 root index:IndexScan_15 │ └─IndexScan_15 4.00 cop table:t1, index:a, range:[NULL,+inf], keep order:true, stats:pseudo └─IndexReader_18 4.00 root index:IndexScan_17 @@ -256,7 +256,7 @@ Limit_12 5.00 root offset:0, count:5 explain select /*+ TIDB_HJ(t1, t2) */ * from t t1 join t t2 on t1.a = t2.a limit 5; id count task operator info Limit_11 5.00 root offset:0, count:5 -└─HashLeftJoin_19 5.00 root inner join, inner:TableReader_24, equal:[eq(t1.a, t2.a)] +└─HashLeftJoin_19 5.00 root inner join, inner:TableReader_24, equal:[eq(test.t1.a, test.t2.a)] ├─TableReader_22 4.00 root data:TableScan_21 │ └─TableScan_21 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_24 10000.00 root data:TableScan_23 @@ -264,8 +264,8 @@ Limit_11 5.00 root offset:0, count:5 explain select /*+ TIDB_HJ(t1, t2) */ * from t t1 left join t t2 on t1.a = t2.a where t2.a is null limit 5; id count task operator info Limit_12 5.00 root offset:0, count:5 -└─Selection_13 5.00 root isnull(t2.a) - └─HashLeftJoin_18 5.00 root left outer join, inner:TableReader_22, equal:[eq(t1.a, t2.a)] +└─Selection_13 5.00 root isnull(test.t2.a) + └─HashLeftJoin_18 5.00 root left outer join, inner:TableReader_22, equal:[eq(test.t1.a, test.t2.a)] ├─TableReader_20 4.00 root data:TableScan_19 │ └─TableScan_19 4.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo └─TableReader_22 10000.00 root data:TableScan_21 diff --git a/cmd/explaintest/r/tpch.result b/cmd/explaintest/r/tpch.result index a083977411ea6..21ce363ba67ac 100644 --- a/cmd/explaintest/r/tpch.result +++ b/cmd/explaintest/r/tpch.result @@ -443,17 +443,17 @@ supp_nation, cust_nation, l_year; id count task operator info -Sort_22 768.91 root shipping.supp_nation:asc, shipping.cust_nation:asc, shipping.l_year:asc -└─Projection_24 768.91 root shipping.supp_nation, shipping.cust_nation, shipping.l_year, 14_col_0 - └─HashAgg_27 768.91 root group by:shipping.cust_nation, shipping.l_year, shipping.supp_nation, funcs:sum(shipping.volume), firstrow(shipping.supp_nation), firstrow(shipping.cust_nation), firstrow(shipping.l_year) - └─Projection_28 1957240.42 root n1.n_name, n2.n_name, extract("YEAR", tpch.lineitem.l_shipdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)) - └─HashLeftJoin_33 1957240.42 root inner join, inner:TableReader_68, equal:[eq(tpch.customer.c_nationkey, n2.n_nationkey)], other cond:or(and(eq(n1.n_name, "JAPAN"), eq(n2.n_name, "INDIA")), and(eq(n1.n_name, "INDIA"), eq(n2.n_name, "JAPAN"))) +Sort_22 768.91 root tpch.shipping.supp_nation:asc, tpch.shipping.cust_nation:asc, shipping.l_year:asc +└─Projection_24 768.91 root tpch.shipping.supp_nation, tpch.shipping.cust_nation, shipping.l_year, 14_col_0 + └─HashAgg_27 768.91 root group by:shipping.l_year, tpch.shipping.cust_nation, tpch.shipping.supp_nation, funcs:sum(shipping.volume), firstrow(tpch.shipping.supp_nation), firstrow(tpch.shipping.cust_nation), firstrow(shipping.l_year) + └─Projection_28 1957240.42 root tpch.n1.n_name, tpch.n2.n_name, extract("YEAR", tpch.lineitem.l_shipdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)) + └─HashLeftJoin_33 1957240.42 root inner join, inner:TableReader_68, equal:[eq(tpch.customer.c_nationkey, tpch.n2.n_nationkey)], other cond:or(and(eq(tpch.n1.n_name, "JAPAN"), eq(tpch.n2.n_name, "INDIA")), and(eq(tpch.n1.n_name, "INDIA"), eq(tpch.n2.n_name, "JAPAN"))) ├─IndexJoin_37 24465505.20 root inner join, inner:TableReader_36, outer key:tpch.orders.o_custkey, inner key:tpch.customer.c_custkey │ ├─IndexJoin_43 24465505.20 root inner join, inner:TableReader_42, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey │ │ ├─HashRightJoin_47 24465505.20 root inner join, inner:HashRightJoin_53, equal:[eq(tpch.supplier.s_suppkey, tpch.lineitem.l_suppkey)] - │ │ │ ├─HashRightJoin_53 40000.00 root inner join, inner:TableReader_58, equal:[eq(n1.n_nationkey, tpch.supplier.s_nationkey)] + │ │ │ ├─HashRightJoin_53 40000.00 root inner join, inner:TableReader_58, equal:[eq(tpch.n1.n_nationkey, tpch.supplier.s_nationkey)] │ │ │ │ ├─TableReader_58 2.00 root data:Selection_57 - │ │ │ │ │ └─Selection_57 2.00 cop or(eq(n1.n_name, "JAPAN"), eq(n1.n_name, "INDIA")) + │ │ │ │ │ └─Selection_57 2.00 cop or(eq(tpch.n1.n_name, "JAPAN"), eq(tpch.n1.n_name, "INDIA")) │ │ │ │ │ └─TableScan_56 25.00 cop table:n1, range:[-inf,+inf], keep order:false │ │ │ │ └─TableReader_55 500000.00 root data:TableScan_54 │ │ │ │ └─TableScan_54 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false @@ -465,7 +465,7 @@ Sort_22 768.91 root shipping.supp_nation:asc, shipping.cust_nation:asc, shipping │ └─TableReader_36 1.00 root data:TableScan_35 │ └─TableScan_35 1.00 cop table:customer, range: decided by [tpch.orders.o_custkey], keep order:false └─TableReader_68 2.00 root data:Selection_67 - └─Selection_67 2.00 cop or(eq(n2.n_name, "INDIA"), eq(n2.n_name, "JAPAN")) + └─Selection_67 2.00 cop or(eq(tpch.n2.n_name, "INDIA"), eq(tpch.n2.n_name, "JAPAN")) └─TableScan_66 25.00 cop table:n2, range:[-inf,+inf], keep order:false /* Q8 National Market Share Query @@ -518,15 +518,15 @@ id count task operator info Sort_29 718.01 root all_nations.o_year:asc └─Projection_31 718.01 root all_nations.o_year, div(18_col_0, 18_col_1) └─HashAgg_34 718.01 root group by:col_3, funcs:sum(col_0), sum(col_1), firstrow(col_2) - └─Projection_89 562348.12 root case(eq(all_nations.nation, "INDIA"), all_nations.volume, 0), all_nations.volume, all_nations.o_year, all_nations.o_year - └─Projection_35 562348.12 root extract("YEAR", tpch.orders.o_orderdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), n2.n_name - └─HashLeftJoin_39 562348.12 root inner join, inner:TableReader_87, equal:[eq(tpch.supplier.s_nationkey, n2.n_nationkey)] + └─Projection_89 562348.12 root case(eq(tpch.all_nations.nation, "INDIA"), all_nations.volume, 0), all_nations.volume, all_nations.o_year, all_nations.o_year + └─Projection_35 562348.12 root extract("YEAR", tpch.orders.o_orderdate), mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), tpch.n2.n_name + └─HashLeftJoin_39 562348.12 root inner join, inner:TableReader_87, equal:[eq(tpch.supplier.s_nationkey, tpch.n2.n_nationkey)] ├─IndexJoin_43 562348.12 root inner join, inner:TableReader_42, outer key:tpch.lineitem.l_suppkey, inner key:tpch.supplier.s_suppkey │ ├─HashLeftJoin_50 562348.12 root inner join, inner:TableReader_83, equal:[eq(tpch.lineitem.l_partkey, tpch.part.p_partkey)] │ │ ├─IndexJoin_56 90661378.61 root inner join, inner:IndexLookUp_55, outer key:tpch.orders.o_orderkey, inner key:tpch.lineitem.l_orderkey │ │ │ ├─HashRightJoin_60 22382008.93 root inner join, inner:HashRightJoin_62, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] - │ │ │ │ ├─HashRightJoin_62 1500000.00 root inner join, inner:HashRightJoin_68, equal:[eq(n1.n_nationkey, tpch.customer.c_nationkey)] - │ │ │ │ │ ├─HashRightJoin_68 5.00 root inner join, inner:TableReader_73, equal:[eq(tpch.region.r_regionkey, n1.n_regionkey)] + │ │ │ │ ├─HashRightJoin_62 1500000.00 root inner join, inner:HashRightJoin_68, equal:[eq(tpch.n1.n_nationkey, tpch.customer.c_nationkey)] + │ │ │ │ │ ├─HashRightJoin_68 5.00 root inner join, inner:TableReader_73, equal:[eq(tpch.region.r_regionkey, tpch.n1.n_regionkey)] │ │ │ │ │ │ ├─TableReader_73 1.00 root data:Selection_72 │ │ │ │ │ │ │ └─Selection_72 1.00 cop eq(tpch.region.r_name, "ASIA") │ │ │ │ │ │ │ └─TableScan_71 5.00 cop table:region, range:[-inf,+inf], keep order:false @@ -591,9 +591,9 @@ order by nation, o_year desc; id count task operator info -Sort_25 2406.00 root profit.nation:asc, profit.o_year:desc -└─Projection_27 2406.00 root profit.nation, profit.o_year, 14_col_0 - └─HashAgg_30 2406.00 root group by:profit.nation, profit.o_year, funcs:sum(profit.amount), firstrow(profit.nation), firstrow(profit.o_year) +Sort_25 2406.00 root tpch.profit.nation:asc, profit.o_year:desc +└─Projection_27 2406.00 root tpch.profit.nation, profit.o_year, 14_col_0 + └─HashAgg_30 2406.00 root group by:profit.o_year, tpch.profit.nation, funcs:sum(profit.amount), firstrow(tpch.profit.nation), firstrow(profit.o_year) └─Projection_31 971049283.51 root tpch.nation.n_name, extract("YEAR", tpch.orders.o_orderdate), minus(mul(tpch.lineitem.l_extendedprice, minus(1, tpch.lineitem.l_discount)), mul(tpch.partsupp.ps_supplycost, tpch.lineitem.l_quantity)) └─IndexJoin_35 971049283.51 root inner join, inner:IndexLookUp_34, outer key:tpch.lineitem.l_suppkey, tpch.lineitem.l_partkey, inner key:tpch.partsupp.ps_suppkey, tpch.partsupp.ps_partkey ├─IndexJoin_41 241379546.70 root inner join, inner:TableReader_40, outer key:tpch.lineitem.l_orderkey, inner key:tpch.orders.o_orderkey @@ -1222,10 +1222,10 @@ id count task operator info Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 └─TopN_28 1.00 root 17_col_0:desc, tpch.supplier.s_name:asc, offset:0, count:100 └─HashAgg_34 1.00 root group by:tpch.supplier.s_name, funcs:count(1), firstrow(tpch.supplier.s_name) - └─IndexJoin_40 7828961.66 root anti semi join, inner:IndexLookUp_39, outer key:l1.l_orderkey, inner key:l3.l_orderkey, other cond:ne(l3.l_suppkey, l1.l_suppkey), ne(l3.l_suppkey, tpch.supplier.s_suppkey) - ├─IndexJoin_56 9786202.08 root semi join, inner:IndexLookUp_55, outer key:l1.l_orderkey, inner key:l2.l_orderkey, other cond:ne(l2.l_suppkey, l1.l_suppkey), ne(l2.l_suppkey, tpch.supplier.s_suppkey) - │ ├─IndexJoin_62 12232752.60 root inner join, inner:TableReader_61, outer key:l1.l_orderkey, inner key:tpch.orders.o_orderkey - │ │ ├─HashRightJoin_66 12232752.60 root inner join, inner:HashRightJoin_72, equal:[eq(tpch.supplier.s_suppkey, l1.l_suppkey)] + └─IndexJoin_40 7828961.66 root anti semi join, inner:IndexLookUp_39, outer key:tpch.l1.l_orderkey, inner key:tpch.l3.l_orderkey, other cond:ne(tpch.l3.l_suppkey, tpch.l1.l_suppkey), ne(tpch.l3.l_suppkey, tpch.supplier.s_suppkey) + ├─IndexJoin_56 9786202.08 root semi join, inner:IndexLookUp_55, outer key:tpch.l1.l_orderkey, inner key:tpch.l2.l_orderkey, other cond:ne(tpch.l2.l_suppkey, tpch.l1.l_suppkey), ne(tpch.l2.l_suppkey, tpch.supplier.s_suppkey) + │ ├─IndexJoin_62 12232752.60 root inner join, inner:TableReader_61, outer key:tpch.l1.l_orderkey, inner key:tpch.orders.o_orderkey + │ │ ├─HashRightJoin_66 12232752.60 root inner join, inner:HashRightJoin_72, equal:[eq(tpch.supplier.s_suppkey, tpch.l1.l_suppkey)] │ │ │ ├─HashRightJoin_72 20000.00 root inner join, inner:TableReader_77, equal:[eq(tpch.nation.n_nationkey, tpch.supplier.s_nationkey)] │ │ │ │ ├─TableReader_77 1.00 root data:Selection_76 │ │ │ │ │ └─Selection_76 1.00 cop eq(tpch.nation.n_name, "EGYPT") @@ -1233,17 +1233,17 @@ Projection_25 1.00 root tpch.supplier.s_name, 17_col_0 │ │ │ │ └─TableReader_74 500000.00 root data:TableScan_73 │ │ │ │ └─TableScan_73 500000.00 cop table:supplier, range:[-inf,+inf], keep order:false │ │ │ └─TableReader_80 240004648.80 root data:Selection_79 - │ │ │ └─Selection_79 240004648.80 cop gt(l1.l_receiptdate, l1.l_commitdate) + │ │ │ └─Selection_79 240004648.80 cop gt(tpch.l1.l_receiptdate, tpch.l1.l_commitdate) │ │ │ └─TableScan_78 300005811.00 cop table:l1, range:[-inf,+inf], keep order:false │ │ └─TableReader_61 0.80 root data:Selection_60 │ │ └─Selection_60 0.80 cop eq(tpch.orders.o_orderstatus, "F") - │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [l1.l_orderkey], keep order:false + │ │ └─TableScan_59 1.00 cop table:orders, range: decided by [tpch.l1.l_orderkey], keep order:false │ └─IndexLookUp_55 1.00 root - │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l2.l_orderkey, l1.l_orderkey)], keep order:false + │ ├─IndexScan_53 1.00 cop table:l2, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.l2.l_orderkey, tpch.l1.l_orderkey)], keep order:false │ └─TableScan_54 1.00 cop table:lineitem, keep order:false └─IndexLookUp_39 240004648.80 root - ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(l3.l_orderkey, l1.l_orderkey)], keep order:false - └─Selection_38 240004648.80 cop gt(l3.l_receiptdate, l3.l_commitdate) + ├─IndexScan_36 1.00 cop table:l3, index:L_ORDERKEY, L_LINENUMBER, range: decided by [eq(tpch.l3.l_orderkey, tpch.l1.l_orderkey)], keep order:false + └─Selection_38 240004648.80 cop gt(tpch.l3.l_receiptdate, tpch.l3.l_commitdate) └─TableScan_37 1.00 cop table:lineitem, keep order:false /* Q22 Global Sales Opportunity Query @@ -1294,7 +1294,7 @@ cntrycode; id count task operator info Sort_32 1.00 root custsale.cntrycode:asc └─Projection_34 1.00 root custsale.cntrycode, 28_col_0, 28_col_1 - └─HashAgg_37 1.00 root group by:custsale.cntrycode, funcs:count(1), sum(custsale.c_acctbal), firstrow(custsale.cntrycode) + └─HashAgg_37 1.00 root group by:custsale.cntrycode, funcs:count(1), sum(tpch.custsale.c_acctbal), firstrow(custsale.cntrycode) └─Projection_38 0.00 root substring(tpch.customer.c_phone, 1, 2), tpch.customer.c_acctbal └─HashLeftJoin_39 0.00 root anti semi join, inner:TableReader_45, equal:[eq(tpch.customer.c_custkey, tpch.orders.o_custkey)] ├─Selection_40 0.00 root in(substring(tpch.customer.c_phone, 1, 2), "20", "40", "22", "30", "39", "42", "21") diff --git a/executor/merge_join_test.go b/executor/merge_join_test.go index 4c7aa2a378406..b6fa5e032ce2d 100644 --- a/executor/merge_join_test.go +++ b/executor/merge_join_test.go @@ -328,7 +328,7 @@ func (s *testSuite1) TestMergeJoin(c *C) { tk.MustExec("create table t(a int)") tk.MustExec("insert into t value(1),(2)") tk.MustQuery("explain select /*+ TIDB_SMJ(t1, t2) */ * from t t1 join t t2 order by t1.a, t2.a").Check(testkit.Rows( - "Sort_6 100000000.00 root t1.a:asc, t2.a:asc", + "Sort_6 100000000.00 root test.t1.a:asc, test.t2.a:asc", "└─MergeJoin_9 100000000.00 root inner join", " ├─TableReader_11 10000.00 root data:TableScan_10", " │ └─TableScan_10 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index f817600b4e1aa..8dca13b7215d2 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -694,12 +694,12 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { " ├─TableReader_15 10.00 root data:TableScan_14", " │ └─TableScan_14 10.00 cop table:t, range:[-inf,+inf], keep order:false", " └─StreamAgg_20 1.00 root funcs:count(1)", - " └─HashLeftJoin_21 1.00 root inner join, inner:TableReader_28, equal:[eq(s.a, t1.a)]", + " └─HashLeftJoin_21 1.00 root inner join, inner:TableReader_28, equal:[eq(test.s.a, test.t1.a)]", " ├─TableReader_25 1.00 root data:Selection_24", - " │ └─Selection_24 1.00 cop eq(s.a, test.t.a), not(isnull(s.a))", + " │ └─Selection_24 1.00 cop eq(test.s.a, test.t.a), not(isnull(test.s.a))", " │ └─TableScan_23 10.00 cop table:s, range:[-inf,+inf], keep order:false", " └─TableReader_28 1.00 root data:Selection_27", - " └─Selection_27 1.00 cop eq(t1.a, test.t.a), not(isnull(t1.a))", + " └─Selection_27 1.00 cop eq(test.t1.a, test.t.a), not(isnull(test.t1.a))", " └─TableScan_26 10.00 cop table:t1, range:[-inf,+inf], keep order:false", )) tk.MustQuery("explain select (select concat(t1.a, \",\", t1.b) from t t1 where t1.a=t.a and t1.c=t.c) from t"). @@ -709,10 +709,10 @@ func (s *testAnalyzeSuite) TestCorrelatedEstimation(c *C) { " ├─TableReader_12 10.00 root data:TableScan_11", " │ └─TableScan_11 10.00 cop table:t, range:[-inf,+inf], keep order:false", " └─MaxOneRow_13 1.00 root ", - " └─Projection_14 0.10 root concat(cast(t1.a), \",\", cast(t1.b))", + " └─Projection_14 0.10 root concat(cast(test.t1.a), \",\", cast(test.t1.b))", " └─IndexLookUp_21 0.10 root ", - " ├─IndexScan_18 1.00 cop table:t1, index:c, range: decided by [eq(t1.c, test.t.c)], keep order:false", - " └─Selection_20 0.10 cop eq(t1.a, test.t.a)", + " ├─IndexScan_18 1.00 cop table:t1, index:c, range: decided by [eq(test.t1.c, test.t.c)], keep order:false", + " └─Selection_20 0.10 cop eq(test.t1.a, test.t.a)", " └─TableScan_19 1.00 cop table:t, keep order:false", )) } @@ -915,13 +915,13 @@ func (s *testAnalyzeSuite) TestIssue9562(c *C) { tk.MustExec("create table t(a int, b int, index idx_ab(a, b))") tk.MustQuery("explain select * from t t1 join t t2 where t1.b = t2.b and t2.b is null").Check(testkit.Rows( - "Projection_7 0.00 root t1.a, t1.b, t2.a, t2.b", - "└─HashRightJoin_9 0.00 root inner join, inner:TableReader_12, equal:[eq(t2.b, t1.b)]", + "Projection_7 0.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b", + "└─HashRightJoin_9 0.00 root inner join, inner:TableReader_12, equal:[eq(test.t2.b, test.t1.b)]", " ├─TableReader_12 0.00 root data:Selection_11", - " │ └─Selection_11 0.00 cop isnull(t2.b), not(isnull(t2.b))", + " │ └─Selection_11 0.00 cop isnull(test.t2.b), not(isnull(test.t2.b))", " │ └─TableScan_10 10000.00 cop table:t2, range:[-inf,+inf], keep order:false, stats:pseudo", " └─TableReader_15 9990.00 root data:Selection_14", - " └─Selection_14 9990.00 cop not(isnull(t1.b))", + " └─Selection_14 9990.00 cop not(isnull(test.t1.b))", " └─TableScan_13 10000.00 cop table:t1, range:[-inf,+inf], keep order:false, stats:pseudo", )) } @@ -1030,13 +1030,13 @@ func (s *testAnalyzeSuite) TestLimitCrossEstimation(c *C) { // Outer plan of index join (to test that correct column ID is used). tk.MustQuery("EXPLAIN SELECT *, t1.a IN (SELECT t2.b FROM t t2) FROM t t1 WHERE t1.b <= 6 ORDER BY t1.a limit 1").Check(testkit.Rows( "Limit_17 1.00 root offset:0, count:1", - "└─IndexJoin_58 1.00 root left outer semi join, inner:IndexReader_57, outer key:t1.a, inner key:t2.b", - " ├─TopN_23 1.00 root t1.a:asc, offset:0, count:1", + "└─IndexJoin_58 1.00 root left outer semi join, inner:IndexReader_57, outer key:test.t1.a, inner key:test.t2.b", + " ├─TopN_23 1.00 root test.t1.a:asc, offset:0, count:1", " │ └─IndexReader_31 1.00 root index:TopN_30", - " │ └─TopN_30 1.00 cop t1.a:asc, offset:0, count:1", + " │ └─TopN_30 1.00 cop test.t1.a:asc, offset:0, count:1", " │ └─IndexScan_29 6.00 cop table:t1, index:b, range:[-inf,6], keep order:false", " └─IndexReader_57 1.04 root index:IndexScan_56", - " └─IndexScan_56 1.04 cop table:t2, index:b, range: decided by [eq(t2.b, t1.a)], keep order:false", + " └─IndexScan_56 1.04 cop table:t2, index:b, range: decided by [eq(test.t2.b, test.t1.a)], keep order:false", )) // Desc TableScan. tk.MustExec("truncate table t") diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index b9dcec36df240..6a8199c6ec087 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -157,7 +157,6 @@ func (b *PlanBuilder) buildResultSetNode(node ast.ResultSetNode) (p LogicalPlan, col.OrigTblName = col.TblName if x.AsName.L != "" { col.TblName = x.AsName - col.DBName = model.NewCIStr("") } } // Duplicate column name in one table is not allowed. diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 578155d266563..2af974ba048c8 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -60,7 +60,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { }{ { sql: "select count(*) from t a, t b where a.a = b.a", - best: "Join{DataScan(a)->DataScan(b)}(a.a,b.a)->Aggr(count(1))->Projection", + best: "Join{DataScan(a)->DataScan(b)}(test.a.a,test.b.a)->Aggr(count(1))->Projection", }, { sql: "select a from (select a from t where d = 0) k where k.a = 5", @@ -80,27 +80,27 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { }, { sql: "select * from t ta, t tb where (ta.d, ta.a) = (tb.b, tb.c)", - best: "Join{DataScan(ta)->DataScan(tb)}(ta.d,tb.b)(ta.a,tb.c)->Projection", + best: "Join{DataScan(ta)->DataScan(tb)}(test.ta.d,test.tb.b)(test.ta.a,test.tb.c)->Projection", }, { sql: "select * from t t1, t t2 where t1.a = t2.b and t2.b > 0 and t1.a = t1.c and t1.d like 'abc' and t2.d = t1.d", - best: "Join{DataScan(t1)->Sel([eq(cast(t1.d), cast(abc))])->DataScan(t2)->Sel([eq(cast(t2.d), cast(abc))])}(t1.a,t2.b)(t1.d,t2.d)->Projection", + best: "Join{DataScan(t1)->Sel([eq(cast(test.t1.d), cast(abc))])->DataScan(t2)->Sel([eq(cast(test.t2.d), cast(abc))])}(test.t1.a,test.t2.b)(test.t1.d,test.t2.d)->Projection", }, { sql: "select * from t ta join t tb on ta.d = tb.d and ta.d > 1 where tb.a = 0", - best: "Join{DataScan(ta)->DataScan(tb)}(ta.d,tb.d)->Projection", + best: "Join{DataScan(ta)->DataScan(tb)}(test.ta.d,test.tb.d)->Projection", }, { sql: "select * from t ta join t tb on ta.d = tb.d where ta.d > 1 and tb.a = 0", - best: "Join{DataScan(ta)->DataScan(tb)}(ta.d,tb.d)->Projection", + best: "Join{DataScan(ta)->DataScan(tb)}(test.ta.d,test.tb.d)->Projection", }, { sql: "select * from t ta left outer join t tb on ta.d = tb.d and ta.d > 1 where tb.a = 0", - best: "Join{DataScan(ta)->DataScan(tb)}(ta.d,tb.d)->Projection", + best: "Join{DataScan(ta)->DataScan(tb)}(test.ta.d,test.tb.d)->Projection", }, { sql: "select * from t ta right outer join t tb on ta.d = tb.d and ta.a > 1 where tb.a = 0", - best: "Join{DataScan(ta)->DataScan(tb)}(ta.d,tb.d)->Projection", + best: "Join{DataScan(ta)->DataScan(tb)}(test.ta.d,test.tb.d)->Projection", }, { sql: "select * from t ta left outer join t tb on ta.d = tb.d and ta.a > 1 where ta.d = 0", @@ -112,31 +112,31 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { }, { sql: "select * from t ta left outer join t tb on ta.d = tb.d and ta.a > 1 where tb.c is not null and tb.c = 0 and ifnull(tb.d, 1)", - best: "Join{DataScan(ta)->DataScan(tb)}(ta.d,tb.d)->Projection", + best: "Join{DataScan(ta)->DataScan(tb)}(test.ta.d,test.tb.d)->Projection", }, { sql: "select * from t ta left outer join t tb on ta.a = tb.a left outer join t tc on tb.b = tc.b where tc.c > 0", - best: "Join{Join{DataScan(ta)->DataScan(tb)}(ta.a,tb.a)->DataScan(tc)}(tb.b,tc.b)->Projection", + best: "Join{Join{DataScan(ta)->DataScan(tb)}(test.ta.a,test.tb.a)->DataScan(tc)}(test.tb.b,test.tc.b)->Projection", }, { sql: "select * from t ta left outer join t tb on ta.a = tb.a left outer join t tc on tc.b = ta.b where tb.c > 0", - best: "Join{Join{DataScan(ta)->DataScan(tb)}(ta.a,tb.a)->DataScan(tc)}(ta.b,tc.b)->Projection", + best: "Join{Join{DataScan(ta)->DataScan(tb)}(test.ta.a,test.tb.a)->DataScan(tc)}(test.ta.b,test.tc.b)->Projection", }, { sql: "select * from t as ta left outer join (t as tb left join t as tc on tc.b = tb.b) on tb.a = ta.a where tc.c > 0", - best: "Join{DataScan(ta)->Join{DataScan(tb)->DataScan(tc)}(tb.b,tc.b)}(ta.a,tb.a)->Projection", + best: "Join{DataScan(ta)->Join{DataScan(tb)->DataScan(tc)}(test.tb.b,test.tc.b)}(test.ta.a,test.tb.a)->Projection", }, { sql: "select * from ( t as ta left outer join t as tb on ta.a = tb.a) join ( t as tc left join t as td on tc.b = td.b) on ta.c = td.c where tb.c = 2 and td.a = 1", - best: "Join{Join{DataScan(ta)->DataScan(tb)}(ta.a,tb.a)->Join{DataScan(tc)->DataScan(td)}(tc.b,td.b)}(ta.c,td.c)->Projection", + best: "Join{Join{DataScan(ta)->DataScan(tb)}(test.ta.a,test.tb.a)->Join{DataScan(tc)->DataScan(td)}(test.tc.b,test.td.b)}(test.ta.c,test.td.c)->Projection", }, { sql: "select * from t ta left outer join (t tb left outer join t tc on tc.b = tb.b) on tb.a = ta.a and tc.c = ta.c where tc.d > 0 or ta.d > 0", - best: "Join{DataScan(ta)->Join{DataScan(tb)->DataScan(tc)}(tb.b,tc.b)}(ta.a,tb.a)(ta.c,tc.c)->Sel([or(gt(tc.d, 0), gt(ta.d, 0))])->Projection", + best: "Join{DataScan(ta)->Join{DataScan(tb)->DataScan(tc)}(test.tb.b,test.tc.b)}(test.ta.a,test.tb.a)(test.ta.c,test.tc.c)->Sel([or(gt(test.tc.d, 0), gt(test.ta.d, 0))])->Projection", }, { sql: "select * from t ta left outer join t tb on ta.d = tb.d and ta.a > 1 where ifnull(tb.d, 1) or tb.d is null", - best: "Join{DataScan(ta)->DataScan(tb)}(ta.d,tb.d)->Sel([or(ifnull(tb.d, 1), isnull(tb.d))])->Projection", + best: "Join{DataScan(ta)->DataScan(tb)}(test.ta.d,test.tb.d)->Sel([or(ifnull(test.tb.d, 1), isnull(test.tb.d))])->Projection", }, { sql: "select a, d from (select * from t union all select * from t union all select * from t) z where a < 10", @@ -145,22 +145,21 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { { sql: "select (select count(*) from t where t.a = k.a) from t k", best: "Apply{DataScan(k)->DataScan(t)->Aggr(count(1))->Projection->MaxOneRow}->Projection", - }, - { + }, { sql: "select a from t where exists(select 1 from t as x where x.a < t.a)", best: "Join{DataScan(t)->DataScan(x)}->Projection", }, { sql: "select a from t where exists(select 1 from t as x where x.a = t.a and t.a < 1 and x.a < 1)", - best: "Join{DataScan(t)->DataScan(x)}(test.t.a,x.a)->Projection", + best: "Join{DataScan(t)->DataScan(x)}(test.t.a,test.x.a)->Projection", }, { sql: "select a from t where exists(select 1 from t as x where x.a = t.a and x.a < 1) and a < 1", - best: "Join{DataScan(t)->DataScan(x)}(test.t.a,x.a)->Projection", + best: "Join{DataScan(t)->DataScan(x)}(test.t.a,test.x.a)->Projection", }, { sql: "select a from t where exists(select 1 from t as x where x.a = t.a) and exists(select 1 from t as x where x.a = t.a)", - best: "Join{Join{DataScan(t)->DataScan(x)}(test.t.a,x.a)->DataScan(x)}(test.t.a,x.a)->Projection", + best: "Join{Join{DataScan(t)->DataScan(x)}(test.t.a,test.x.a)->DataScan(x)}(test.t.a,test.x.a)->Projection", }, { sql: "select * from (select a, b, sum(c) as s from t group by a, b) k where k.a > k.b * 2 + 1", @@ -172,7 +171,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { }, { sql: "select * from (select k.a, sum(k.s) as ss from (select a, sum(b) as s from t group by a) k group by k.a) l where l.a > 2", - best: "DataScan(t)->Aggr(sum(test.t.b),firstrow(test.t.a))->Projection->Aggr(sum(k.s),firstrow(k.a))->Projection->Projection", + best: "DataScan(t)->Aggr(sum(test.t.b),firstrow(test.t.a))->Projection->Aggr(sum(k.s),firstrow(test.k.a))->Projection->Projection", }, { sql: "select * from (select a, sum(b) as s from t group by a) k where a > s", @@ -184,7 +183,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { }, { sql: "select * from (select a, sum(b) as s from t group by a having 1 = 0) k where a > 1", - best: "Dual->Sel([gt(k.a, 1)])->Projection", + best: "Dual->Sel([gt(test.k.a, 1)])->Projection", }, { sql: "select a, count(a) cnt from t group by a having cnt < 1", @@ -193,7 +192,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { // issue #3873 { sql: "select t1.a, t2.a from t as t1 left join t as t2 on t1.a = t2.a where t1.a < 1.0", - best: "Join{DataScan(t1)->DataScan(t2)}(t1.a,t2.a)->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.a,test.t2.a)->Projection", }, // issue #7728 { @@ -201,7 +200,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { best: "Dual->Projection", }, } - for _, ca := range tests { + for ith, ca := range tests { comment := Commentf("for %s", ca.sql) stmt, err := s.ParseOneStmt(ca.sql, "", "") c.Assert(err, IsNil, comment) @@ -209,7 +208,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) { c.Assert(err, IsNil) p, err = logicalOptimize(flagPredicatePushDown|flagDecorrelate|flagPrunColumns, p.(LogicalPlan)) c.Assert(err, IsNil) - c.Assert(ToString(p), Equals, ca.best, Commentf("for %s", ca.sql)) + c.Assert(ToString(p), Equals, ca.best, Commentf("for %s %d", ca.sql, ith)) } } @@ -233,34 +232,34 @@ func (s *testPlanSuite) TestJoinPredicatePushDown(c *C) { }, { sql: "select * from t as t1 join t as t2 on t1.b = t2.b where (t1.a=1 and t2.a=1) or (t1.a=2 and t2.a=2)", - left: "[or(eq(t1.a, 1), eq(t1.a, 2))]", - right: "[or(eq(t2.a, 1), eq(t2.a, 2))]", + left: "[or(eq(test.t1.a, 1), eq(test.t1.a, 2))]", + right: "[or(eq(test.t2.a, 1), eq(test.t2.a, 2))]", }, { sql: "select * from t as t1 join t as t2 on t1.b = t2.b where (t1.c=1 and (t1.a=3 or t2.a=3)) or (t1.a=2 and t2.a=2)", - left: "[or(eq(t1.c, 1), eq(t1.a, 2))]", + left: "[or(eq(test.t1.c, 1), eq(test.t1.a, 2))]", right: "[]", }, { sql: "select * from t as t1 join t as t2 on t1.b = t2.b where (t1.c=1 and ((t1.a=3 and t2.a=3) or (t1.a=4 and t2.a=4)))", - left: "[eq(t1.c, 1) or(eq(t1.a, 3), eq(t1.a, 4))]", - right: "[or(eq(t2.a, 3), eq(t2.a, 4))]", + left: "[eq(test.t1.c, 1) or(eq(test.t1.a, 3), eq(test.t1.a, 4))]", + right: "[or(eq(test.t2.a, 3), eq(test.t2.a, 4))]", }, { sql: "select * from t as t1 join t as t2 on t1.b = t2.b where (t1.a>1 and t1.a < 3 and t2.a=1) or (t1.a=2 and t2.a=2)", - left: "[or(and(gt(t1.a, 1), lt(t1.a, 3)), eq(t1.a, 2))]", - right: "[or(eq(t2.a, 1), eq(t2.a, 2))]", + left: "[or(and(gt(test.t1.a, 1), lt(test.t1.a, 3)), eq(test.t1.a, 2))]", + right: "[or(eq(test.t2.a, 1), eq(test.t2.a, 2))]", }, { sql: "select * from t as t1 join t as t2 on t1.b = t2.b and ((t1.a=1 and t2.a=1) or (t1.a=2 and t2.a=2))", - left: "[or(eq(t1.a, 1), eq(t1.a, 2))]", - right: "[or(eq(t2.a, 1), eq(t2.a, 2))]", + left: "[or(eq(test.t1.a, 1), eq(test.t1.a, 2))]", + right: "[or(eq(test.t2.a, 1), eq(test.t2.a, 2))]", }, // issue #7628, left join { sql: "select * from t as t1 left join t as t2 on t1.b = t2.b and ((t1.a=1 and t2.a=1) or (t1.a=2 and t2.a=2))", left: "[]", - right: "[or(eq(t2.a, 1), eq(t2.a, 2))]", + right: "[or(eq(test.t2.a, 1), eq(test.t2.a, 2))]", }, { sql: "select * from t as t1 left join t as t2 on t1.b = t2.b and t1.a > t2.a", @@ -280,17 +279,17 @@ func (s *testPlanSuite) TestJoinPredicatePushDown(c *C) { { sql: "select * from t as t1 left join t as t2 on t1.b = t2.b and ((t2.c=1 and (t1.a=3 or t2.a=3)) or (t1.a=2 and t2.a=2))", left: "[]", - right: "[or(eq(t2.c, 1), eq(t2.a, 2))]", + right: "[or(eq(test.t2.c, 1), eq(test.t2.a, 2))]", }, { sql: "select * from t as t1 left join t as t2 on t1.b = t2.b and ((t1.c=1 and ((t1.a=3 and t2.a=3) or (t1.a=4 and t2.a=4))) or (t1.a=2 and t2.a=2))", left: "[]", - right: "[or(or(eq(t2.a, 3), eq(t2.a, 4)), eq(t2.a, 2))]", + right: "[or(or(eq(test.t2.a, 3), eq(test.t2.a, 4)), eq(test.t2.a, 2))]", }, // Duplicate condition would be removed. { sql: "select * from t t1 join t t2 on t1.a > 1 and t1.a > 1", - left: "[gt(t1.a, 1)]", + left: "[gt(test.t1.a, 1)]", right: "[]", }, } @@ -328,20 +327,20 @@ func (s *testPlanSuite) TestOuterWherePredicatePushDown(c *C) { // issue #7628, left join with where condition { sql: "select * from t as t1 left join t as t2 on t1.b = t2.b where (t1.a=1 and t2.a is null) or (t1.a=2 and t2.a=2)", - sel: "[or(and(eq(t1.a, 1), isnull(t2.a)), and(eq(t1.a, 2), eq(t2.a, 2)))]", - left: "[or(eq(t1.a, 1), eq(t1.a, 2))]", + sel: "[or(and(eq(test.t1.a, 1), isnull(test.t2.a)), and(eq(test.t1.a, 2), eq(test.t2.a, 2)))]", + left: "[or(eq(test.t1.a, 1), eq(test.t1.a, 2))]", right: "[]", }, { sql: "select * from t as t1 left join t as t2 on t1.b = t2.b where (t1.c=1 and (t1.a=3 or t2.a=3)) or (t1.a=2 and t2.a=2)", - sel: "[or(and(eq(t1.c, 1), or(eq(t1.a, 3), eq(t2.a, 3))), and(eq(t1.a, 2), eq(t2.a, 2)))]", - left: "[or(eq(t1.c, 1), eq(t1.a, 2))]", + sel: "[or(and(eq(test.t1.c, 1), or(eq(test.t1.a, 3), eq(test.t2.a, 3))), and(eq(test.t1.a, 2), eq(test.t2.a, 2)))]", + left: "[or(eq(test.t1.c, 1), eq(test.t1.a, 2))]", right: "[]", }, { sql: "select * from t as t1 left join t as t2 on t1.b = t2.b where (t1.c=1 and ((t1.a=3 and t2.a=3) or (t1.a=4 and t2.a=4))) or (t1.a=2 and t2.a is null)", - sel: "[or(and(eq(t1.c, 1), or(and(eq(t1.a, 3), eq(t2.a, 3)), and(eq(t1.a, 4), eq(t2.a, 4)))), and(eq(t1.a, 2), isnull(t2.a)))]", - left: "[or(and(eq(t1.c, 1), or(eq(t1.a, 3), eq(t1.a, 4))), eq(t1.a, 2))]", + sel: "[or(and(eq(test.t1.c, 1), or(and(eq(test.t1.a, 3), eq(test.t2.a, 3)), and(eq(test.t1.a, 4), eq(test.t2.a, 4)))), and(eq(test.t1.a, 2), isnull(test.t2.a)))]", + left: "[or(and(eq(test.t1.c, 1), or(eq(test.t1.a, 3), eq(test.t1.a, 4))), eq(test.t1.a, 2))]", right: "[]", }, } @@ -381,32 +380,32 @@ func (s *testPlanSuite) TestSimplifyOuterJoin(c *C) { }{ { sql: "select * from t t1 left join t t2 on t1.b = t2.b where t1.c > 1 or t2.c > 1;", - best: "Join{DataScan(t1)->DataScan(t2)}(t1.b,t2.b)->Sel([or(gt(t1.c, 1), gt(t2.c, 1))])->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.b,test.t2.b)->Sel([or(gt(test.t1.c, 1), gt(test.t2.c, 1))])->Projection", joinType: "left outer join", }, { sql: "select * from t t1 left join t t2 on t1.b = t2.b where t1.c > 1 and t2.c > 1;", - best: "Join{DataScan(t1)->DataScan(t2)}(t1.b,t2.b)->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.b,test.t2.b)->Projection", joinType: "inner join", }, { sql: "select * from t t1 left join t t2 on t1.b = t2.b where not (t1.c > 1 or t2.c > 1);", - best: "Join{DataScan(t1)->DataScan(t2)}(t1.b,t2.b)->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.b,test.t2.b)->Projection", joinType: "inner join", }, { sql: "select * from t t1 left join t t2 on t1.b = t2.b where not (t1.c > 1 and t2.c > 1);", - best: "Join{DataScan(t1)->DataScan(t2)}(t1.b,t2.b)->Sel([not(and(le(t1.c, 1), le(t2.c, 1)))])->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.b,test.t2.b)->Sel([not(and(le(test.t1.c, 1), le(test.t2.c, 1)))])->Projection", joinType: "left outer join", }, { sql: "select * from t t1 left join t t2 on t1.b > 1 where t1.c = t2.c;", - best: "Join{DataScan(t1)->DataScan(t2)}(t1.c,t2.c)->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.c,test.t2.c)->Projection", joinType: "inner join", }, { sql: "select * from t t1 left join t t2 on true where t1.b <=> t2.b;", - best: "Join{DataScan(t1)->DataScan(t2)}->Sel([nulleq(t1.b, t2.b)])->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}->Sel([nulleq(test.t1.b, test.t2.b)])->Projection", joinType: "left outer join", }, } @@ -465,44 +464,44 @@ func (s *testPlanSuite) TestDeriveNotNullConds(c *C) { }{ { sql: "select * from t t1 inner join t t2 on t1.e = t2.e", - plan: "Join{DataScan(t1)->DataScan(t2)}(t1.e,t2.e)->Projection", - left: "[not(isnull(t1.e))]", - right: "[not(isnull(t2.e))]", + plan: "Join{DataScan(t1)->DataScan(t2)}(test.t1.e,test.t2.e)->Projection", + left: "[not(isnull(test.t1.e))]", + right: "[not(isnull(test.t2.e))]", }, { sql: "select * from t t1 inner join t t2 on t1.e > t2.e", plan: "Join{DataScan(t1)->DataScan(t2)}->Projection", - left: "[not(isnull(t1.e))]", - right: "[not(isnull(t2.e))]", + left: "[not(isnull(test.t1.e))]", + right: "[not(isnull(test.t2.e))]", }, { sql: "select * from t t1 inner join t t2 on t1.e = t2.e and t1.e is not null", - plan: "Join{DataScan(t1)->DataScan(t2)}(t1.e,t2.e)->Projection", - left: "[not(isnull(t1.e))]", - right: "[not(isnull(t2.e))]", + plan: "Join{DataScan(t1)->DataScan(t2)}(test.t1.e,test.t2.e)->Projection", + left: "[not(isnull(test.t1.e))]", + right: "[not(isnull(test.t2.e))]", }, { sql: "select * from t t1 left join t t2 on t1.e = t2.e", - plan: "Join{DataScan(t1)->DataScan(t2)}(t1.e,t2.e)->Projection", + plan: "Join{DataScan(t1)->DataScan(t2)}(test.t1.e,test.t2.e)->Projection", left: "[]", - right: "[not(isnull(t2.e))]", + right: "[not(isnull(test.t2.e))]", }, { sql: "select * from t t1 left join t t2 on t1.e > t2.e", plan: "Join{DataScan(t1)->DataScan(t2)}->Projection", left: "[]", - right: "[not(isnull(t2.e))]", + right: "[not(isnull(test.t2.e))]", }, { sql: "select * from t t1 left join t t2 on t1.e = t2.e and t2.e is not null", - plan: "Join{DataScan(t1)->DataScan(t2)}(t1.e,t2.e)->Projection", + plan: "Join{DataScan(t1)->DataScan(t2)}(test.t1.e,test.t2.e)->Projection", left: "[]", - right: "[not(isnull(t2.e))]", + right: "[not(isnull(test.t2.e))]", }, { sql: "select * from t t1 right join t t2 on t1.e = t2.e and t1.e is not null", - plan: "Join{DataScan(t1)->DataScan(t2)}(t1.e,t2.e)->Projection", - left: "[not(isnull(t1.e))]", + plan: "Join{DataScan(t1)->DataScan(t2)}(test.t1.e,test.t2.e)->Projection", + left: "[not(isnull(test.t1.e))]", right: "[]", }, { @@ -520,13 +519,13 @@ func (s *testPlanSuite) TestDeriveNotNullConds(c *C) { // Not deriving if column has NotNull flag already. { sql: "select * from t t1 inner join t t2 on t1.b = t2.b", - plan: "Join{DataScan(t1)->DataScan(t2)}(t1.b,t2.b)->Projection", + plan: "Join{DataScan(t1)->DataScan(t2)}(test.t1.b,test.t2.b)->Projection", left: "[]", right: "[]", }, { sql: "select * from t t1 left join t t2 on t1.b = t2.b", - plan: "Join{DataScan(t1)->DataScan(t2)}(t1.b,t2.b)->Projection", + plan: "Join{DataScan(t1)->DataScan(t2)}(test.t1.b,test.t2.b)->Projection", left: "[]", right: "[]", }, @@ -539,7 +538,7 @@ func (s *testPlanSuite) TestDeriveNotNullConds(c *C) { // Not deriving for AntiSemiJoin { sql: "select * from t t1 where not exists (select * from t t2 where t2.e = t1.e)", - plan: "Join{DataScan(t1)->DataScan(t2)}(t1.e,t2.e)->Projection", + plan: "Join{DataScan(t1)->DataScan(t2)}(test.t1.e,test.t2.e)->Projection", left: "[]", right: "[]", }, @@ -580,7 +579,7 @@ func (s *testPlanSuite) TestDupRandJoinCondsPushDown(c *C) { c.Assert(ok, IsTrue, comment) leftCond := fmt.Sprintf("%s", leftPlan.Conditions) // Condition with mutable function cannot be de-duplicated when push down join conds. - c.Assert(leftCond, Equals, "[gt(cast(t1.a), rand()) gt(cast(t1.a), rand())]", comment) + c.Assert(leftCond, Equals, "[gt(cast(test.t1.a), rand()) gt(cast(test.t1.a), rand())]", comment) } func (s *testPlanSuite) TestTablePartition(c *C) { @@ -692,72 +691,72 @@ func (s *testPlanSuite) TestSubquery(c *C) { { // This will be resolved as in sub query. sql: "select * from t where 10 in (select b from t s where s.a = t.a)", - best: "Join{DataScan(t)->DataScan(s)}(test.t.a,s.a)->Projection", + best: "Join{DataScan(t)->DataScan(s)}(test.t.a,test.s.a)->Projection", }, { sql: "select count(c) ,(select b from t s where s.a = t.a) from t", - best: "Join{DataScan(t)->Aggr(count(test.t.c),firstrow(test.t.a))->DataScan(s)}(test.t.a,s.a)->Projection->Projection", + best: "Join{DataScan(t)->Aggr(count(test.t.c),firstrow(test.t.a))->DataScan(s)}(test.t.a,test.s.a)->Projection->Projection", }, { sql: "select count(c) ,(select count(s.b) from t s where s.a = t.a) from t", - best: "Join{DataScan(t)->Aggr(count(test.t.c),firstrow(test.t.a))->DataScan(s)}(test.t.a,s.a)->Aggr(firstrow(2_col_0),firstrow(test.t.a),count(s.b))->Projection->Projection", + best: "Join{DataScan(t)->Aggr(count(test.t.c),firstrow(test.t.a))->DataScan(s)}(test.t.a,test.s.a)->Aggr(firstrow(2_col_0),firstrow(test.t.a),count(test.s.b))->Projection->Projection", }, { // Semi-join with agg cannot decorrelate. sql: "select t.c in (select count(s.b) from t s where s.a = t.a) from t", - best: "Apply{DataScan(t)->DataScan(s)->Sel([eq(s.a, test.t.a)])->Aggr(count(s.b))}->Projection", + best: "Apply{DataScan(t)->DataScan(s)->Sel([eq(test.s.a, test.t.a)])->Aggr(count(test.s.b))}->Projection", }, { sql: "select (select count(s.b) k from t s where s.a = t.a having k != 0) from t", - best: "Join{DataScan(t)->DataScan(s)->Aggr(count(s.b),firstrow(s.a))}(test.t.a,s.a)->Projection->Projection->Projection", + best: "Join{DataScan(t)->DataScan(s)->Aggr(count(test.s.b),firstrow(test.s.a))}(test.t.a,test.s.a)->Projection->Projection->Projection", }, { sql: "select (select count(s.b) k from t s where s.a = t1.a) from t t1, t t2", - best: "Join{Join{DataScan(t1)->DataScan(t2)}->DataScan(s)->Aggr(count(s.b),firstrow(s.a))}(t1.a,s.a)->Projection->Projection->Projection", + best: "Join{Join{DataScan(t1)->DataScan(t2)}->DataScan(s)->Aggr(count(test.s.b),firstrow(test.s.a))}(test.t1.a,test.s.a)->Projection->Projection->Projection", }, { sql: "select (select count(1) k from t s where s.a = t.a having k != 0) from t", - best: "Join{DataScan(t)->DataScan(s)->Aggr(count(1),firstrow(s.a))}(test.t.a,s.a)->Projection->Projection->Projection", + best: "Join{DataScan(t)->DataScan(s)->Aggr(count(1),firstrow(test.s.a))}(test.t.a,test.s.a)->Projection->Projection->Projection", }, { sql: "select a from t where a in (select a from t s group by t.b)", - best: "Join{DataScan(t)->DataScan(s)->Aggr(firstrow(s.a))->Projection}(test.t.a,s.a)->Projection", + best: "Join{DataScan(t)->DataScan(s)->Aggr(firstrow(test.s.a))->Projection}(test.t.a,test.s.a)->Projection", }, { // This will be resolved as in sub query. sql: "select * from t where 10 in (((select b from t s where s.a = t.a)))", - best: "Join{DataScan(t)->DataScan(s)}(test.t.a,s.a)->Projection", + best: "Join{DataScan(t)->DataScan(s)}(test.t.a,test.s.a)->Projection", }, { // This will be resolved as in function. sql: "select * from t where 10 in (((select b from t s where s.a = t.a)), 10)", - best: "Join{DataScan(t)->DataScan(s)}(test.t.a,s.a)->Projection->Sel([in(10, s.b, 10)])->Projection", + best: "Join{DataScan(t)->DataScan(s)}(test.t.a,test.s.a)->Projection->Sel([in(10, test.s.b, 10)])->Projection", }, { sql: "select * from t where exists (select s.a from t s having sum(s.a) = t.a )", - best: "Join{DataScan(t)->DataScan(s)->Aggr(sum(s.a))->Projection}->Projection", + best: "Join{DataScan(t)->DataScan(s)->Aggr(sum(test.s.a))->Projection}->Projection", }, { // Test MaxOneRow for limit. sql: "select (select * from (select b from t limit 1) x where x.b = t1.b) from t t1", - best: "Join{DataScan(t1)->DataScan(t)->Projection->Limit}(t1.b,x.b)->Projection->Projection", + best: "Join{DataScan(t1)->DataScan(t)->Projection->Limit}(test.t1.b,test.x.b)->Projection->Projection", }, { // Test Nested sub query. sql: "select * from t where exists (select s.a from t s where s.c in (select c from t as k where k.d = s.d) having sum(s.a) = t.a )", - best: "Join{DataScan(t)->Join{DataScan(s)->DataScan(k)}(s.d,k.d)(s.c,k.c)->Aggr(sum(s.a))->Projection}->Projection", + best: "Join{DataScan(t)->Join{DataScan(s)->DataScan(k)}(test.s.d,test.k.d)(test.s.c,test.k.c)->Aggr(sum(test.s.a))->Projection}->Projection", }, { sql: "select t1.b from t t1 where t1.b = (select max(t2.a) from t t2 where t1.b=t2.b)", - best: "Join{DataScan(t1)->DataScan(t2)->Aggr(max(t2.a),firstrow(t2.b))}(t1.b,t2.b)->Projection->Sel([eq(t1.b, max(t2.a))])->Projection", + best: "Join{DataScan(t1)->DataScan(t2)->Aggr(max(test.t2.a),firstrow(test.t2.b))}(test.t1.b,test.t2.b)->Projection->Sel([eq(test.t1.b, max(t2.a))])->Projection", }, { sql: "select t1.b from t t1 where t1.b = (select avg(t2.a) from t t2 where t1.g=t2.g and (t1.b = 4 or t2.b = 2))", - best: "Apply{DataScan(t1)->DataScan(t2)->Sel([eq(t1.g, t2.g) or(eq(t1.b, 4), eq(t2.b, 2))])->Aggr(avg(t2.a))}->Projection->Sel([eq(cast(t1.b), avg(t2.a))])->Projection", + best: "Apply{DataScan(t1)->DataScan(t2)->Sel([eq(test.t1.g, test.t2.g) or(eq(test.t1.b, 4), eq(test.t2.b, 2))])->Aggr(avg(test.t2.a))}->Projection->Sel([eq(cast(test.t1.b), avg(t2.a))])->Projection", }, } - for _, ca := range tests { + for ith, ca := range tests { comment := Commentf("for %s", ca.sql) stmt, err := s.ParseOneStmt(ca.sql, "", "") c.Assert(err, IsNil, comment) @@ -769,7 +768,7 @@ func (s *testPlanSuite) TestSubquery(c *C) { p, err = logicalOptimize(flagBuildKeyInfo|flagDecorrelate|flagPrunColumns, lp) c.Assert(err, IsNil) } - c.Assert(ToString(p), Equals, ca.best, Commentf("for %s", ca.sql)) + c.Assert(ToString(p), Equals, ca.best, Commentf("for %s %d", ca.sql, ith)) } } @@ -886,15 +885,15 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { }{ { sql: "select * from t t1, t t2, t t3, t t4, t t5, t t6 where t1.a = t2.b and t2.a = t3.b and t3.c = t4.a and t4.d = t2.c and t5.d = t6.d", - best: "Join{Join{Join{Join{DataScan(t1)->DataScan(t2)}(t1.a,t2.b)->DataScan(t3)}(t2.a,t3.b)->DataScan(t4)}(t3.c,t4.a)(t2.c,t4.d)->Join{DataScan(t5)->DataScan(t6)}(t5.d,t6.d)}->Projection", + best: "Join{Join{Join{Join{DataScan(t1)->DataScan(t2)}(test.t1.a,test.t2.b)->DataScan(t3)}(test.t2.a,test.t3.b)->DataScan(t4)}(test.t3.c,test.t4.a)(test.t2.c,test.t4.d)->Join{DataScan(t5)->DataScan(t6)}(test.t5.d,test.t6.d)}->Projection", }, { sql: "select * from t t1, t t2, t t3, t t4, t t5, t t6, t t7, t t8 where t1.a = t8.a", - best: "Join{Join{Join{Join{DataScan(t1)->DataScan(t8)}(t1.a,t8.a)->DataScan(t2)}->Join{DataScan(t3)->DataScan(t4)}}->Join{Join{DataScan(t5)->DataScan(t6)}->DataScan(t7)}}->Projection", + best: "Join{Join{Join{Join{DataScan(t1)->DataScan(t8)}(test.t1.a,test.t8.a)->DataScan(t2)}->Join{DataScan(t3)->DataScan(t4)}}->Join{Join{DataScan(t5)->DataScan(t6)}->DataScan(t7)}}->Projection", }, { sql: "select * from t t1, t t2, t t3, t t4, t t5 where t1.a = t5.a and t5.a = t4.a and t4.a = t3.a and t3.a = t2.a and t2.a = t1.a and t1.a = t3.a and t2.a = t4.a and t5.b < 8", - best: "Join{Join{Join{Join{DataScan(t5)->DataScan(t1)}(t5.a,t1.a)->DataScan(t2)}(t1.a,t2.a)->DataScan(t3)}(t2.a,t3.a)(t1.a,t3.a)->DataScan(t4)}(t5.a,t4.a)(t3.a,t4.a)(t2.a,t4.a)->Projection", + best: "Join{Join{Join{Join{DataScan(t5)->DataScan(t1)}(test.t5.a,test.t1.a)->DataScan(t2)}(test.t1.a,test.t2.a)->DataScan(t3)}(test.t2.a,test.t3.a)(test.t1.a,test.t3.a)->DataScan(t4)}(test.t5.a,test.t4.a)(test.t3.a,test.t4.a)(test.t2.a,test.t4.a)->Projection", }, { sql: "select * from t t1, t t2, t t3, t t4, t t5 where t1.a = t5.a and t5.a = t4.a and t4.a = t3.a and t3.a = t2.a and t2.a = t1.a and t1.a = t3.a and t2.a = t4.a and t3.b = 1 and t4.a = 1", @@ -902,7 +901,7 @@ func (s *testPlanSuite) TestJoinReOrder(c *C) { }, { sql: "select * from t o where o.b in (select t3.c from t t1, t t2, t t3 where t1.a = t3.a and t2.a = t3.a and t2.a = o.a)", - best: "Apply{DataScan(o)->Join{Join{DataScan(t1)->DataScan(t3)}(t1.a,t3.a)->DataScan(t2)}(t3.a,t2.a)->Projection}->Projection", + best: "Apply{DataScan(o)->Join{Join{DataScan(t1)->DataScan(t3)}(test.t1.a,test.t3.a)->DataScan(t2)}(test.t3.a,test.t2.a)->Projection}->Projection", }, { sql: "select * from t o where o.b in (select t3.c from t t1, t t2, t t3 where t1.a = t3.a and t2.a = t3.a and t2.a = o.a and t1.a = 1)", @@ -938,47 +937,47 @@ func (s *testPlanSuite) TestEagerAggregation(c *C) { }, { sql: "select sum(a.a) from t a, t b where a.c = b.c", - best: "Join{DataScan(a)->Aggr(sum(a.a),firstrow(a.c))->DataScan(b)}(a.c,b.c)->Aggr(sum(join_agg_0))->Projection", + best: "Join{DataScan(a)->Aggr(sum(test.a.a),firstrow(test.a.c))->DataScan(b)}(test.a.c,test.b.c)->Aggr(sum(join_agg_0))->Projection", }, { sql: "select sum(b.a) from t a, t b where a.c = b.c", - best: "Join{DataScan(a)->DataScan(b)->Aggr(sum(b.a),firstrow(b.c))}(a.c,b.c)->Aggr(sum(join_agg_0))->Projection", + best: "Join{DataScan(a)->DataScan(b)->Aggr(sum(test.b.a),firstrow(test.b.c))}(test.a.c,test.b.c)->Aggr(sum(join_agg_0))->Projection", }, { sql: "select sum(b.a), a.a from t a, t b where a.c = b.c", - best: "Join{DataScan(a)->DataScan(b)->Aggr(sum(b.a),firstrow(b.c))}(a.c,b.c)->Aggr(sum(join_agg_0),firstrow(a.a))->Projection", + best: "Join{DataScan(a)->DataScan(b)->Aggr(sum(test.b.a),firstrow(test.b.c))}(test.a.c,test.b.c)->Aggr(sum(join_agg_0),firstrow(test.a.a))->Projection", }, { sql: "select sum(a.a), b.a from t a, t b where a.c = b.c", - best: "Join{DataScan(a)->Aggr(sum(a.a),firstrow(a.c))->DataScan(b)}(a.c,b.c)->Aggr(sum(join_agg_0),firstrow(b.a))->Projection", + best: "Join{DataScan(a)->Aggr(sum(test.a.a),firstrow(test.a.c))->DataScan(b)}(test.a.c,test.b.c)->Aggr(sum(join_agg_0),firstrow(test.b.a))->Projection", }, { sql: "select sum(a.a), sum(b.a) from t a, t b where a.c = b.c", - best: "Join{DataScan(a)->DataScan(b)}(a.c,b.c)->Aggr(sum(a.a),sum(b.a))->Projection", + best: "Join{DataScan(a)->DataScan(b)}(test.a.c,test.b.c)->Aggr(sum(test.a.a),sum(test.b.a))->Projection", }, { sql: "select sum(a.a), max(b.a) from t a, t b where a.c = b.c", - best: "Join{DataScan(a)->Aggr(sum(a.a),firstrow(a.c))->DataScan(b)}(a.c,b.c)->Aggr(sum(join_agg_0),max(b.a))->Projection", + best: "Join{DataScan(a)->Aggr(sum(test.a.a),firstrow(test.a.c))->DataScan(b)}(test.a.c,test.b.c)->Aggr(sum(join_agg_0),max(test.b.a))->Projection", }, { sql: "select max(a.a), sum(b.a) from t a, t b where a.c = b.c", - best: "Join{DataScan(a)->DataScan(b)->Aggr(sum(b.a),firstrow(b.c))}(a.c,b.c)->Aggr(max(a.a),sum(join_agg_0))->Projection", + best: "Join{DataScan(a)->DataScan(b)->Aggr(sum(test.b.a),firstrow(test.b.c))}(test.a.c,test.b.c)->Aggr(max(test.a.a),sum(join_agg_0))->Projection", }, { sql: "select sum(a.a) from t a, t b, t c where a.c = b.c and b.c = c.c", - best: "Join{Join{DataScan(a)->DataScan(b)}(a.c,b.c)->DataScan(c)}(b.c,c.c)->Aggr(sum(a.a))->Projection", + best: "Join{Join{DataScan(a)->DataScan(b)}(test.a.c,test.b.c)->DataScan(c)}(test.b.c,test.c.c)->Aggr(sum(test.a.a))->Projection", }, { sql: "select sum(b.a) from t a left join t b on a.c = b.c", - best: "Join{DataScan(a)->DataScan(b)->Aggr(sum(b.a),firstrow(b.c))}(a.c,b.c)->Aggr(sum(join_agg_0))->Projection", + best: "Join{DataScan(a)->DataScan(b)->Aggr(sum(test.b.a),firstrow(test.b.c))}(test.a.c,test.b.c)->Aggr(sum(join_agg_0))->Projection", }, { sql: "select sum(a.a) from t a left join t b on a.c = b.c", - best: "Join{DataScan(a)->Aggr(sum(a.a),firstrow(a.c))->DataScan(b)}(a.c,b.c)->Aggr(sum(join_agg_0))->Projection", + best: "Join{DataScan(a)->Aggr(sum(test.a.a),firstrow(test.a.c))->DataScan(b)}(test.a.c,test.b.c)->Aggr(sum(join_agg_0))->Projection", }, { sql: "select sum(a.a) from t a right join t b on a.c = b.c", - best: "Join{DataScan(a)->Aggr(sum(a.a),firstrow(a.c))->DataScan(b)}(a.c,b.c)->Aggr(sum(join_agg_0))->Projection", + best: "Join{DataScan(a)->Aggr(sum(test.a.a),firstrow(test.a.c))->DataScan(b)}(test.a.c,test.b.c)->Aggr(sum(join_agg_0))->Projection", }, { sql: "select sum(a) from (select * from t) x", @@ -986,15 +985,15 @@ func (s *testPlanSuite) TestEagerAggregation(c *C) { }, { 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", - best: "UnionAll{DataScan(a)->Projection->Aggr(sum(a.c1),firstrow(a.c2))->DataScan(b)->Projection->Aggr(sum(b.c1),firstrow(b.c2))->DataScan(c)->Projection->Aggr(sum(c.c1),firstrow(c.c2))}->Aggr(sum(join_agg_0))->Projection", + best: "UnionAll{DataScan(a)->Projection->Aggr(sum(test.a.c1),firstrow(test.a.c2))->DataScan(b)->Projection->Aggr(sum(test.b.c1),firstrow(test.b.c2))->DataScan(c)->Projection->Aggr(sum(test.c.c1),firstrow(test.c.c2))}->Aggr(sum(join_agg_0))->Projection", }, { sql: "select max(a.b), max(b.b) from t a join t b on a.c = b.c group by a.a", - best: "Join{DataScan(a)->DataScan(b)->Aggr(max(b.b),firstrow(b.c))}(a.c,b.c)->Projection->Projection", + best: "Join{DataScan(a)->DataScan(b)->Aggr(max(test.b.b),firstrow(test.b.c))}(test.a.c,test.b.c)->Projection->Projection", }, { sql: "select max(a.b), max(b.b) from t a join t b on a.a = b.a group by a.c", - best: "Join{DataScan(a)->DataScan(b)}(a.a,b.a)->Aggr(max(a.b),max(b.b))->Projection", + best: "Join{DataScan(a)->DataScan(b)}(test.a.a,test.b.a)->Aggr(max(test.a.b),max(test.b.b))->Projection", }, { sql: "select max(c.b) from (select * from t a union all select * from t b) c group by c.a", @@ -1002,15 +1001,15 @@ func (s *testPlanSuite) TestEagerAggregation(c *C) { }, { 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", - best: "Join{DataScan(a)->DataScan(b)}(a.a,b.a)(a.b,b.b)->Aggr(max(a.c))->Projection", + best: "Join{DataScan(a)->DataScan(b)}(test.a.a,test.b.a)(test.a.b,test.b.b)->Aggr(max(test.a.c))->Projection", }, { sql: "select t1.a, count(t2.b) from t t1, t t2 where t1.a = t2.a group by t1.a", - best: "Join{DataScan(t1)->DataScan(t2)}(t1.a,t2.a)->Projection->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.a,test.t2.a)->Projection->Projection", }, } s.ctx.GetSessionVars().AllowAggPushDown = true - for _, tt := range tests { + for ith, tt := range tests { comment := Commentf("for %s", tt.sql) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) @@ -1019,7 +1018,7 @@ func (s *testPlanSuite) TestEagerAggregation(c *C) { c.Assert(err, IsNil) p, err = logicalOptimize(flagBuildKeyInfo|flagPredicatePushDown|flagPrunColumns|flagPushDownAgg, p.(LogicalPlan)) c.Assert(err, IsNil) - c.Assert(ToString(p), Equals, tt.best, Commentf("for %s", tt.sql)) + c.Assert(ToString(p), Equals, tt.best, Commentf("for %s %d", tt.sql, ith)) } s.ctx.GetSessionVars().AllowAggPushDown = false } @@ -1395,10 +1394,10 @@ func (s *testPlanSuite) TestUniqueKeyInfo(c *C) { { sql: "select * from t t1 join t t2 on t1.a = t2.e", ans: map[int][][]string{ - 1: {{"t1.f"}, {"t1.f", "t1.g"}, {"t1.a"}}, - 2: {{"t2.f"}, {"t2.f", "t2.g"}, {"t2.a"}}, - 3: {{"t2.f"}, {"t2.f", "t2.g"}, {"t2.a"}}, - 4: {{"t2.f"}, {"t2.f", "t2.g"}, {"t2.a"}}, + 1: {{"test.t1.f"}, {"test.t1.f", "test.t1.g"}, {"test.t1.a"}}, + 2: {{"test.t2.f"}, {"test.t2.f", "test.t2.g"}, {"test.t2.a"}}, + 3: {{"test.t2.f"}, {"test.t2.f", "test.t2.g"}, {"test.t2.a"}}, + 4: {{"test.t2.f"}, {"test.t2.f", "test.t2.g"}, {"test.t2.a"}}, }, }, { @@ -1414,15 +1413,15 @@ func (s *testPlanSuite) TestUniqueKeyInfo(c *C) { { sql: "select * from t t1 left join t t2 on t1.a = t2.a", ans: map[int][][]string{ - 1: {{"t1.f"}, {"t1.f", "t1.g"}, {"t1.a"}}, - 2: {{"t2.f"}, {"t2.f", "t2.g"}, {"t2.a"}}, - 3: {{"t1.f"}, {"t1.f", "t1.g"}, {"t1.a"}}, - 4: {{"t1.f"}, {"t1.f", "t1.g"}, {"t1.a"}}, + 1: {{"test.t1.f"}, {"test.t1.f", "test.t1.g"}, {"test.t1.a"}}, + 2: {{"test.t2.f"}, {"test.t2.f", "test.t2.g"}, {"test.t2.a"}}, + 3: {{"test.t1.f"}, {"test.t1.f", "test.t1.g"}, {"test.t1.a"}}, + 4: {{"test.t1.f"}, {"test.t1.f", "test.t1.g"}, {"test.t1.a"}}, }, }, } - for _, tt := range tests { - comment := Commentf("for %s", tt.sql) + for ith, tt := range tests { + comment := Commentf("for %s %d", tt.sql, ith) stmt, err := s.ParseOneStmt(tt.sql, "", "") c.Assert(err, IsNil, comment) @@ -1855,72 +1854,72 @@ func (s *testPlanSuite) TestTopNPushDown(c *C) { // Test TopN + Left Join + Proj. { sql: "select * from t left outer join t s on t.a = s.a order by t.a limit 5", - best: "Join{DataScan(t)->TopN([test.t.a],0,5)->DataScan(s)}(test.t.a,s.a)->TopN([test.t.a],0,5)->Projection", + best: "Join{DataScan(t)->TopN([test.t.a],0,5)->DataScan(s)}(test.t.a,test.s.a)->TopN([test.t.a],0,5)->Projection", }, // Test TopN + Left Join + Proj. { sql: "select * from t left outer join t s on t.a = s.a order by t.a limit 5, 5", - best: "Join{DataScan(t)->TopN([test.t.a],0,10)->DataScan(s)}(test.t.a,s.a)->TopN([test.t.a],5,5)->Projection", + best: "Join{DataScan(t)->TopN([test.t.a],0,10)->DataScan(s)}(test.t.a,test.s.a)->TopN([test.t.a],5,5)->Projection", }, // Test Limit + Left Join + Proj. { sql: "select * from t left outer join t s on t.a = s.a limit 5", - best: "Join{DataScan(t)->Limit->DataScan(s)}(test.t.a,s.a)->Limit->Projection", + best: "Join{DataScan(t)->Limit->DataScan(s)}(test.t.a,test.s.a)->Limit->Projection", }, // Test Limit + Left Join Apply + Proj. { sql: "select (select s.a from t s where t.a = s.a) from t limit 5", - best: "Join{DataScan(t)->Limit->DataScan(s)}(test.t.a,s.a)->Limit->Projection", + best: "Join{DataScan(t)->Limit->DataScan(s)}(test.t.a,test.s.a)->Limit->Projection", }, // Test TopN + Left Join Apply + Proj. { sql: "select (select s.a from t s where t.a = s.a) from t order by t.a limit 5", - best: "Join{DataScan(t)->TopN([test.t.a],0,5)->DataScan(s)}(test.t.a,s.a)->TopN([test.t.a],0,5)->Projection", + best: "Join{DataScan(t)->TopN([test.t.a],0,5)->DataScan(s)}(test.t.a,test.s.a)->TopN([test.t.a],0,5)->Projection", }, // Test TopN + Left Semi Join Apply + Proj. { sql: "select exists (select s.a from t s where t.a = s.a) from t order by t.a limit 5", - best: "Join{DataScan(t)->TopN([test.t.a],0,5)->DataScan(s)}(test.t.a,s.a)->TopN([test.t.a],0,5)->Projection", + best: "Join{DataScan(t)->TopN([test.t.a],0,5)->DataScan(s)}(test.t.a,test.s.a)->TopN([test.t.a],0,5)->Projection", }, // Test TopN + Semi Join Apply + Proj. { sql: "select * from t where exists (select s.a from t s where t.a = s.a) order by t.a limit 5", - best: "Join{DataScan(t)->DataScan(s)}(test.t.a,s.a)->TopN([test.t.a],0,5)->Projection", + best: "Join{DataScan(t)->DataScan(s)}(test.t.a,test.s.a)->TopN([test.t.a],0,5)->Projection", }, // Test TopN + Right Join + Proj. { sql: "select * from t right outer join t s on t.a = s.a order by s.a limit 5", - best: "Join{DataScan(t)->DataScan(s)->TopN([s.a],0,5)}(test.t.a,s.a)->TopN([s.a],0,5)->Projection", + best: "Join{DataScan(t)->DataScan(s)->TopN([test.s.a],0,5)}(test.t.a,test.s.a)->TopN([test.s.a],0,5)->Projection", }, // Test Limit + Right Join + Proj. { sql: "select * from t right outer join t s on t.a = s.a order by s.a,t.b limit 5", - best: "Join{DataScan(t)->DataScan(s)}(test.t.a,s.a)->TopN([s.a test.t.b],0,5)->Projection", + best: "Join{DataScan(t)->DataScan(s)}(test.t.a,test.s.a)->TopN([test.s.a test.t.b],0,5)->Projection", }, // Test TopN + UA + Proj. { sql: "select * from t union all (select * from t s) order by a,b limit 5", - 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([a b],0,5)", + best: "UnionAll{DataScan(t)->TopN([test.t.a test.t.b],0,5)->Projection->DataScan(s)->TopN([test.s.a test.s.b],0,5)->Projection}->TopN([a b],0,5)", }, // Test TopN + UA + Proj. { sql: "select * from t union all (select * from t s) order by a,b limit 5, 5", - 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([a b],5,5)", + best: "UnionAll{DataScan(t)->TopN([test.t.a test.t.b],0,10)->Projection->DataScan(s)->TopN([test.s.a test.s.b],0,10)->Projection}->TopN([a b],5,5)", }, // Test Limit + UA + Proj + Sort. { sql: "select * from t union all (select * from t s order by a) limit 5", - best: "UnionAll{DataScan(t)->Limit->Projection->DataScan(s)->TopN([s.a],0,5)->Projection}->Limit", + best: "UnionAll{DataScan(t)->Limit->Projection->DataScan(s)->TopN([test.s.a],0,5)->Projection}->Limit", }, // Test `ByItem` containing column from both sides. { sql: "select ifnull(t1.b, t2.a) from t t1 left join t t2 on t1.e=t2.e order by ifnull(t1.b, t2.a) limit 5", - best: "Join{DataScan(t1)->TopN([t1.b],0,5)->DataScan(t2)}(t1.e,t2.e)->TopN([t1.b],0,5)->Projection", + best: "Join{DataScan(t1)->TopN([test.t1.b],0,5)->DataScan(t2)}(test.t1.e,test.t2.e)->TopN([test.t1.b],0,5)->Projection", }, // Test ifnull cannot be eliminated { sql: "select ifnull(t1.h, t2.b) from t t1 left join t t2 on t1.e=t2.e order by ifnull(t1.h, t2.b) limit 5", - best: "Join{DataScan(t1)->DataScan(t2)}(t1.e,t2.e)->TopN([ifnull(t1.h, t2.b)],0,5)->Projection->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.e,test.t2.e)->TopN([ifnull(test.t1.h, test.t2.b)],0,5)->Projection->Projection", }, } for i, tt := range tests { @@ -1997,25 +1996,25 @@ func (s *testPlanSuite) TestOuterJoinEliminator(c *C) { // Test left outer join + distinct { sql: "select distinct t1.a, t1.b from t t1 left outer join t t2 on t1.b = t2.b", - best: "DataScan(t1)->Aggr(firstrow(t1.a),firstrow(t1.b))", + best: "DataScan(t1)->Aggr(firstrow(test.t1.a),firstrow(test.t1.b))", }, // Test right outer join + distinct { sql: "select distinct t2.a, t2.b from t t1 right outer join t t2 on t1.b = t2.b", - best: "DataScan(t2)->Aggr(firstrow(t2.a),firstrow(t2.b))", + best: "DataScan(t2)->Aggr(firstrow(test.t2.a),firstrow(test.t2.b))", }, // Test duplicate agnostic agg functions on join { - sql: "select max(t1.a), min(t1.b) from t t1 left join t t2 on t1.b = t2.b", - best: "DataScan(t1)->Aggr(max(t1.a),min(t1.b))->Projection", + sql: "select max(t1.a), min(test.t1.b) from t t1 left join t t2 on t1.b = t2.b", + best: "DataScan(t1)->Aggr(max(test.t1.a),min(test.t1.b))->Projection", }, { sql: "select sum(distinct t1.a) from t t1 left join t t2 on t1.a = t2.a and t1.b = t2.b", - best: "DataScan(t1)->Aggr(sum(t1.a))->Projection", + best: "DataScan(t1)->Aggr(sum(test.t1.a))->Projection", }, { sql: "select count(distinct t1.a, t1.b) from t t1 left join t t2 on t1.b = t2.b", - best: "DataScan(t1)->Aggr(count(t1.a, t1.b))->Projection", + best: "DataScan(t1)->Aggr(count(test.t1.a, test.t1.b))->Projection", }, // Test left outer join { @@ -2030,7 +2029,7 @@ func (s *testPlanSuite) TestOuterJoinEliminator(c *C) { // For complex join query { sql: "select max(t3.b) from (t t1 left join t t2 on t1.a = t2.a) right join t t3 on t1.b = t3.b", - best: "DataScan(t3)->TopN([t3.b true],0,1)->Aggr(max(t3.b))->Projection", + best: "DataScan(t3)->TopN([test.t3.b true],0,1)->Aggr(max(test.t3.b))->Projection", }, } @@ -2135,7 +2134,7 @@ func (s *testPlanSuite) TestWindowFunction(c *C) { }, { sql: "select a from t having (select sum(a) over() as w from t tt where a > t.a)", - result: "Apply{TableReader(Table(t))->TableReader(Table(t)->Sel([gt(tt.a, test.t.a)]))->Window(sum(cast(tt.a)) over())->MaxOneRow->Sel([w])}->Projection", + result: "Apply{TableReader(Table(t))->TableReader(Table(t)->Sel([gt(test.tt.a, test.t.a)]))->Window(sum(cast(test.tt.a)) over())->MaxOneRow->Sel([w])}->Projection", }, { sql: "select avg(a) over() as w from t having w > 1", diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 8a5b34fd20882..07d4c89e96edc 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -190,7 +190,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderSimpleCase(c *C) { }, { sql: "select * from (select * from t use index() order by b) t left join t t1 on t.a=t1.a limit 10", - best: "IndexJoin{TableReader(Table(t)->TopN([test.t.b],0,10))->TopN([test.t.b],0,10)->TableReader(Table(t))}(test.t.a,t1.a)->Limit", + best: "IndexJoin{TableReader(Table(t)->TopN([test.t.b],0,10))->TopN([test.t.b],0,10)->TableReader(Table(t))}(test.t.a,test.t1.a)->Limit", }, // Test embedded ORDER BY which imposes on different number of columns than outer query. { @@ -234,195 +234,195 @@ func (s *testPlanSuite) TestDAGPlanBuilderJoin(c *C) { }{ { sql: "select * from t t1 join t t2 on t1.a = t2.c_str", - best: "LeftHashJoin{TableReader(Table(t))->Projection->TableReader(Table(t))->Projection}(cast(t1.a),cast(t2.c_str))->Projection", + best: "LeftHashJoin{TableReader(Table(t))->Projection->TableReader(Table(t))->Projection}(cast(test.t1.a),cast(test.t2.c_str))->Projection", }, { sql: "select * from t t1 join t t2 on t1.b = t2.a", - best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t2.a)", + best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.b,test.t2.a)", }, { sql: "select * from t t1 join t t2 on t1.a = t2.a join t t3 on t1.a = t3.a", - best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t1.a,t3.a)", + best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->TableReader(Table(t))}(test.t1.a,test.t3.a)", }, { sql: "select * from t t1 join t t2 on t1.a = t2.a join t t3 on t1.b = t3.a", - best: "LeftHashJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t1.b,t3.a)", + best: "LeftHashJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->TableReader(Table(t))}(test.t1.b,test.t3.a)", }, { sql: "select * from t t1 join t t2 on t1.b = t2.a order by t1.a", - best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t2.a)->Sort", + best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.b,test.t2.a)->Sort", }, { sql: "select * from t t1 join t t2 on t1.b = t2.a order by t1.a limit 1", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t2.a)->Limit", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.b,test.t2.a)->Limit", }, // Test hash join's hint. { sql: "select /*+ TIDB_HJ(t1, t2) */ * from t t1 join t t2 on t1.b = t2.a order by t1.a limit 1", - best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t2.a)->TopN([t1.a],0,1)", + best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.b,test.t2.a)->TopN([test.t1.a],0,1)", }, { sql: "select * from t t1 left join t t2 on t1.b = t2.a where 1 = 1 limit 1", - best: "IndexJoin{TableReader(Table(t)->Limit)->Limit->TableReader(Table(t))}(t1.b,t2.a)->Limit", + best: "IndexJoin{TableReader(Table(t)->Limit)->Limit->TableReader(Table(t))}(test.t1.b,test.t2.a)->Limit", }, { sql: "select * from t t1 join t t2 on t1.b = t2.a and t1.c = 1 and t1.d = 1 and t1.e = 1 order by t1.a limit 1", - best: "IndexJoin{IndexLookUp(Index(t.c_d_e)[[1 1 1,1 1 1]], Table(t))->TableReader(Table(t))}(t1.b,t2.a)->TopN([t1.a],0,1)", + best: "IndexJoin{IndexLookUp(Index(t.c_d_e)[[1 1 1,1 1 1]], Table(t))->TableReader(Table(t))}(test.t1.b,test.t2.a)->TopN([test.t1.a],0,1)", }, { sql: "select * from t t1 join t t2 on t1.b = t2.b join t t3 on t1.b = t3.b", - best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.b,t2.b)->TableReader(Table(t))}(t1.b,t3.b)", + best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.b,test.t2.b)->TableReader(Table(t))}(test.t1.b,test.t3.b)", }, { sql: "select * from t t1 join t t2 on t1.a = t2.a order by t1.a", - best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", + best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)", }, { sql: "select * from t t1 left outer join t t2 on t1.a = t2.a right outer join t t3 on t1.a = t3.a", - best: "MergeRightOuterJoin{MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t1.a,t3.a)", + best: "MergeRightOuterJoin{MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->TableReader(Table(t))}(test.t1.a,test.t3.a)", }, { sql: "select * from t t1 join t t2 on t1.a = t2.a join t t3 on t1.a = t3.a and t1.b = 1 and t3.c = 1", - best: "IndexJoin{IndexJoin{TableReader(Table(t)->Sel([eq(t1.b, 1)]))->IndexLookUp(Index(t.c_d_e)[[1,1]], Table(t))}(t3.a,t1.a)->TableReader(Table(t))}(t1.a,t2.a)->Projection", + best: "IndexJoin{IndexJoin{TableReader(Table(t)->Sel([eq(test.t1.b, 1)]))->IndexLookUp(Index(t.c_d_e)[[1,1]], Table(t))}(test.t3.a,test.t1.a)->TableReader(Table(t))}(test.t1.a,test.t2.a)->Projection", }, { sql: "select * from t where t.c in (select b from t s where s.a = t.a)", - best: "MergeSemiJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,s.a)", + best: "MergeSemiJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.s.a)", }, { sql: "select t.c in (select b from t s where s.a = t.a) from t", - best: "MergeLeftOuterSemiJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,s.a)->Projection", + best: "MergeLeftOuterSemiJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.s.a)->Projection", }, // Test Single Merge Join. // Merge Join now enforce a sort. { sql: "select /*+ TIDB_SMJ(t1,t2)*/ * from t t1, t t2 where t1.a = t2.b", - best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))->Sort}(t1.a,t2.b)", + best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))->Sort}(test.t1.a,test.t2.b)", }, { sql: "select /*+ TIDB_SMJ(t1,t2)*/ * from t t1, t t2 where t1.a = t2.a", - best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", + best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)", }, // Test Single Merge Join + Sort. { sql: "select /*+ TIDB_SMJ(t1,t2)*/ * from t t1, t t2 where t1.a = t2.a order by t2.a", - best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", + best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)", }, { sql: "select /*+ TIDB_SMJ(t1,t2)*/ * from t t1, t t2 where t1.b = t2.b order by t2.a", - best: "MergeInnerJoin{TableReader(Table(t))->Sort->TableReader(Table(t))->Sort}(t1.b,t2.b)->Sort", + best: "MergeInnerJoin{TableReader(Table(t))->Sort->TableReader(Table(t))->Sort}(test.t1.b,test.t2.b)->Sort", }, // Test Single Merge Join + Sort + desc. { sql: "select /*+ TIDB_SMJ(t1,t2)*/ * from t t1, t t2 where t1.a = t2.a order by t2.a desc", - best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", + best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)", }, { sql: "select /*+ TIDB_SMJ(t1,t2)*/ * from t t1, t t2 where t1.b = t2.b order by t2.b desc", - best: "MergeInnerJoin{TableReader(Table(t))->Sort->TableReader(Table(t))->Sort}(t1.b,t2.b)", + best: "MergeInnerJoin{TableReader(Table(t))->Sort->TableReader(Table(t))->Sort}(test.t1.b,test.t2.b)", }, // Test Multi Merge Join. { sql: "select /*+ TIDB_SMJ(t1,t2,t3)*/ * from t t1, t t2, t t3 where t1.a = t2.a and t2.a = t3.a", - best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t2.a,t3.a)", + best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->TableReader(Table(t))}(test.t2.a,test.t3.a)", }, { sql: "select /*+ TIDB_SMJ(t1,t2,t3)*/ * from t t1, t t2, t t3 where t1.a = t2.b and t2.a = t3.b", - best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))->Sort}(t1.a,t2.b)->Sort->TableReader(Table(t))->Sort}(t2.a,t3.b)", + best: "MergeInnerJoin{MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))->Sort}(test.t1.a,test.t2.b)->Sort->TableReader(Table(t))->Sort}(test.t2.a,test.t3.b)", }, // Test Multi Merge Join with multi keys. // TODO: More tests should be added. { sql: "select /*+ TIDB_SMJ(t1,t2,t3)*/ * from t t1, t t2, t t3 where t1.c = t2.c and t1.d = t2.d and t3.c = t1.c and t3.d = t1.d", - best: "MergeInnerJoin{MergeInnerJoin{IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.c,t2.c)(t1.d,t2.d)->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.c,t3.c)(t1.d,t3.d)", + best: "MergeInnerJoin{MergeInnerJoin{IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.c,test.t2.c)(test.t1.d,test.t2.d)->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.c,test.t3.c)(test.t1.d,test.t3.d)", }, { sql: "select /*+ TIDB_SMJ(t1,t2,t3)*/ * from t t1, t t2, t t3 where t1.c = t2.c and t1.d = t2.d and t3.c = t1.c and t3.d = t1.d order by t1.c", - best: "MergeInnerJoin{MergeInnerJoin{IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.c,t2.c)(t1.d,t2.d)->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.c,t3.c)(t1.d,t3.d)", + best: "MergeInnerJoin{MergeInnerJoin{IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.c,test.t2.c)(test.t1.d,test.t2.d)->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.c,test.t3.c)(test.t1.d,test.t3.d)", }, // Test Multi Merge Join + Outer Join. { sql: "select /*+ TIDB_SMJ(t1,t2,t3)*/ * from t t1 left outer join t t2 on t1.a = t2.a left outer join t t3 on t2.a = t3.a", - best: "MergeLeftOuterJoin{MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t2.a,t3.a)", + best: "MergeLeftOuterJoin{MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->TableReader(Table(t))}(test.t2.a,test.t3.a)", }, { sql: "select /*+ TIDB_SMJ(t1,t2,t3)*/ * from t t1 left outer join t t2 on t1.a = t2.a left outer join t t3 on t1.a = t3.a", - best: "MergeLeftOuterJoin{MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->TableReader(Table(t))}(t1.a,t3.a)", + best: "MergeLeftOuterJoin{MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->TableReader(Table(t))}(test.t1.a,test.t3.a)", }, // Test Index Join + TableScan. { sql: "select /*+ TIDB_INLJ(t1, t2) */ * from t t1, t t2 where t1.a = t2.a", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)", }, // Test Index Join + DoubleRead. { sql: "select /*+ TIDB_INLJ(t1, t2) */ * from t t1, t t2 where t1.a = t2.c", - best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.a,t2.c)", + best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.a,test.t2.c)", }, // Test Index Join + SingleRead. { sql: "select /*+ TIDB_INLJ(t1, t2) */ t1.a , t2.a from t t1, t t2 where t1.a = t2.c", - best: "IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(t1.a,t2.c)->Projection", + best: "IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t1.a,test.t2.c)->Projection", }, // Test Index Join + Order by. { sql: "select /*+ TIDB_INLJ(t1, t2) */ t1.a, t2.a from t t1, t t2 where t1.a = t2.a order by t1.c", - best: "IndexJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]])->TableReader(Table(t))}(t1.a,t2.a)->Projection", + best: "IndexJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]])->TableReader(Table(t))}(test.t1.a,test.t2.a)->Projection", }, // Test Index Join + Order by. { sql: "select /*+ TIDB_INLJ(t1, t2) */ t1.a, t2.a from t t1, t t2 where t1.a = t2.a order by t2.c", - best: "IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(t2.a,t1.a)->Projection", + best: "IndexJoin{TableReader(Table(t))->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.t2.a,test.t1.a)->Projection", }, // Test Index Join + TableScan + Rotate. { sql: "select /*+ TIDB_INLJ(t1) */ t1.a , t2.a from t t1, t t2 where t1.a = t2.c", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t2.c,t1.a)->Projection", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t2.c,test.t1.a)->Projection", }, // Test Index Join + OuterJoin + TableScan. { sql: "select /*+ TIDB_INLJ(t1, t2) */ * from t t1 left outer join t t2 on t1.a = t2.a and t2.b < 1", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t)->Sel([lt(t2.b, 1)]))}(t1.a,t2.a)", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t)->Sel([lt(test.t2.b, 1)]))}(test.t1.a,test.t2.a)", }, { sql: "select /*+ TIDB_INLJ(t1, t2) */ * from t t1 join t t2 on t1.d=t2.d and t2.c = 1", - best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.d,t2.d)", + best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.d,test.t2.d)", }, // Test Index Join failed. { sql: "select /*+ TIDB_INLJ(t1, t2) */ * from t t1 left outer join t t2 on t1.a = t2.b", - best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.b)", + best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.b)", }, // Test Index Join failed. { sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 right outer join t t2 on t1.a = t2.b", - best: "RightHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.b)", + best: "RightHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.b)", }, // Test Semi Join hint success. { sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 where t1.a in (select a from t t2)", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->Projection", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->Projection", }, // Test Semi Join hint fail. { sql: "select /*+ TIDB_INLJ(t1) */ * from t t1 where t1.a in (select a from t t2)", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t2.a,t1.a)->Projection", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t2.a,test.t1.a)->Projection", }, { sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 where t1.c=t2.c and t1.f=t2.f", - best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.c,t2.c)", + best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.c,test.t2.c)", }, { sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 where t1.a = t2.a and t1.f=t2.f", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)", }, { sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 where t1.f=t2.f and t1.a=t2.a", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)", }, { sql: "select /*+ TIDB_INLJ(t2) */ * from t t1 join t t2 where t1.a=t2.a and t2.a in (1, 2)", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t)->Sel([in(t2.a, 1, 2)]))}(t1.a,t2.a)", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t)->Sel([in(test.t2.a, 1, 2)]))}(test.t1.a,test.t2.a)", }, } for i, tt := range tests { @@ -469,12 +469,12 @@ func (s *testPlanSuite) TestDAGPlanBuilderSubquery(c *C) { //}, { sql: "select * from t where a in (select s.a from t s) order by t.a", - best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,s.a)->Projection", + best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.s.a)->Projection", }, // Test Nested sub query. { sql: "select * from t where exists (select s.a from t s where s.c in (select c from t as k where k.d = s.d) having sum(s.a) = t.a )", - best: "LeftHashJoin{TableReader(Table(t))->Projection->MergeSemiJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]])->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(s.c,k.c)(s.d,k.d)->Projection->StreamAgg}(cast(test.t.a),sel_agg_1)->Projection", + best: "LeftHashJoin{TableReader(Table(t))->Projection->MergeSemiJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]])->IndexReader(Index(t.c_d_e)[[NULL,+inf]])}(test.s.c,test.k.c)(test.s.d,test.k.d)->Projection->StreamAgg}(cast(test.t.a),sel_agg_1)->Projection", }, // Test Semi Join + Order by. { @@ -484,15 +484,15 @@ func (s *testPlanSuite) TestDAGPlanBuilderSubquery(c *C) { // Test Apply. { sql: "select t.c in (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t", - best: "Apply{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->TableReader(Table(t)->Sel([eq(t1.a, test.t.a)]))}(s.a,t1.a)->StreamAgg}->Projection", + best: "Apply{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->TableReader(Table(t)->Sel([eq(test.t1.a, test.t.a)]))}(test.s.a,test.t1.a)->StreamAgg}->Projection", }, { sql: "select (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t", - best: "LeftHashJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(s.a,t1.a)->StreamAgg}(test.t.a,s.a)->Projection->Projection", + best: "LeftHashJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.s.a,test.t1.a)->StreamAgg}(test.t.a,test.s.a)->Projection->Projection", }, { sql: "select (select count(*) from t s , t t1 where s.a = t.a and s.a = t1.a) from t order by t.a", - best: "LeftHashJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(s.a,t1.a)->StreamAgg}(test.t.a,s.a)->Projection->Sort->Projection", + best: "LeftHashJoin{TableReader(Table(t))->MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.s.a,test.t1.a)->StreamAgg}(test.t.a,test.s.a)->Projection->Sort->Projection", }, } for _, tt := range tests { @@ -525,15 +525,15 @@ func (s *testPlanSuite) TestDAGPlanTopN(c *C) { }{ { sql: "select * from t t1 left join t t2 on t1.b = t2.b left join t t3 on t2.b = t3.b order by t1.a limit 1", - best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t)->Limit)->Limit->TableReader(Table(t))}(t1.b,t2.b)->TopN([t1.a],0,1)->TableReader(Table(t))}(t2.b,t3.b)->TopN([t1.a],0,1)", + best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t)->Limit)->Limit->TableReader(Table(t))}(test.t1.b,test.t2.b)->TopN([test.t1.a],0,1)->TableReader(Table(t))}(test.t2.b,test.t3.b)->TopN([test.t1.a],0,1)", }, { sql: "select * from t t1 left join t t2 on t1.b = t2.b left join t t3 on t2.b = t3.b order by t1.b limit 1", - best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t)->TopN([t1.b],0,1))->TopN([t1.b],0,1)->TableReader(Table(t))}(t1.b,t2.b)->TopN([t1.b],0,1)->TableReader(Table(t))}(t2.b,t3.b)->TopN([t1.b],0,1)", + best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t)->TopN([test.t1.b],0,1))->TopN([test.t1.b],0,1)->TableReader(Table(t))}(test.t1.b,test.t2.b)->TopN([test.t1.b],0,1)->TableReader(Table(t))}(test.t2.b,test.t3.b)->TopN([test.t1.b],0,1)", }, { sql: "select * from t t1 left join t t2 on t1.b = t2.b left join t t3 on t2.b = t3.b limit 1", - best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t)->Limit)->Limit->TableReader(Table(t))}(t1.b,t2.b)->Limit->TableReader(Table(t))}(t2.b,t3.b)->Limit", + best: "LeftHashJoin{LeftHashJoin{TableReader(Table(t)->Limit)->Limit->TableReader(Table(t))}(test.t1.b,test.t2.b)->Limit->TableReader(Table(t))}(test.t2.b,test.t3.b)->Limit", }, { sql: "select * from t where b = 1 and c = 1 order by c limit 1", @@ -601,19 +601,19 @@ func (s *testPlanSuite) TestDAGPlanBuilderBasePhysicalPlan(c *C) { // Test join hint for delete and update { sql: "delete /*+ TIDB_INLJ(t1, t2) */ t1 from t t1, t t2 where t1.c=t2.c", - best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.c,t2.c)->Delete", + best: "IndexJoin{TableReader(Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.c,test.t2.c)->Delete", }, { sql: "delete /*+ TIDB_SMJ(t1, t2) */ from t1 using t t1, t t2 where t1.c=t2.c", - best: "MergeInnerJoin{IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(t1.c,t2.c)->Delete", + best: "MergeInnerJoin{IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))}(test.t1.c,test.t2.c)->Delete", }, { sql: "update /*+ TIDB_SMJ(t1, t2) */ t t1, t t2 set t1.a=1, t2.a=1 where t1.a=t2.a", - best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->Update", + best: "MergeInnerJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->Update", }, { sql: "update /*+ TIDB_HJ(t1, t2) */ t t1, t t2 set t1.a=1, t2.a=1 where t1.a=t2.a", - best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->Update", + best: "LeftHashJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->Update", }, // Test complex delete. { @@ -867,7 +867,7 @@ func (s *testPlanSuite) TestDAGPlanBuilderAgg(c *C) { }, { sql: "select (select count(1) k from t s where s.a = t.a having k != 0) from t", - best: "MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))->Projection}(test.t.a,s.a)->Projection->Projection", + best: "MergeLeftOuterJoin{TableReader(Table(t))->TableReader(Table(t))->Projection}(test.t.a,test.s.a)->Projection->Projection", }, // Test stream agg with multi group by columns. { @@ -919,16 +919,16 @@ func (s *testPlanSuite) TestDAGPlanBuilderAgg(c *C) { // Test merge join + stream agg { sql: "select sum(a.g), sum(b.g) from t a join t b on a.g = b.g group by a.g", - best: "MergeInnerJoin{IndexReader(Index(t.g)[[NULL,+inf]])->IndexReader(Index(t.g)[[NULL,+inf]])}(a.g,b.g)->Projection->StreamAgg", + best: "MergeInnerJoin{IndexReader(Index(t.g)[[NULL,+inf]])->IndexReader(Index(t.g)[[NULL,+inf]])}(test.a.g,test.b.g)->Projection->StreamAgg", }, // Test index join + stream agg { sql: "select /*+ tidb_inlj(a,b) */ sum(a.g), sum(b.g) from t a join t b on a.g = b.g and a.g > 60 group by a.g order by a.g limit 1", - best: "IndexJoin{IndexReader(Index(t.g)[(60,+inf]])->IndexReader(Index(t.g)[[NULL,+inf]]->Sel([gt(b.g, 60)]))}(a.g,b.g)->Projection->StreamAgg->Limit->Projection", + best: "IndexJoin{IndexReader(Index(t.g)[(60,+inf]])->IndexReader(Index(t.g)[[NULL,+inf]]->Sel([gt(test.b.g, 60)]))}(test.a.g,test.b.g)->Projection->StreamAgg->Limit->Projection", }, { sql: "select sum(a.g), sum(b.g) from t a join t b on a.g = b.g and a.a>5 group by a.g order by a.g limit 1", - best: "IndexJoin{IndexReader(Index(t.g)[[NULL,+inf]]->Sel([gt(a.a, 5)]))->IndexReader(Index(t.g)[[NULL,+inf]])}(a.g,b.g)->Projection->StreamAgg->Limit->Projection", + best: "IndexJoin{IndexReader(Index(t.g)[[NULL,+inf]]->Sel([gt(test.a.a, 5)]))->IndexReader(Index(t.g)[[NULL,+inf]])}(test.a.g,test.b.g)->Projection->StreamAgg->Limit->Projection", }, { sql: "select sum(d) from t", @@ -1235,7 +1235,7 @@ func (s *testPlanSuite) TestAggEliminater(c *C) { // If inner is not a data source, we can still do transformation. { sql: "select max(a) from (select t1.a from t t1 join t t2 on t1.a=t2.a) t", - best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(t1.a,t2.a)->Limit->StreamAgg", + best: "IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t1.a,test.t2.a)->Limit->StreamAgg", }, } @@ -1322,25 +1322,25 @@ func (s *testPlanSuite) TestIndexJoinUnionScan(c *C) { // Test Index Join + UnionScan + TableScan. { sql: "select /*+ TIDB_INLJ(t1, t2) */ * from t t1, t t2 where t1.a = t2.a", - best: "IndexJoin{TableReader(Table(t))->UnionScan([])->TableReader(Table(t))->UnionScan([])}(t1.a,t2.a)", + best: "IndexJoin{TableReader(Table(t))->UnionScan([])->TableReader(Table(t))->UnionScan([])}(test.t1.a,test.t2.a)", is: s.is, }, // Test Index Join + UnionScan + DoubleRead. { sql: "select /*+ TIDB_INLJ(t1, t2) */ * from t t1, t t2 where t1.a = t2.c", - best: "IndexJoin{TableReader(Table(t))->UnionScan([])->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))->UnionScan([])}(t1.a,t2.c)", + best: "IndexJoin{TableReader(Table(t))->UnionScan([])->IndexLookUp(Index(t.c_d_e)[[NULL,+inf]], Table(t))->UnionScan([])}(test.t1.a,test.t2.c)", is: s.is, }, // Test Index Join + UnionScan + IndexScan. { sql: "select /*+ TIDB_INLJ(t1, t2) */ t1.a , t2.c from t t1, t t2 where t1.a = t2.c", - best: "IndexJoin{TableReader(Table(t))->UnionScan([])->IndexReader(Index(t.c_d_e)[[NULL,+inf]])->UnionScan([])}(t1.a,t2.c)->Projection", + best: "IndexJoin{TableReader(Table(t))->UnionScan([])->IndexReader(Index(t.c_d_e)[[NULL,+inf]])->UnionScan([])}(test.t1.a,test.t2.c)->Projection", is: s.is, }, // Index Join + Union Scan + Union All is not supported now. { sql: "select /*+ TIDB_INLJ(t1, t2) */ * from t t1, t t2 where t1.a = t2.a", - best: "LeftHashJoin{UnionAll{TableReader(Table(t))->UnionScan([])->TableReader(Table(t))->UnionScan([])}->UnionAll{TableReader(Table(t))->UnionScan([])->TableReader(Table(t))->UnionScan([])}}(t1.a,t2.a)", + best: "LeftHashJoin{UnionAll{TableReader(Table(t))->UnionScan([])->TableReader(Table(t))->UnionScan([])}->UnionAll{TableReader(Table(t))->UnionScan([])->TableReader(Table(t))->UnionScan([])}}(test.t1.a,test.t2.a)", is: pis, }, } @@ -1433,7 +1433,7 @@ func (s *testPlanSuite) TestSemiJoinToInner(c *C) { c.Assert(err, IsNil) p, err := planner.Optimize(se, stmt, s.is) c.Assert(err, IsNil) - c.Assert(core.ToString(p), Equals, "Apply{TableReader(Table(t))->IndexJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]]->HashAgg)->HashAgg->IndexReader(Index(t.g)[[NULL,+inf]])}(t3.d,t2.g)}->StreamAgg") + c.Assert(core.ToString(p), Equals, "Apply{TableReader(Table(t))->IndexJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]]->HashAgg)->HashAgg->IndexReader(Index(t.g)[[NULL,+inf]])}(test.t3.d,test.t2.g)}->StreamAgg") } func (s *testPlanSuite) TestUnmatchedTableInHint(c *C) { diff --git a/session/session_test.go b/session/session_test.go index d3de1403beaa3..714001eac15c7 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2527,6 +2527,29 @@ func (s *testSessionSuite) TestUpdatePrivilege(c *C) { []byte(""), []byte("")), IsTrue) tk1.MustExec("use weperk") tk1.MustExec("update tb_wehub_server a set a.active_count=a.active_count+1,a.used_count=a.used_count+1 where id=1") + + tk.MustExec("create database service") + tk.MustExec("create database report") + tk.MustExec(`CREATE TABLE service.t1 ( + id int(11) DEFAULT NULL, + a bigint(20) NOT NULL, + b text DEFAULT NULL, + PRIMARY KEY (a) +)`) + tk.MustExec(`CREATE TABLE report.t2 ( + a bigint(20) DEFAULT NULL, + c bigint(20) NOT NULL +)`) + tk.MustExec("grant all privileges on service.* to weperk") + tk.MustExec("grant all privileges on report.* to weperk") + tk1.Se.GetSessionVars().CurrentDB = "" + tk1.MustExec(`update service.t1 s, +report.t2 t +set s.a = t.a +WHERE +s.a = t.a +and t.c >= 1 and t.c <= 10000 +and s.b !='xx';`) } func (s *testSessionSuite) TestTxnGoString(c *C) { diff --git a/util/ranger/ranger_test.go b/util/ranger/ranger_test.go index 5d17fdb91e31e..fa01ae36281ce 100644 --- a/util/ranger/ranger_test.go +++ b/util/ranger/ranger_test.go @@ -1011,11 +1011,11 @@ func (s *testRangerSuite) TestCompIndexInExprCorrCol(c *C) { " ├─TableReader_15 2.00 root data:TableScan_14", " │ └─TableScan_14 2.00 cop table:t, range:[-inf,+inf], keep order:false", " └─StreamAgg_20 1.00 root funcs:count(1)", - " └─IndexJoin_32 2.00 root inner join, inner:TableReader_31, outer key:s.a, inner key:t1.a", + " └─IndexJoin_32 2.00 root inner join, inner:TableReader_31, outer key:test.s.a, inner key:test.t1.a", " ├─IndexReader_27 2.00 root index:IndexScan_26", - " │ └─IndexScan_26 2.00 cop table:s, index:b, c, d, range: decided by [eq(s.b, 1) in(s.c, 1, 2) eq(s.d, test.t.a)], keep order:false", + " │ └─IndexScan_26 2.00 cop table:s, index:b, c, d, range: decided by [eq(test.s.b, 1) in(test.s.c, 1, 2) eq(test.s.d, test.t.a)], keep order:false", " └─TableReader_31 1.00 root data:TableScan_30", - " └─TableScan_30 1.00 cop table:t1, range: decided by [s.a], keep order:false", + " └─TableScan_30 1.00 cop table:t1, range: decided by [test.s.a], keep order:false", )) testKit.MustQuery("select t.e in (select count(*) from t s use index(idx), t t1 where s.b = 1 and s.c in (1, 2) and s.d = t.a and s.a = t1.a) from t").Check(testkit.Rows( "1", From cf0ca744a2afe50bbbb7d155d61ee2ab4f9525a1 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Tue, 30 Apr 2019 13:41:21 +0800 Subject: [PATCH 27/49] bindinfo: fix data race of parser in global handle (#10321) --- bindinfo/bind_test.go | 6 +++--- bindinfo/handle.go | 21 ++++++++++----------- domain/domain.go | 5 ++--- session/session.go | 2 +- 4 files changed, 16 insertions(+), 18 deletions(-) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index e37ae0f45eff3..3c716fc39354d 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -117,7 +117,7 @@ func (s *testSuite) TestBindParse(c *C) { sql := fmt.Sprintf(`INSERT INTO mysql.bind_info(original_sql,bind_sql,default_db,status,create_time,update_time,charset,collation) VALUES ('%s', '%s', '%s', '%s', NOW(), NOW(),'%s', '%s')`, originSQL, bindSQL, defaultDb, status, charset, collation) tk.MustExec(sql) - bindHandle := bindinfo.NewBindHandle(tk.Se, s.Parser) + bindHandle := bindinfo.NewBindHandle(tk.Se) err := bindHandle.Update(true) c.Check(err, IsNil) c.Check(bindHandle.Size(), Equals, 1) @@ -178,7 +178,7 @@ func (s *testSuite) TestGlobalBinding(c *C) { c.Check(row.GetString(6), NotNil) c.Check(row.GetString(7), NotNil) - bindHandle := bindinfo.NewBindHandle(tk.Se, s.Parser) + bindHandle := bindinfo.NewBindHandle(tk.Se) err = bindHandle.Update(true) c.Check(err, IsNil) c.Check(bindHandle.Size(), Equals, 1) @@ -199,7 +199,7 @@ func (s *testSuite) TestGlobalBinding(c *C) { bindData = s.domain.BindHandle().GetBindRecord("select * from t where i > ?", "test") c.Check(bindData, IsNil) - bindHandle = bindinfo.NewBindHandle(tk.Se, s.Parser) + bindHandle = bindinfo.NewBindHandle(tk.Se) err = bindHandle.Update(true) c.Check(err, IsNil) c.Check(bindHandle.Size(), Equals, 0) diff --git a/bindinfo/handle.go b/bindinfo/handle.go index fc33f2cd3442c..140b6f8cd8884 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -59,6 +59,7 @@ type BindHandle struct { bindInfo struct { sync.Mutex atomic.Value + parser *parser.Parser } // invalidBindRecordMap indicates the invalid bind records found during querying. @@ -68,7 +69,6 @@ type BindHandle struct { atomic.Value } - parser *parser.Parser lastUpdateTime types.Time } @@ -78,10 +78,11 @@ type invalidBindRecordMap struct { } // NewBindHandle creates a new BindHandle. -func NewBindHandle(ctx sessionctx.Context, parser *parser.Parser) *BindHandle { - handle := &BindHandle{parser: parser} +func NewBindHandle(ctx sessionctx.Context) *BindHandle { + handle := &BindHandle{} handle.sctx.Context = ctx handle.bindInfo.Value.Store(make(cache, 32)) + handle.bindInfo.parser = parser.New() handle.invalidBindRecordMap.Value.Store(make(map[string]*invalidBindRecordMap)) return handle } @@ -151,14 +152,18 @@ func (h *BindHandle) AddBindRecord(record *BindRecord) (err error) { return } + // Make sure there is only one goroutine writes the cache and use parser. + h.bindInfo.Lock() // update the BindMeta to the cache. hash, meta, err1 := h.newBindMeta(record) if err1 != nil { err = err1 + h.bindInfo.Unlock() return } h.appendBindMeta(hash, meta) + h.bindInfo.Unlock() }() // remove all the unused sql binds. @@ -294,7 +299,7 @@ func (h *BindHandle) GetAllBindRecord() (bindRecords []*BindMeta) { func (h *BindHandle) newBindMeta(record *BindRecord) (hash string, meta *BindMeta, err error) { hash = parser.DigestHash(record.OriginalSQL) - stmtNodes, _, err := h.parser.Parse(record.BindSQL, record.Charset, record.Collation) + stmtNodes, _, err := h.bindInfo.parser.Parse(record.BindSQL, record.Charset, record.Collation) if err != nil { return "", nil, err } @@ -311,16 +316,10 @@ func newBindMetaWithoutAst(record *BindRecord) (hash string, meta *BindMeta) { // appendBindMeta addes the BindMeta to the cache, all the stale bindMetas are // removed from the cache after this operation. func (h *BindHandle) appendBindMeta(hash string, meta *BindMeta) { - // Make sure there is only one goroutine writes the cache. - h.bindInfo.Lock() newCache := h.bindInfo.Value.Load().(cache).copy() - defer func() { - h.bindInfo.Value.Store(newCache) - h.bindInfo.Unlock() - }() - newCache.removeStaleBindMetas(hash, meta) newCache[hash] = append(newCache[hash], meta) + h.bindInfo.Value.Store(newCache) } // removeBindMeta removes the BindMeta from the cache. diff --git a/domain/domain.go b/domain/domain.go index 814452ea9870a..1941cae21341e 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -26,7 +26,6 @@ import ( "github.com/ngaut/sync2" "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/parser" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/parser/terror" @@ -784,9 +783,9 @@ func (do *Domain) BindHandle() *bindinfo.BindHandle { // LoadBindInfoLoop create a goroutine loads BindInfo in a loop, it should // be called only once in BootstrapSession. -func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context, parser *parser.Parser) error { +func (do *Domain) LoadBindInfoLoop(ctx sessionctx.Context) error { ctx.GetSessionVars().InRestrictedSQL = true - do.bindHandle = bindinfo.NewBindHandle(ctx, parser) + do.bindHandle = bindinfo.NewBindHandle(ctx) err := do.bindHandle.Update(true) if err != nil { return err diff --git a/session/session.go b/session/session.go index 1f504b9dd4ddd..e220ed42def4f 100644 --- a/session/session.go +++ b/session/session.go @@ -1466,7 +1466,7 @@ func BootstrapSession(store kv.Storage) (*domain.Domain, error) { if err != nil { return nil, err } - err = dom.LoadBindInfoLoop(se2, se2.parser) + err = dom.LoadBindInfoLoop(se2) if err != nil { return nil, err } From b29e19fd8ca870db0399c0ed151d9d3e03c34fbd Mon Sep 17 00:00:00 2001 From: qupeng Date: Tue, 30 Apr 2019 13:51:33 +0800 Subject: [PATCH 28/49] tikvclient: fix a bug that recreate connection forever (#10301) --- store/tikv/client.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/store/tikv/client.go b/store/tikv/client.go index 3a52a5f67de09..a6dc09fd99898 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -126,13 +126,13 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) { // When `conn.Close()` is called, `client.Recv()` will return an error. resp, err := c.client.Recv() if err != nil { - if c.isStopped() { - return - } logutil.Logger(context.Background()).Error("batchRecvLoop error when receive", zap.Error(err)) now := time.Now() for { // try to re-create the streaming in the loop. + if c.isStopped() { + return + } // Hold the lock to forbid batchSendLoop using the old client. c.clientLock.Lock() c.failPendingRequests(err) // fail all pending requests. From 52e67447fbd2cd10d8c921032215dd7f8d407fb9 Mon Sep 17 00:00:00 2001 From: mantuliu <240951888@qq.com> Date: Tue, 30 Apr 2019 14:59:05 +0800 Subject: [PATCH 29/49] Modify method parameters and return values (#10288) --- plugin/conn_ip_example/conn_ip_example.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin/conn_ip_example/conn_ip_example.go b/plugin/conn_ip_example/conn_ip_example.go index ebdc33ca5f114..08c14f9250445 100644 --- a/plugin/conn_ip_example/conn_ip_example.go +++ b/plugin/conn_ip_example/conn_ip_example.go @@ -41,9 +41,9 @@ func OnShutdown(ctx context.Context, manifest *plugin.Manifest) error { } // OnGeneralEvent implements TiDB Audit plugin's OnGeneralEvent SPI. -func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd byte, stmt string) error { +func OnGeneralEvent(ctx context.Context, sctx *variable.SessionVars, event plugin.GeneralEvent, cmd string) { fmt.Println("conn_ip_example notifiy called") fmt.Println("variable test: ", variable.GetSysVar("conn_ip_example_test_variable").Value) fmt.Printf("new connection by %s\n", ctx.Value("ip")) - return nil + return } From e90804f0a98947130266b776405f97366e000c2e Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 30 Apr 2019 16:40:36 +0800 Subject: [PATCH 30/49] doamin: fix GetSnapshotInfoSchema using current TS (#10309) --- domain/domain.go | 9 +++------ domain/domain_test.go | 12 ++++++++++++ 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 1941cae21341e..8c12fc9e8b8f9 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -218,14 +218,10 @@ func isTooOldSchema(usedVersion, newVersion int64) bool { // The second returned value is the delta updated table IDs. func (do *Domain) tryLoadSchemaDiffs(m *meta.Meta, usedVersion, newVersion int64) (bool, []int64, error) { // If there isn't any used version, or used version is too old, we do full load. + // And when users use history read feature, we will set usedVersion to initialVersion, then full load is needed. if isTooOldSchema(usedVersion, newVersion) { return false, nil, nil } - if usedVersion > newVersion { - // When user use History Read feature, history schema will be loaded. - // usedVersion may be larger than newVersion, full load is needed. - return false, nil, nil - } var diffs []*model.SchemaDiff for usedVersion < newVersion { usedVersion++ @@ -260,7 +256,8 @@ func (do *Domain) InfoSchema() infoschema.InfoSchema { // GetSnapshotInfoSchema gets a snapshot information schema. func (do *Domain) GetSnapshotInfoSchema(snapshotTS uint64) (infoschema.InfoSchema, error) { snapHandle := do.infoHandle.EmptyClone() - _, _, _, err := do.loadInfoSchema(snapHandle, do.infoHandle.Get().SchemaMetaVersion(), snapshotTS) + // For the snapHandle, it's an empty Handle, so its usedSchemaVersion is initialVersion. + _, _, _, err := do.loadInfoSchema(snapHandle, initialVersion, snapshotTS) if err != nil { return nil, err } diff --git a/domain/domain_test.go b/domain/domain_test.go index d01fa94ad65ff..5c706a0a0523c 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testleak" dto "github.com/prometheus/client_model/go" @@ -63,6 +64,7 @@ func (*testSuite) TestT(c *C) { store = dom.Store() ctx := mock.NewContext() ctx.Store = store + snapTS := oracle.EncodeTSO(oracle.GetPhysical(time.Now())) dd := dom.DDL() c.Assert(dd, NotNil) c.Assert(dd.GetLease(), Equals, 80*time.Millisecond) @@ -78,6 +80,16 @@ func (*testSuite) TestT(c *C) { // for setting lease lease := 100 * time.Millisecond + // for GetSnapshotInfoSchema + snapIs, err := dom.GetSnapshotInfoSchema(snapTS) + c.Assert(err, IsNil) + c.Assert(snapIs, NotNil) + snapTS = oracle.EncodeTSO(oracle.GetPhysical(time.Now())) + snapIs, err = dom.GetSnapshotInfoSchema(snapTS) + c.Assert(err, IsNil) + c.Assert(snapIs, NotNil) + c.Assert(snapIs.SchemaMetaVersion(), Equals, is.SchemaMetaVersion()) + // for schemaValidator schemaVer := dom.SchemaValidator.(*schemaValidator).latestSchemaVer ver, err := store.CurrentVersion() From e8d9a3f075e3b81c6cf27c41605f43b6aef39b3b Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Tue, 30 Apr 2019 17:20:23 +0800 Subject: [PATCH 31/49] expression: check timezone when encoding timestamp datum (#10303) --- expression/distsql_builtin.go | 2 +- expression/expr_to_pb.go | 8 +++---- expression/integration_test.go | 14 +++++++++++++ util/codec/codec.go | 38 ++++++++++++++++++++++------------ 4 files changed, 43 insertions(+), 19 deletions(-) diff --git a/expression/distsql_builtin.go b/expression/distsql_builtin.go index 96ac0397d0cd5..127e6baa95c78 100644 --- a/expression/distsql_builtin.go +++ b/expression/distsql_builtin.go @@ -557,7 +557,7 @@ func convertTime(data []byte, ftPB *tipb.FieldType, tz *time.Location) (*Constan if err != nil { return nil, err } - if ft.Tp == mysql.TypeTimestamp && !t.IsZero() { + if ft.Tp == mysql.TypeTimestamp && tz != time.UTC { err = t.ConvertTimeZone(time.UTC, tz) if err != nil { return nil, err diff --git a/expression/expr_to_pb.go b/expression/expr_to_pb.go index 2caecae8cb059..4416b464c1f4c 100644 --- a/expression/expr_to_pb.go +++ b/expression/expr_to_pb.go @@ -106,7 +106,7 @@ func (pc PbConverter) conOrCorColToPBExpr(expr Expression) *tipb.Expr { logutil.Logger(context.Background()).Error("eval constant or correlated column", zap.String("expression", expr.ExplainInfo()), zap.Error(err)) return nil } - tp, val, ok := pc.encodeDatum(d) + tp, val, ok := pc.encodeDatum(ft, d) if !ok { return nil } @@ -117,7 +117,7 @@ func (pc PbConverter) conOrCorColToPBExpr(expr Expression) *tipb.Expr { return &tipb.Expr{Tp: tp, Val: val, FieldType: ToPBFieldType(ft)} } -func (pc *PbConverter) encodeDatum(d types.Datum) (tipb.ExprType, []byte, bool) { +func (pc *PbConverter) encodeDatum(ft *types.FieldType, d types.Datum) (tipb.ExprType, []byte, bool) { var ( tp tipb.ExprType val []byte @@ -157,13 +157,11 @@ func (pc *PbConverter) encodeDatum(d types.Datum) (tipb.ExprType, []byte, bool) case types.KindMysqlTime: if pc.client.IsRequestTypeSupported(kv.ReqTypeDAG, int64(tipb.ExprType_MysqlTime)) { tp = tipb.ExprType_MysqlTime - t := d.GetMysqlTime() - v, err := t.ToPackedUint() + val, err := codec.EncodeMySQLTime(pc.sc, d, ft.Tp, nil) if err != nil { logutil.Logger(context.Background()).Error("encode mysql time", zap.Error(err)) return tp, nil, false } - val = codec.EncodeUint(nil, v) return tp, val, true } return tp, nil, false diff --git a/expression/integration_test.go b/expression/integration_test.go index 33b8f5a8076da..e67e0b84964dc 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -4235,3 +4235,17 @@ where datediff(b.date8, date(from_unixtime(a.starttime))) >= 0` tk.MustQuery(q) } + +func (s *testIntegrationSuite) TestTimestampDatumEncode(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec(`drop table if exists t;`) + tk.MustExec(`create table t (a bigint primary key, b timestamp)`) + tk.MustExec(`insert into t values (1, "2019-04-29 11:56:12")`) + tk.MustQuery(`explain select * from t where b = (select max(b) from t)`).Check(testkit.Rows( + "TableReader_43 10.00 root data:Selection_42", + "└─Selection_42 10.00 cop eq(test.t.b, 2019-04-29 11:56:12)", + " └─TableScan_41 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo", + )) + tk.MustQuery(`select * from t where b = (select max(b) from t)`).Check(testkit.Rows(`1 2019-04-29 11:56:12`)) +} diff --git a/util/codec/codec.go b/util/codec/codec.go index fde8afe1b7bd3..113dabf780ced 100644 --- a/util/codec/codec.go +++ b/util/codec/codec.go @@ -67,21 +67,10 @@ func encode(sc *stmtctx.StatementContext, b []byte, vals []types.Datum, comparab b = encodeBytes(b, vals[i].GetBytes(), comparable) case types.KindMysqlTime: b = append(b, uintFlag) - t := vals[i].GetMysqlTime() - // Encoding timestamp need to consider timezone. - // If it's not in UTC, transform to UTC first. - if t.Type == mysql.TypeTimestamp && sc.TimeZone != time.UTC { - err = t.ConvertTimeZone(sc.TimeZone, time.UTC) - if err != nil { - return nil, errors.Trace(err) - } - } - var v uint64 - v, err = t.ToPackedUint() + b, err = EncodeMySQLTime(sc, vals[i], mysql.TypeUnspecified, b) if err != nil { - return nil, errors.Trace(err) + return nil, err } - b = EncodeUint(b, v) case types.KindMysqlDuration: // duration may have negative value, so we cannot use String to encode directly. b = append(b, durationFlag) @@ -134,6 +123,29 @@ func encode(sc *stmtctx.StatementContext, b []byte, vals []types.Datum, comparab return b, errors.Trace(err) } +// EncodeMySQLTime encodes datum of `KindMysqlTime` to []byte. +func EncodeMySQLTime(sc *stmtctx.StatementContext, d types.Datum, tp byte, b []byte) (_ []byte, err error) { + t := d.GetMysqlTime() + // Encoding timestamp need to consider timezone. If it's not in UTC, transform to UTC first. + // This is compatible with `PBToExpr > convertTime`, and coprocessor assumes the passed timestamp is in UTC as well. + if tp == mysql.TypeUnspecified { + tp = t.Type + } + if tp == mysql.TypeTimestamp && sc.TimeZone != time.UTC { + err = t.ConvertTimeZone(sc.TimeZone, time.UTC) + if err != nil { + return nil, err + } + } + var v uint64 + v, err = t.ToPackedUint() + if err != nil { + return nil, err + } + b = EncodeUint(b, v) + return b, nil +} + func encodeBytes(b []byte, v []byte, comparable bool) []byte { if comparable { b = append(b, bytesFlag) From 667124bd91345fd96941dbe97830de82e39f489a Mon Sep 17 00:00:00 2001 From: Lonng Date: Tue, 30 Apr 2019 17:59:23 +0800 Subject: [PATCH 32/49] executor: refine point get failpoint injection logic (#10319) --- executor/executor_test.go | 25 ++++++++++++++++--------- executor/point_get.go | 24 +++++++++++------------- go.mod | 1 - go.sum | 2 -- 4 files changed, 27 insertions(+), 25 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index 9f24b23c0206a..cce1ea9b35432 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -67,7 +67,6 @@ import ( "github.com/pingcap/tidb/util/testutil" "github.com/pingcap/tidb/util/timeutil" "github.com/pingcap/tipb/go-tipb" - "github.com/tiancaiamao/debugger" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -1876,9 +1875,6 @@ func (s *testSuite) TestIsPointGet(c *C) { } func (s *testSuite) TestPointGetRepeatableRead(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/tidb/executor/pointGetRepeatableReadTest", `return(true)`), IsNil) - defer failpoint.Disable("github.com/pingcap/tidb/executor/pointGetRepeatableReadTest") - tk1 := testkit.NewTestKit(c, s.store) tk1.MustExec("use test") tk1.MustExec(`create table point_get (a int, b int, c int, @@ -1888,19 +1884,30 @@ func (s *testSuite) TestPointGetRepeatableRead(c *C) { tk2 := testkit.NewTestKit(c, s.store) tk2.MustExec("use test") + var ( + step1 = "github.com/pingcap/tidb/executor/pointGetRepeatableReadTest-step1" + step2 = "github.com/pingcap/tidb/executor/pointGetRepeatableReadTest-step2" + ) + + c.Assert(failpoint.Enable(step1, "return"), IsNil) + c.Assert(failpoint.Enable(step2, "pause"), IsNil) + + updateWaitCh := make(chan struct{}) go func() { - ctx := context.WithValue(context.Background(), "pointGetRepeatableReadTest", true) + ctx := context.WithValue(context.Background(), "pointGetRepeatableReadTest", updateWaitCh) + ctx = failpoint.WithHook(ctx, func(ctx context.Context, fpname string) bool { + return fpname == step1 || fpname == step2 + }) rs, err := tk1.Se.Execute(ctx, "select c from point_get where b = 1") c.Assert(err, IsNil) result := tk1.ResultSetToResultWithCtx(ctx, rs[0], Commentf("execute sql fail")) result.Check(testkit.Rows("1")) }() - label := debugger.Bind("point-get-g2") - debugger.Continue("point-get-g1") - debugger.Breakpoint(label) + <-updateWaitCh // Wait `POINT GET` first time `get` + c.Assert(failpoint.Disable(step1), IsNil) tk2.MustExec("update point_get set b = 2, c = 2 where a = 1") - debugger.Continue("point-get-g1") + c.Assert(failpoint.Disable(step2), IsNil) } func (s *testSuite) TestRow(c *C) { diff --git a/executor/point_get.go b/executor/point_get.go index 17f76e6079ae9..9ec774c4621e3 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -29,7 +29,6 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" - "github.com/tiancaiamao/debugger" ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { @@ -91,13 +90,6 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.RecordBatch) err return err1 } - failpoint.Inject("pointGetRepeatableReadTest", func(val failpoint.Value) { - if val.(bool) && ctx.Value("pointGetRepeatableReadTest") != nil { - label := debugger.Bind("point-get-g1") - debugger.Breakpoint(label) - } - }) - handleVal, err1 := e.get(idxKey) if err1 != nil && !kv.ErrNotExist.Equal(err1) { return err1 @@ -110,12 +102,18 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.RecordBatch) err return err1 } - failpoint.Inject("pointGetRepeatableReadTest", func(val failpoint.Value) { - if val.(bool) && ctx.Value("pointGetRepeatableReadTest") != nil { - label := debugger.Bind("point-get-g1") - debugger.Continue("point-get-g2") - debugger.Breakpoint(label) + // The injection is used to simulate following scenario: + // 1. Session A create a point get query but pause before second time `GET` kv from backend + // 2. Session B create an UPDATE query to update the record that will be obtained in step 1 + // 3. Then point get retrieve data from backend after step 2 finished + // 4. Check the result + failpoint.InjectContext(ctx, "pointGetRepeatableReadTest-step1", func() { + if ch, ok := ctx.Value("pointGetRepeatableReadTest").(chan struct{}); ok { + // Make `UPDATE` continue + close(ch) } + // Wait `UPDATE` finished + failpoint.InjectContext(ctx, "pointGetRepeatableReadTest-step2", nil) }) } diff --git a/go.mod b/go.mod index 19a9d81e6ca3c..135aae3a520a1 100644 --- a/go.mod +++ b/go.mod @@ -55,7 +55,6 @@ require ( github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 github.com/struCoder/pidusage v0.1.2 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22 github.com/twinj/uuid v1.0.0 github.com/uber-go/atomic v1.3.2 // indirect github.com/uber/jaeger-client-go v2.15.0+incompatible diff --git a/go.sum b/go.sum index 7cddadde9083e..e80aa9e6cef57 100644 --- a/go.sum +++ b/go.sum @@ -215,8 +215,6 @@ github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d h1:4J9HCZVpvDmj2t github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d/go.mod h1:Z4AUp2Km+PwemOoO/VB5AOx9XSsIItzFjoJlOSiYmn0= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJfDRtkanvQPiooDH8HvJ2FBh+iKT/OmiQQ= github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU= -github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22 h1:P4sgavMKEdqNOws2VfR2c/Bye9nYFgV8gHyiW1wpQhE= -github.com/tiancaiamao/debugger v0.0.0-20190428065433-3a10ffa41d22/go.mod h1:qaShs3uDBYnvaQZJAJ6PjPg8kuAHR9zUJ8ilSLK1y/w= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6 h1:lYIiVDtZnyTWlNwiAxLj0bbpTcx1BWCFhXjfsvmPdNc= github.com/tmc/grpc-websocket-proxy v0.0.0-20171017195756-830351dc03c6/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= From ad20d9250d5d055b3ee418b9214e5cb1a803bf5b Mon Sep 17 00:00:00 2001 From: bb7133 Date: Tue, 30 Apr 2019 18:59:37 +0800 Subject: [PATCH 33/49] expression: fix a compatibility issue in JSON_CONTAINS (#10298) --- expression/builtin_json.go | 14 ++++++ expression/builtin_json_test.go | 84 ++++++++++++++++++++------------- go.mod | 2 +- go.sum | 4 +- 4 files changed, 68 insertions(+), 36 deletions(-) diff --git a/expression/builtin_json.go b/expression/builtin_json.go index 0693a43fde382..462ea88ed0d6a 100644 --- a/expression/builtin_json.go +++ b/expression/builtin_json.go @@ -647,10 +647,24 @@ func (b *builtinJSONContainsSig) Clone() builtinFunc { return newSig } +func (c *jsonContainsFunctionClass) verifyArgs(args []Expression) error { + if err := c.baseFunctionClass.verifyArgs(args); err != nil { + return err + } + if evalType := args[0].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString { + return json.ErrInvalidJSONData.GenWithStackByArgs(1, "json_contains") + } + if evalType := args[1].GetType().EvalType(); evalType != types.ETJson && evalType != types.ETString { + return json.ErrInvalidJSONData.GenWithStackByArgs(2, "json_contains") + } + return nil +} + func (c *jsonContainsFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (builtinFunc, error) { if err := c.verifyArgs(args); err != nil { return nil, err } + argTps := []types.EvalType{types.ETJson, types.ETJson} if len(args) == 3 { argTps = append(argTps, types.ETString) diff --git a/expression/builtin_json_test.go b/expression/builtin_json_test.go index b544e36cc81d8..dba46f94b3afb 100644 --- a/expression/builtin_json_test.go +++ b/expression/builtin_json_test.go @@ -379,50 +379,54 @@ func (s *testEvaluatorSuite) TestJSONContains(c *C) { tbl := []struct { input []interface{} expected interface{} - success bool + err error }{ // Tests nil arguments - {[]interface{}{nil, `1`, "$.c"}, nil, true}, - {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, nil, "$.a[3]"}, nil, true}, - {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, nil}, nil, true}, + {[]interface{}{nil, `1`, "$.c"}, nil, nil}, + {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, nil, "$.a[3]"}, nil, nil}, + {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, nil}, nil, nil}, // Tests with path expression - {[]interface{}{`[1,2,[1,[5,[3]]]]`, `[1,3]`, "$[2]"}, 1, true}, - {[]interface{}{`[1,2,[1,[5,{"a":[2,3]}]]]`, `[1,{"a":[3]}]`, "$[2]"}, 1, true}, - {[]interface{}{`[{"a":1}]`, `{"a":1}`, "$"}, 1, true}, - {[]interface{}{`[{"a":1,"b":2}]`, `{"a":1,"b":2}`, "$"}, 1, true}, - {[]interface{}{`[{"a":{"a":1},"b":2}]`, `{"a":1}`, "$.a"}, 0, true}, + {[]interface{}{`[1,2,[1,[5,[3]]]]`, `[1,3]`, "$[2]"}, 1, nil}, + {[]interface{}{`[1,2,[1,[5,{"a":[2,3]}]]]`, `[1,{"a":[3]}]`, "$[2]"}, 1, nil}, + {[]interface{}{`[{"a":1}]`, `{"a":1}`, "$"}, 1, nil}, + {[]interface{}{`[{"a":1,"b":2}]`, `{"a":1,"b":2}`, "$"}, 1, nil}, + {[]interface{}{`[{"a":{"a":1},"b":2}]`, `{"a":1}`, "$.a"}, 0, nil}, // Tests without path expression - {[]interface{}{`{}`, `{}`}, 1, true}, - {[]interface{}{`{"a":1}`, `{}`}, 1, true}, - {[]interface{}{`{"a":1}`, `1`}, 0, true}, - {[]interface{}{`{"a":[1]}`, `[1]`}, 0, true}, - {[]interface{}{`{"b":2, "c":3}`, `{"c":3}`}, 1, true}, - {[]interface{}{`1`, `1`}, 1, true}, - {[]interface{}{`[1]`, `1`}, 1, true}, - {[]interface{}{`[1,2]`, `[1]`}, 1, true}, - {[]interface{}{`[1,2]`, `[1,3]`}, 0, true}, - {[]interface{}{`[1,2]`, `["1"]`}, 0, true}, - {[]interface{}{`[1,2,[1,3]]`, `[1,3]`}, 1, true}, - {[]interface{}{`[1,2,[1,[5,[3]]]]`, `[1,3]`}, 1, true}, - {[]interface{}{`[1,2,[1,[5,{"a":[2,3]}]]]`, `[1,{"a":[3]}]`}, 1, true}, - {[]interface{}{`[{"a":1}]`, `{"a":1}`}, 1, true}, - {[]interface{}{`[{"a":1,"b":2}]`, `{"a":1}`}, 1, true}, - {[]interface{}{`[{"a":{"a":1},"b":2}]`, `{"a":1}`}, 0, true}, + {[]interface{}{`{}`, `{}`}, 1, nil}, + {[]interface{}{`{"a":1}`, `{}`}, 1, nil}, + {[]interface{}{`{"a":1}`, `1`}, 0, nil}, + {[]interface{}{`{"a":[1]}`, `[1]`}, 0, nil}, + {[]interface{}{`{"b":2, "c":3}`, `{"c":3}`}, 1, nil}, + {[]interface{}{`1`, `1`}, 1, nil}, + {[]interface{}{`[1]`, `1`}, 1, nil}, + {[]interface{}{`[1,2]`, `[1]`}, 1, nil}, + {[]interface{}{`[1,2]`, `[1,3]`}, 0, nil}, + {[]interface{}{`[1,2]`, `["1"]`}, 0, nil}, + {[]interface{}{`[1,2,[1,3]]`, `[1,3]`}, 1, nil}, + {[]interface{}{`[1,2,[1,3]]`, `[1, 3]`}, 1, nil}, + {[]interface{}{`[1,2,[1,[5,[3]]]]`, `[1,3]`}, 1, nil}, + {[]interface{}{`[1,2,[1,[5,{"a":[2,3]}]]]`, `[1,{"a":[3]}]`}, 1, nil}, + {[]interface{}{`[{"a":1}]`, `{"a":1}`}, 1, nil}, + {[]interface{}{`[{"a":1,"b":2}]`, `{"a":1}`}, 1, nil}, + {[]interface{}{`[{"a":{"a":1},"b":2}]`, `{"a":1}`}, 0, nil}, // Tests path expression contains any asterisk - {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.*"}, nil, false}, - {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$[*]"}, nil, false}, - {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$**.a"}, nil, false}, + {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.*"}, nil, json.ErrInvalidJSONPathWildcard}, + {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$[*]"}, nil, json.ErrInvalidJSONPathWildcard}, + {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$**.a"}, nil, json.ErrInvalidJSONPathWildcard}, // Tests path expression does not identify a section of the target document - {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.c"}, nil, true}, - {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.a[3]"}, nil, true}, - {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.a[2].b"}, nil, true}, + {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.c"}, nil, nil}, + {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.a[3]"}, nil, nil}, + {[]interface{}{`{"a": [1, 2, {"aa": "xx"}]}`, `1`, "$.a[2].b"}, nil, nil}, + // For issue 9957: test 'argument 1 and 2 as valid json object' + {[]interface{}{`[1,2,[1,3]]`, `a:1`}, 1, json.ErrInvalidJSONText}, + {[]interface{}{`a:1`, `1`}, 1, json.ErrInvalidJSONText}, } for _, t := range tbl { args := types.MakeDatums(t.input...) f, err := fc.getFunction(s.ctx, s.datumsToConstants(args)) c.Assert(err, IsNil) d, err := evalBuiltinFunc(f, chunk.Row{}) - if t.success { + if t.err == nil { c.Assert(err, IsNil) if t.expected == nil { c.Assert(d.IsNull(), IsTrue) @@ -430,9 +434,23 @@ func (s *testEvaluatorSuite) TestJSONContains(c *C) { c.Assert(d.GetInt64(), Equals, int64(t.expected.(int))) } } else { - c.Assert(err, NotNil) + c.Assert(t.err.(*terror.Error).Equal(err), IsTrue) } } + // For issue 9957: test 'argument 1 and 2 as valid json object' + cases := []struct { + arg1 interface{} + arg2 interface{} + }{ + {1, ""}, + {0.05, ""}, + {"", 1}, + {"", 0.05}, + } + for _, cs := range cases { + _, err := fc.getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(cs.arg1, cs.arg2))) + c.Assert(json.ErrInvalidJSONData.Equal(err), IsTrue) + } } func (s *testEvaluatorSuite) TestJSONContainsPath(c *C) { diff --git a/go.mod b/go.mod index 135aae3a520a1..56b76b68b2666 100644 --- a/go.mod +++ b/go.mod @@ -39,7 +39,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6 + github.com/pingcap/parser v0.0.0-20190429120706-c378059f7f42 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 diff --git a/go.sum b/go.sum index e80aa9e6cef57..f2543ff30249b 100644 --- a/go.sum +++ b/go.sum @@ -160,8 +160,8 @@ github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6 h1:m35H8yhrMcATKp8yUVUHAmvuzIT8xjL+ialXyBrBWHU= -github.com/pingcap/parser v0.0.0-20190429074050-34bcb57d11a6/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190429120706-c378059f7f42 h1:iuZ/y1DLC/4gLTxw/xtEbo9R2SMRq7CKzLXhffff/kc= +github.com/pingcap/parser v0.0.0-20190429120706-c378059f7f42/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= From b6218d415cf77bca6910057a7b391a379c287e59 Mon Sep 17 00:00:00 2001 From: Kolbe Kegel Date: Tue, 30 Apr 2019 09:56:01 -0700 Subject: [PATCH 34/49] Fix TiDB Slack link (#10335) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 0b98df9ea9d9b..c587c8bb911c4 100644 --- a/README.md +++ b/README.md @@ -10,7 +10,7 @@ [![Coverage Status](https://codecov.io/gh/pingcap/tidb/branch/master/graph/badge.svg)](https://codecov.io/gh/pingcap/tidb) - [**Stack Overflow**](https://stackoverflow.com/questions/tagged/tidb) -- Community [**Slack Channel**](http://bit.ly/tidbslack) +- Community [**Slack Channel**](https://join.slack.com/t/tidbcommunity/shared_invite/enQtNjIyNjA5Njk0NTAxLTVmZDkxOWY1ZGZhMDg3YzcwNGU0YmM4ZjIyODRhOTg4MWEwZjJmMGQzZTJlNjllMGY1YzdlNzIxZGE2NzRlMGY) - **Twitter**: [@PingCAP](https://twitter.com/PingCAP) - [**Reddit**](https://www.reddit.com/r/TiDB/) - **Mailing list**: [Google Group](https://groups.google.com/forum/#!forum/tidb-user) From 6f67214b50fbb5a2d6ef3866a702f1d7dde94898 Mon Sep 17 00:00:00 2001 From: Lynn Date: Sun, 5 May 2019 10:58:21 +0800 Subject: [PATCH 35/49] kv: remove the useless field of `snapshot` (#10276) --- kv/union_store.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/kv/union_store.go b/kv/union_store.go index e2d5e3fbc23fc..ccf0a72a91cd5 100644 --- a/kv/union_store.go +++ b/kv/union_store.go @@ -71,7 +71,6 @@ func (c *conditionPair) Err() error { // snapshot for read. type unionStore struct { *BufferStore - snapshot Snapshot // for read lazyConditionPairs map[string]*conditionPair // for delay check opts options } @@ -80,7 +79,6 @@ type unionStore struct { func NewUnionStore(snapshot Snapshot) UnionStore { return &unionStore{ BufferStore: NewBufferStore(snapshot, DefaultTxnMembufCap), - snapshot: snapshot, lazyConditionPairs: make(map[string]*conditionPair), opts: make(map[Option]interface{}), } From 71d15aa5a3a4bcb47a0111025704d45d0b46b844 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Sun, 5 May 2019 15:15:03 +0800 Subject: [PATCH 36/49] make tidb_disable_txn_auto_retry as it used to be (#10339) Signed-off-by: Shuaipeng Yu --- session/session.go | 2 +- session/session_test.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/session/session.go b/session/session.go index e220ed42def4f..e25198f8b57e5 100644 --- a/session/session.go +++ b/session/session.go @@ -405,7 +405,7 @@ func (s *session) doCommitWithRetry(ctx context.Context) error { // For autocommit single statement transactions, the history count is always 1. // For explicit transactions, the statement count is more than 1. history := GetHistory(s) - if history.Count() > 1 && strings.Contains(err.Error(), util.WriteConflictMarker) { + if history.Count() > 1 { commitRetryLimit = 0 } } diff --git a/session/session_test.go b/session/session_test.go index 714001eac15c7..93d51601e17f0 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -2421,12 +2421,12 @@ func (s *testSchemaSuite) TestDisableTxnAutoRetry(c *C) { tk2.MustExec("alter table no_retry add index idx(id)") tk2.MustQuery("select * from no_retry").Check(testkit.Rows("8")) tk1.MustExec("update no_retry set id = 10") - tk1.MustExec("commit") - tk2.MustQuery("select * from no_retry").Check(testkit.Rows("10")) + _, err = tk1.Se.Execute(context.Background(), "commit") + c.Assert(err, NotNil) // set autocommit to begin and commit tk1.MustExec("set autocommit = 0") - tk1.MustQuery("select * from no_retry").Check(testkit.Rows("10")) + tk1.MustQuery("select * from no_retry").Check(testkit.Rows("8")) tk2.MustExec("update no_retry set id = 11") tk1.MustExec("update no_retry set id = 12") _, err = tk1.Se.Execute(context.Background(), "set autocommit = 1") From 68e7b00bd12e33247c1091c22f6c86acbec0a270 Mon Sep 17 00:00:00 2001 From: Zhang Jian Date: Sun, 5 May 2019 17:38:51 +0800 Subject: [PATCH 37/49] chunk: change offset type to int64 (#10348) --- util/chunk/chunk.go | 6 +++--- util/chunk/chunk_test.go | 4 ++-- util/chunk/chunk_util.go | 4 ++-- util/chunk/codec.go | 28 ++++++++++++++-------------- util/chunk/column.go | 4 ++-- util/chunk/column_test.go | 11 ++++++++++- util/chunk/mutrow.go | 10 +++++----- 7 files changed, 38 insertions(+), 29 deletions(-) diff --git a/util/chunk/chunk.go b/util/chunk/chunk.go index b44fef2c71ef8..5a375814b7a18 100644 --- a/util/chunk/chunk.go +++ b/util/chunk/chunk.go @@ -139,7 +139,7 @@ func newVarLenColumn(cap int, old *column) *column { estimatedElemLen = (len(old.data) + len(old.data)/8) / old.length } return &column{ - offsets: make([]int32, 1, cap+1), + offsets: make([]int64, 1, cap+1), data: make([]byte, 0, cap*estimatedElemLen), nullBitmap: make([]byte, 0, cap>>3), } @@ -318,7 +318,7 @@ func (c *Chunk) AppendPartialRow(colIdx int, row Row) { } else { start, end := rowCol.offsets[row.idx], rowCol.offsets[row.idx+1] chkCol.data = append(chkCol.data, rowCol.data[start:end]...) - chkCol.offsets = append(chkCol.offsets, int32(len(chkCol.data))) + chkCol.offsets = append(chkCol.offsets, int64(len(chkCol.data))) } chkCol.length++ } @@ -342,7 +342,7 @@ func (c *Chunk) PreAlloc(row Row) (rowIdx uint32) { elemLen := len(srcCol.elemBuf) if !srcCol.isFixed() { elemLen = int(srcCol.offsets[row.idx+1] - srcCol.offsets[row.idx]) - dstCol.offsets = append(dstCol.offsets, int32(len(dstCol.data)+elemLen)) + dstCol.offsets = append(dstCol.offsets, int64(len(dstCol.data)+elemLen)) } dstCol.length++ needCap := len(dstCol.data) + elemLen diff --git a/util/chunk/chunk_test.go b/util/chunk/chunk_test.go index 07fc85f6f453d..704ad3fe93fdd 100644 --- a/util/chunk/chunk_test.go +++ b/util/chunk/chunk_test.go @@ -169,7 +169,7 @@ func (s *testChunkSuite) TestAppend(c *check.C) { c.Assert(dst.columns[1].length, check.Equals, 12) c.Assert(dst.columns[1].nullCount, check.Equals, 6) c.Assert(string(dst.columns[0].nullBitmap), check.Equals, string([]byte{0x55, 0x05})) - c.Assert(string(dst.columns[1].offsets), check.Equals, string([]int32{0, 3, 3, 6, 6, 9, 9, 12, 12, 15, 15, 18, 18})) + c.Assert(fmt.Sprintf("%v", dst.columns[1].offsets), check.Equals, fmt.Sprintf("%v", []int64{0, 3, 3, 6, 6, 9, 9, 12, 12, 15, 15, 18, 18})) c.Assert(string(dst.columns[1].data), check.Equals, "abcabcabcabcabcabc") c.Assert(len(dst.columns[1].elemBuf), check.Equals, 0) @@ -222,7 +222,7 @@ func (s *testChunkSuite) TestTruncateTo(c *check.C) { c.Assert(src.columns[1].length, check.Equals, 12) c.Assert(src.columns[1].nullCount, check.Equals, 6) c.Assert(string(src.columns[0].nullBitmap), check.Equals, string([]byte{0x55, 0x05})) - c.Assert(string(src.columns[1].offsets), check.Equals, string([]int32{0, 3, 3, 6, 6, 9, 9, 12, 12, 15, 15, 18, 18})) + c.Assert(fmt.Sprintf("%v", src.columns[1].offsets), check.Equals, fmt.Sprintf("%v", []int64{0, 3, 3, 6, 6, 9, 9, 12, 12, 15, 15, 18, 18})) c.Assert(string(src.columns[1].data), check.Equals, "abcabcabcabcabcabc") c.Assert(len(src.columns[1].elemBuf), check.Equals, 0) diff --git a/util/chunk/chunk_util.go b/util/chunk/chunk_util.go index 71b0f88768575..ea2da90585a7e 100644 --- a/util/chunk/chunk_util.go +++ b/util/chunk/chunk_util.go @@ -64,7 +64,7 @@ func copySelectedInnerRows(innerColOffset, outerColOffset int, src *Chunk, selec start, end := srcCol.offsets[i], srcCol.offsets[i+1] dstCol.data = append(dstCol.data, srcCol.data[start:end]...) - dstCol.offsets = append(dstCol.offsets, int32(len(dstCol.data))) + dstCol.offsets = append(dstCol.offsets, int64(len(dstCol.data))) } } } @@ -99,7 +99,7 @@ func copyOuterRows(innerColOffset, outerColOffset int, src *Chunk, numRows int, offsets := dstCol.offsets elemLen := srcCol.offsets[row.idx+1] - srcCol.offsets[row.idx] for j := 0; j < numRows; j++ { - offsets = append(offsets, int32(offsets[len(offsets)-1]+elemLen)) + offsets = append(offsets, int64(offsets[len(offsets)-1]+elemLen)) } dstCol.offsets = offsets } diff --git a/util/chunk/codec.go b/util/chunk/codec.go index 43a5f5b17327d..4458dbe76e1bc 100644 --- a/util/chunk/codec.go +++ b/util/chunk/codec.go @@ -65,8 +65,8 @@ func (c *Codec) encodeColumn(buffer []byte, col *column) []byte { // encode offsets. if !col.isFixed() { - numOffsetBytes := (col.length + 1) * 4 - offsetBytes := c.i32SliceToBytes(col.offsets) + numOffsetBytes := (col.length + 1) * 8 + offsetBytes := c.i64SliceToBytes(col.offsets) buffer = append(buffer, offsetBytes[:numOffsetBytes]...) } @@ -75,14 +75,14 @@ func (c *Codec) encodeColumn(buffer []byte, col *column) []byte { return buffer } -func (c *Codec) i32SliceToBytes(i32s []int32) (b []byte) { - if len(i32s) == 0 { +func (c *Codec) i64SliceToBytes(i64s []int64) (b []byte) { + if len(i64s) == 0 { return nil } hdr := (*reflect.SliceHeader)(unsafe.Pointer(&b)) - hdr.Len = len(i32s) * 4 + hdr.Len = len(i64s) * 8 hdr.Cap = hdr.Len - hdr.Data = uintptr(unsafe.Pointer(&i32s[0])) + hdr.Data = uintptr(unsafe.Pointer(&i64s[0])) return b } @@ -125,12 +125,12 @@ func (c *Codec) decodeColumn(buffer []byte, col *column, ordinal int) (remained // decode offsets. numFixedBytes := getFixedLen(c.colTypes[ordinal]) - numDataBytes := numFixedBytes * col.length + numDataBytes := int64(numFixedBytes * col.length) if numFixedBytes == -1 { - numOffsetBytes := (col.length + 1) * 4 - col.offsets = append(col.offsets[:0], c.bytesToI32Slice(buffer[:numOffsetBytes])...) + numOffsetBytes := (col.length + 1) * 8 + col.offsets = append(col.offsets[:0], c.bytesToI64Slice(buffer[:numOffsetBytes])...) buffer = buffer[numOffsetBytes:] - numDataBytes = int(col.offsets[col.length]) + numDataBytes = col.offsets[col.length] } else if cap(col.elemBuf) < numFixedBytes { col.elemBuf = make([]byte, numFixedBytes) } @@ -152,15 +152,15 @@ func (c *Codec) setAllNotNull(col *column) { } } -func (c *Codec) bytesToI32Slice(b []byte) (i32s []int32) { +func (c *Codec) bytesToI64Slice(b []byte) (i64s []int64) { if len(b) == 0 { return nil } - hdr := (*reflect.SliceHeader)(unsafe.Pointer(&i32s)) - hdr.Len = len(b) / 4 + hdr := (*reflect.SliceHeader)(unsafe.Pointer(&i64s)) + hdr.Len = len(b) / 8 hdr.Cap = hdr.Len hdr.Data = uintptr(unsafe.Pointer(&b[0])) - return i32s + return i64s } // varElemLen indicates this column is a variable length column. diff --git a/util/chunk/column.go b/util/chunk/column.go index 9b98afdfc1d9a..39cac8de0fb63 100644 --- a/util/chunk/column.go +++ b/util/chunk/column.go @@ -47,7 +47,7 @@ type column struct { length int nullCount int nullBitmap []byte - offsets []int32 + offsets []int64 data []byte elemBuf []byte } @@ -158,7 +158,7 @@ func (c *column) appendFloat64(f float64) { func (c *column) finishAppendVar() { c.appendNullBitmap(true) - c.offsets = append(c.offsets, int32(len(c.data))) + c.offsets = append(c.offsets, int64(len(c.data))) c.length++ } diff --git a/util/chunk/column_test.go b/util/chunk/column_test.go index f6f1075620ea9..4b209569fd4eb 100644 --- a/util/chunk/column_test.go +++ b/util/chunk/column_test.go @@ -13,7 +13,9 @@ package chunk -import "github.com/pingcap/check" +import ( + "github.com/pingcap/check" +) func equalColumn(c1, c2 *column) bool { if c1.length != c2.length || @@ -58,3 +60,10 @@ func (s *testChunkSuite) TestColumnCopy(c *check.C) { c1 := col.copyConstruct() c.Check(equalColumn(col, c1), check.IsTrue) } + +func (s *testChunkSuite) TestLargeStringColumnOffset(c *check.C) { + numRows := 1 + col := newVarLenColumn(numRows, nil) + col.offsets[0] = 6 << 30 + c.Check(col.offsets[0], check.Equals, int64(6<<30)) // test no overflow. +} diff --git a/util/chunk/mutrow.go b/util/chunk/mutrow.go index 9bb65e4e1b484..15b5f29999db5 100644 --- a/util/chunk/mutrow.go +++ b/util/chunk/mutrow.go @@ -178,7 +178,7 @@ func newMutRowVarLenColumn(valSize int) *column { buf := make([]byte, valSize+1) col := &column{ length: 1, - offsets: []int32{0, int32(valSize)}, + offsets: []int64{0, int64(valSize)}, data: buf[:valSize], nullBitmap: buf[valSize:], } @@ -314,7 +314,7 @@ func setMutRowBytes(col *column, bin []byte) { col.nullBitmap = buf[len(bin):] } copy(col.data, bin) - col.offsets[1] = int32(len(bin)) + col.offsets[1] = int64(len(bin)) } func setMutRowNameValue(col *column, name string, val uint64) { @@ -328,7 +328,7 @@ func setMutRowNameValue(col *column, name string, val uint64) { } binary.LittleEndian.PutUint64(col.data, val) copy(col.data[8:], name) - col.offsets[1] = int32(dataLen) + col.offsets[1] = int64(dataLen) } func setMutRowJSON(col *column, j json.BinaryJSON) { @@ -344,7 +344,7 @@ func setMutRowJSON(col *column, j json.BinaryJSON) { } col.data[0] = j.TypeCode copy(col.data[1:], j.Value) - col.offsets[1] = int32(dataLen) + col.offsets[1] = int64(dataLen) } // ShallowCopyPartialRow shallow copies the data of `row` to MutRow. @@ -365,7 +365,7 @@ func (mr MutRow) ShallowCopyPartialRow(colIdx int, row Row) { } else { start, end := srcCol.offsets[row.idx], srcCol.offsets[row.idx+1] dstCol.data = srcCol.data[start:end] - dstCol.offsets[1] = int32(len(dstCol.data)) + dstCol.offsets[1] = int64(len(dstCol.data)) } } } From c9f0a1c44f8913f85da7c97801822b2f9d70a6c5 Mon Sep 17 00:00:00 2001 From: Song Guo Date: Sun, 5 May 2019 19:17:45 +0800 Subject: [PATCH 38/49] types: Fix checkMonthDay() to correctly handle leap year (#10342) --- types/time.go | 6 +++--- types/time_test.go | 40 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 43 insertions(+), 3 deletions(-) diff --git a/types/time.go b/types/time.go index f9b2083842840..25f24b95eaf5c 100644 --- a/types/time.go +++ b/types/time.go @@ -1474,7 +1474,7 @@ func checkDateRange(t MysqlTime) error { func checkMonthDay(year, month, day int, allowInvalidDate bool) error { if month < 0 || month > 12 { - return errors.Trace(ErrInvalidTimeFormat.GenWithStackByArgs(month)) + return errors.Trace(ErrIncorrectDatetimeValue.GenWithStackByArgs(month)) } maxDay := 31 @@ -1482,13 +1482,13 @@ func checkMonthDay(year, month, day int, allowInvalidDate bool) error { if month > 0 { maxDay = maxDaysInMonth[month-1] } - if month == 2 && year%4 != 0 { + if month == 2 && !isLeapYear(uint16(year)) { maxDay = 28 } } if day < 0 || day > maxDay { - return errors.Trace(ErrInvalidTimeFormat.GenWithStackByArgs(day)) + return errors.Trace(ErrIncorrectDatetimeValue.GenWithStackByArgs(day)) } return nil } diff --git a/types/time_test.go b/types/time_test.go index c23c0032204c6..28aea65741579 100644 --- a/types/time_test.go +++ b/types/time_test.go @@ -1455,3 +1455,43 @@ func (s *testTimeSuite) TestTimeSub(c *C) { c.Assert(rec, Equals, dur) } } + +func (s *testTimeSuite) TestCheckMonthDay(c *C) { + dates := []struct { + date types.MysqlTime + isValidDate bool + }{ + {types.FromDate(1900, 2, 29, 0, 0, 0, 0), false}, + {types.FromDate(1900, 2, 28, 0, 0, 0, 0), true}, + {types.FromDate(2000, 2, 29, 0, 0, 0, 0), true}, + {types.FromDate(2000, 1, 1, 0, 0, 0, 0), true}, + {types.FromDate(1900, 1, 1, 0, 0, 0, 0), true}, + {types.FromDate(1900, 1, 31, 0, 0, 0, 0), true}, + {types.FromDate(1900, 4, 1, 0, 0, 0, 0), true}, + {types.FromDate(1900, 4, 31, 0, 0, 0, 0), false}, + {types.FromDate(1900, 4, 30, 0, 0, 0, 0), true}, + {types.FromDate(2000, 2, 30, 0, 0, 0, 0), false}, + {types.FromDate(2000, 13, 1, 0, 0, 0, 0), false}, + {types.FromDate(4000, 2, 29, 0, 0, 0, 0), true}, + {types.FromDate(3200, 2, 29, 0, 0, 0, 0), true}, + } + + sc := &stmtctx.StatementContext{ + TimeZone: time.UTC, + AllowInvalidDate: false, + } + + for _, t := range dates { + tt := types.Time{ + Time: t.date, + Type: mysql.TypeDate, + Fsp: types.DefaultFsp, + } + err := tt.Check(sc) + if t.isValidDate { + c.Check(err, IsNil) + } else { + c.Check(types.ErrIncorrectDatetimeValue.Equal(err), IsTrue) + } + } +} From bbe1e690b41f7d380230f6ef4f5f77f6f6aa8664 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Sun, 5 May 2019 22:06:49 +0800 Subject: [PATCH 39/49] expression: fix issue that `timestampadd` is not compatible with Mysql (#10314) --- expression/builtin_time.go | 19 ++++++++++++++++++- expression/integration_test.go | 12 ++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index ba08265945dc6..6a442b8d2d08a 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -5500,13 +5500,30 @@ func (b *builtinTimestampAddSig) evalString(row chunk.Row) (string, bool, error) default: return "", true, types.ErrInvalidTimeFormat.GenWithStackByArgs(unit) } - r := types.Time{Time: types.FromGoTime(tb), Type: mysql.TypeDatetime, Fsp: fsp} + r := types.Time{Time: types.FromGoTime(tb), Type: b.resolveType(arg.Type, unit), Fsp: fsp} if err = r.Check(b.ctx.GetSessionVars().StmtCtx); err != nil { return "", true, handleInvalidTimeError(b.ctx, err) } return r.String(), false, nil } +func (b *builtinTimestampAddSig) resolveType(typ uint8, unit string) uint8 { + // The approach below is from MySQL. + // The field type for the result of an Item_date function is defined as + // follows: + // + //- If first arg is a MYSQL_TYPE_DATETIME result is MYSQL_TYPE_DATETIME + //- If first arg is a MYSQL_TYPE_DATE and the interval type uses hours, + // minutes, seconds or microsecond then type is MYSQL_TYPE_DATETIME. + //- Otherwise the result is MYSQL_TYPE_STRING + // (This is because you can't know if the string contains a DATE, MYSQL_TIME + // or DATETIME argument) + if typ == mysql.TypeDate && (unit == "HOUR" || unit == "MINUTE" || unit == "SECOND" || unit == "MICROSECOND") { + return mysql.TypeDatetime + } + return typ +} + type toDaysFunctionClass struct { baseFunctionClass } diff --git a/expression/integration_test.go b/expression/integration_test.go index e67e0b84964dc..ed9c94ae73d23 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -1342,6 +1342,18 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { result.Check(testkit.Rows("2017-01-18 12:39:50.123 2017-01-18 12:39:50.999")) result = tk.MustQuery("select timestamp('2003-12-31', '01:01:01.01'), timestamp('2003-12-31 12:34', '01:01:01.01')," + " timestamp('2008-12-31','00:00:00.0'), timestamp('2008-12-31 00:00:00.000');") + + tk.MustQuery(`select timestampadd(second, 1, cast("2001-01-01" as date))`).Check(testkit.Rows("2001-01-01 00:00:01")) + tk.MustQuery(`select timestampadd(hour, 1, cast("2001-01-01" as date))`).Check(testkit.Rows("2001-01-01 01:00:00")) + tk.MustQuery(`select timestampadd(day, 1, cast("2001-01-01" as date))`).Check(testkit.Rows("2001-01-02")) + tk.MustQuery(`select timestampadd(month, 1, cast("2001-01-01" as date))`).Check(testkit.Rows("2001-02-01")) + tk.MustQuery(`select timestampadd(year, 1, cast("2001-01-01" as date))`).Check(testkit.Rows("2002-01-01")) + tk.MustQuery(`select timestampadd(second, 1, cast("2001-01-01" as datetime))`).Check(testkit.Rows("2001-01-01 00:00:01")) + tk.MustQuery(`select timestampadd(hour, 1, cast("2001-01-01" as datetime))`).Check(testkit.Rows("2001-01-01 01:00:00")) + tk.MustQuery(`select timestampadd(day, 1, cast("2001-01-01" as datetime))`).Check(testkit.Rows("2001-01-02 00:00:00")) + tk.MustQuery(`select timestampadd(month, 1, cast("2001-01-01" as datetime))`).Check(testkit.Rows("2001-02-01 00:00:00")) + tk.MustQuery(`select timestampadd(year, 1, cast("2001-01-01" as datetime))`).Check(testkit.Rows("2002-01-01 00:00:00")) + result.Check(testkit.Rows("2003-12-31 01:01:01.01 2003-12-31 13:35:01.01 2008-12-31 00:00:00.0 2008-12-31 00:00:00.000")) result = tk.MustQuery("select timestamp('2003-12-31', 1), timestamp('2003-12-31', -1);") result.Check(testkit.Rows("2003-12-31 00:00:01 2003-12-30 23:59:59")) From 62d37a8356964aa3c3179d88360e82a4294b1fa7 Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Sun, 5 May 2019 22:11:38 +0800 Subject: [PATCH 40/49] executor: fix the error message when granting to a non-exist user (#10239) --- executor/errors.go | 1 + executor/grant.go | 2 +- executor/grant_test.go | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/executor/errors.go b/executor/errors.go index d1fea0047ecd5..0e6ad100c1d4d 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -39,6 +39,7 @@ var ( ErrBuildExecutor = terror.ClassExecutor.New(codeErrBuildExec, "Failed to build executor") ErrBatchInsertFail = terror.ClassExecutor.New(codeBatchInsertFail, "Batch insert failed, please clean the table and try again.") + ErrCantCreateUserWithGrant = terror.ClassExecutor.New(mysql.ErrCantCreateUserWithGrant, mysql.MySQLErrName[mysql.ErrCantCreateUserWithGrant]) ErrPasswordNoMatch = terror.ClassExecutor.New(mysql.ErrPasswordNoMatch, mysql.MySQLErrName[mysql.ErrPasswordNoMatch]) ErrCannotUser = terror.ClassExecutor.New(mysql.ErrCannotUser, mysql.MySQLErrName[mysql.ErrCannotUser]) ErrPasswordFormat = terror.ClassExecutor.New(mysql.ErrPasswordFormat, mysql.MySQLErrName[mysql.ErrPasswordFormat]) diff --git a/executor/grant.go b/executor/grant.go index 29335a855994f..cd3d8065b6591 100644 --- a/executor/grant.go +++ b/executor/grant.go @@ -71,7 +71,7 @@ func (e *GrantExec) Next(ctx context.Context, req *chunk.RecordBatch) error { return err } if !exists && e.ctx.GetSessionVars().SQLMode.HasNoAutoCreateUserMode() { - return ErrPasswordNoMatch + return ErrCantCreateUserWithGrant } else if !exists { pwd, ok := user.EncodedPassword() if !ok { diff --git a/executor/grant_test.go b/executor/grant_test.go index 01085eb073f86..0016e0b96c6f1 100644 --- a/executor/grant_test.go +++ b/executor/grant_test.go @@ -189,7 +189,7 @@ func (s *testSuite3) TestNoAutoCreateUser(c *C) { tk.MustExec(`SET sql_mode='NO_AUTO_CREATE_USER'`) _, err := tk.Exec(`GRANT ALL PRIVILEGES ON *.* to 'test'@'%' IDENTIFIED BY 'xxx'`) c.Check(err, NotNil) - c.Assert(terror.ErrorEqual(err, executor.ErrPasswordNoMatch), IsTrue) + c.Assert(terror.ErrorEqual(err, executor.ErrCantCreateUserWithGrant), IsTrue) } func (s *testSuite3) TestCreateUserWhenGrant(c *C) { From 0b037aadf66cbe56bdae726279305fa6d2a56fbb Mon Sep 17 00:00:00 2001 From: Gunjan Patidar Date: Sun, 5 May 2019 20:58:42 +0530 Subject: [PATCH 41/49] Fix incorrect "Internal" display in admin show slow output (#10338) --- executor/executor.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index cc0da1bdddf46..eb32d3cc17f2b 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -617,9 +617,9 @@ func (e *ShowSlowExec) Next(ctx context.Context, req *chunk.RecordBatch) error { req.AppendString(9, slow.TableIDs) req.AppendString(10, slow.IndexIDs) if slow.Internal { - req.AppendInt64(11, 0) - } else { req.AppendInt64(11, 1) + } else { + req.AppendInt64(11, 0) } req.AppendString(12, slow.Digest) e.cursor++ From c9cc3b7e327760174011d78b5b845ef0eb632b86 Mon Sep 17 00:00:00 2001 From: "Ian P. Cooke" Date: Sun, 5 May 2019 19:29:23 -0500 Subject: [PATCH 42/49] executor: support prepare DDL statements with no parameters (#10144) --- executor/errors.go | 2 +- executor/prepared.go | 8 +++++--- executor/prepared_test.go | 8 ++++++++ 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/executor/errors.go b/executor/errors.go index 0e6ad100c1d4d..b7f8ce2ebf19f 100644 --- a/executor/errors.go +++ b/executor/errors.go @@ -34,7 +34,7 @@ var ( ErrGetStartTS = terror.ClassExecutor.New(codeGetStartTS, "Can not get start ts") ErrUnknownPlan = terror.ClassExecutor.New(codeUnknownPlan, "Unknown plan") ErrPrepareMulti = terror.ClassExecutor.New(codePrepareMulti, "Can not prepare multiple statements") - ErrPrepareDDL = terror.ClassExecutor.New(codePrepareDDL, "Can not prepare DDL statements") + ErrPrepareDDL = terror.ClassExecutor.New(codePrepareDDL, "Can not prepare DDL statements with parameters") ErrResultIsEmpty = terror.ClassExecutor.New(codeResultIsEmpty, "result is empty") ErrBuildExecutor = terror.ClassExecutor.New(codeErrBuildExec, "Failed to build executor") ErrBatchInsertFail = terror.ClassExecutor.New(codeBatchInsertFail, "Batch insert failed, please clean the table and try again.") diff --git a/executor/prepared.go b/executor/prepared.go index ecbcaede4753e..bfdb1cc6611f3 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -133,9 +133,6 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.RecordBatch) error { return ErrPrepareMulti } stmt := stmts[0] - if _, ok := stmt.(ast.DDLNode); ok { - return ErrPrepareDDL - } err = ResetContextOfStmt(e.ctx, stmt) if err != nil { return err @@ -143,6 +140,11 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.RecordBatch) error { var extractor paramMarkerExtractor stmt.Accept(&extractor) + // DDL Statements can not accept parameters + if _, ok := stmt.(ast.DDLNode); ok && len(extractor.markers) > 0 { + return ErrPrepareDDL + } + // Prepare parameters should NOT over 2 bytes(MaxUint16) // https://dev.mysql.com/doc/internals/en/com-stmt-prepare-response.html#packet-COM_STMT_PREPARE_OK. if len(extractor.markers) > math.MaxUint16 { diff --git a/executor/prepared_test.go b/executor/prepared_test.go index 1eb64770f412f..82286e5c60e9a 100644 --- a/executor/prepared_test.go +++ b/executor/prepared_test.go @@ -30,3 +30,11 @@ func (s *testSuite1) TestPreparedNameResolver(c *C) { _, err = tk.Exec("prepare stmt from '(select * FROM t) union all (select * FROM t) order by a limit ?'") c.Assert(err.Error(), Equals, "[planner:1054]Unknown column 'a' in 'order clause'") } + +// a 'create table' DDL statement should be accepted if it has no parameters. +func (s *testSuite1) TestPreparedDDL(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("prepare stmt from 'create table t (id int, KEY id (id))'") +} From 303f3c90405260bdc6cfd2d746d37b79d2df785d Mon Sep 17 00:00:00 2001 From: Lynn Date: Mon, 6 May 2019 10:43:34 +0800 Subject: [PATCH 43/49] domain: fix updating the self schema version and update log (#10324) --- domain/domain.go | 45 ++++++++++++++++++++++++++----------------- domain/domain_test.go | 16 +++++++++++++-- 2 files changed, 41 insertions(+), 20 deletions(-) diff --git a/domain/domain.go b/domain/domain.go index 8c12fc9e8b8f9..6f9fde124f0cd 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -83,29 +83,36 @@ func (do *Domain) loadInfoSchema(handle *infoschema.Handle, usedSchemaVersion in return 0, nil, fullLoad, err } m := meta.NewSnapshotMeta(snapshot) - latestSchemaVersion, err := m.GetSchemaVersion() + neededSchemaVersion, err := m.GetSchemaVersion() if err != nil { return 0, nil, fullLoad, err } - if usedSchemaVersion != 0 && usedSchemaVersion == latestSchemaVersion { - return latestSchemaVersion, nil, fullLoad, nil + if usedSchemaVersion != 0 && usedSchemaVersion == neededSchemaVersion { + return neededSchemaVersion, nil, fullLoad, nil } // Update self schema version to etcd. defer func() { - if err != nil { - logutil.Logger(context.Background()).Info("cannot update self schema version to etcd") + // There are two possibilities for not updating the self schema version to etcd. + // 1. Failed to loading schema information. + // 2. When users use history read feature, the neededSchemaVersion isn't the latest schema version. + if err != nil || neededSchemaVersion < do.InfoSchema().SchemaMetaVersion() { + logutil.Logger(context.Background()).Info("do not update self schema version to etcd", + zap.Int64("usedSchemaVersion", usedSchemaVersion), + zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Error(err)) return } - err = do.ddl.SchemaSyncer().UpdateSelfVersion(context.Background(), latestSchemaVersion) + + err = do.ddl.SchemaSyncer().UpdateSelfVersion(context.Background(), neededSchemaVersion) if err != nil { - logutil.Logger(context.Background()).Info("update self version failed", zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("latestSchemaVersion", latestSchemaVersion), zap.Error(err)) + logutil.Logger(context.Background()).Info("update self version failed", + zap.Int64("usedSchemaVersion", usedSchemaVersion), + zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Error(err)) } }() startTime := time.Now() - ok, tblIDs, err := do.tryLoadSchemaDiffs(m, usedSchemaVersion, latestSchemaVersion) + ok, tblIDs, err := do.tryLoadSchemaDiffs(m, usedSchemaVersion, neededSchemaVersion) if err != nil { // We can fall back to full load, don't need to return the error. logutil.Logger(context.Background()).Error("failed to load schema diff", zap.Error(err)) @@ -113,10 +120,10 @@ func (do *Domain) loadInfoSchema(handle *infoschema.Handle, usedSchemaVersion in if ok { logutil.Logger(context.Background()).Info("diff load InfoSchema success", zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("latestSchemaVersion", latestSchemaVersion), + zap.Int64("neededSchemaVersion", neededSchemaVersion), zap.Duration("start time", time.Since(startTime)), zap.Int64s("tblIDs", tblIDs)) - return latestSchemaVersion, tblIDs, fullLoad, nil + return neededSchemaVersion, tblIDs, fullLoad, nil } fullLoad = true @@ -125,14 +132,16 @@ func (do *Domain) loadInfoSchema(handle *infoschema.Handle, usedSchemaVersion in return 0, nil, fullLoad, err } - newISBuilder, err := infoschema.NewBuilder(handle).InitWithDBInfos(schemas, latestSchemaVersion) + newISBuilder, err := infoschema.NewBuilder(handle).InitWithDBInfos(schemas, neededSchemaVersion) if err != nil { return 0, nil, fullLoad, err } - logutil.Logger(context.Background()).Info("full load InfoSchema success", zap.Int64("usedSchemaVersion", usedSchemaVersion), - zap.Int64("latestSchemaVersion", latestSchemaVersion), zap.Duration("start time", time.Since(startTime))) + logutil.Logger(context.Background()).Info("full load InfoSchema success", + zap.Int64("usedSchemaVersion", usedSchemaVersion), + zap.Int64("neededSchemaVersion", neededSchemaVersion), + zap.Duration("start time", time.Since(startTime))) newISBuilder.Build() - return latestSchemaVersion, nil, fullLoad, nil + return neededSchemaVersion, nil, fullLoad, nil } func (do *Domain) fetchAllSchemasWithTables(m *meta.Meta) ([]*model.DBInfo, error) { @@ -310,7 +319,7 @@ func (do *Domain) Reload() error { startTime := time.Now() var err error - var latestSchemaVersion int64 + var neededSchemaVersion int64 ver, err := do.store.CurrentVersion() if err != nil { @@ -327,7 +336,7 @@ func (do *Domain) Reload() error { fullLoad bool changedTableIDs []int64 ) - latestSchemaVersion, changedTableIDs, fullLoad, err = do.loadInfoSchema(do.infoHandle, schemaVersion, ver.Ver) + neededSchemaVersion, changedTableIDs, fullLoad, err = do.loadInfoSchema(do.infoHandle, schemaVersion, ver.Ver) metrics.LoadSchemaDuration.Observe(time.Since(startTime).Seconds()) if err != nil { metrics.LoadSchemaCounter.WithLabelValues("failed").Inc() @@ -339,7 +348,7 @@ func (do *Domain) Reload() error { logutil.Logger(context.Background()).Info("full load and reset schema validator") do.SchemaValidator.Reset() } - do.SchemaValidator.Update(ver.Ver, schemaVersion, latestSchemaVersion, changedTableIDs) + do.SchemaValidator.Update(ver.Ver, schemaVersion, neededSchemaVersion, changedTableIDs) lease := do.DDL().GetLease() sub := time.Since(startTime) diff --git a/domain/domain_test.go b/domain/domain_test.go index 5c706a0a0523c..6b809aa2f217e 100644 --- a/domain/domain_test.go +++ b/domain/domain_test.go @@ -14,6 +14,7 @@ package domain import ( + "context" "testing" "time" @@ -80,10 +81,21 @@ func (*testSuite) TestT(c *C) { // for setting lease lease := 100 * time.Millisecond - // for GetSnapshotInfoSchema - snapIs, err := dom.GetSnapshotInfoSchema(snapTS) + // for updating the self schema version + goCtx, cancel := context.WithTimeout(context.Background(), 10*time.Millisecond) + err = dd.SchemaSyncer().OwnerCheckAllVersions(goCtx, is.SchemaMetaVersion()) + cancel() c.Assert(err, IsNil) + snapIs, err := dom.GetSnapshotInfoSchema(snapTS) c.Assert(snapIs, NotNil) + c.Assert(err, IsNil) + // Make sure that the self schema version doesn't be changed. + goCtx, cancel = context.WithTimeout(context.Background(), 10*time.Millisecond) + err = dd.SchemaSyncer().OwnerCheckAllVersions(goCtx, is.SchemaMetaVersion()) + cancel() + c.Assert(err, IsNil) + + // for GetSnapshotInfoSchema snapTS = oracle.EncodeTSO(oracle.GetPhysical(time.Now())) snapIs, err = dom.GetSnapshotInfoSchema(snapTS) c.Assert(err, IsNil) From 461512b8273a7b64580cd96e34c8e2d7983b5e62 Mon Sep 17 00:00:00 2001 From: qupeng Date: Mon, 6 May 2019 11:30:24 +0800 Subject: [PATCH 44/49] tikvclient: print error logs only if necessary (#10349) --- store/tikv/client.go | 38 +++++++++++++++++++++++++++++++++----- 1 file changed, 33 insertions(+), 5 deletions(-) diff --git a/store/tikv/client.go b/store/tikv/client.go index a6dc09fd99898..c550742f50220 100644 --- a/store/tikv/client.go +++ b/store/tikv/client.go @@ -71,6 +71,9 @@ type Client interface { } type connArray struct { + // The target host. + target string + index uint32 v []*grpc.ClientConn // streamTimeout binds with a background goroutine to process coprocessor streaming timeout. @@ -83,6 +86,9 @@ type connArray struct { } type batchCommandsClient struct { + // The target host. + target string + conn *grpc.ClientConn client tikvpb.Tikv_BatchCommandsClient batched sync.Map @@ -126,13 +132,18 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) { // When `conn.Close()` is called, `client.Recv()` will return an error. resp, err := c.client.Recv() if err != nil { - logutil.Logger(context.Background()).Error("batchRecvLoop error when receive", zap.Error(err)) now := time.Now() for { // try to re-create the streaming in the loop. if c.isStopped() { return } + logutil.Logger(context.Background()).Error( + "batchRecvLoop error when receive", + zap.String("target", c.target), + zap.Error(err), + ) + // Hold the lock to forbid batchSendLoop using the old client. c.clientLock.Lock() c.failPendingRequests(err) // fail all pending requests. @@ -143,11 +154,18 @@ func (c *batchCommandsClient) batchRecvLoop(cfg config.TiKVClient) { c.clientLock.Unlock() if err == nil { - logutil.Logger(context.Background()).Info("batchRecvLoop re-create streaming success") + logutil.Logger(context.Background()).Info( + "batchRecvLoop re-create streaming success", + zap.String("target", c.target), + ) c.client = streamClient break } - logutil.Logger(context.Background()).Error("batchRecvLoop re-create streaming fail", zap.Error(err)) + logutil.Logger(context.Background()).Error( + "batchRecvLoop re-create streaming fail", + zap.String("target", c.target), + zap.Error(err), + ) // TODO: Use a more smart backoff strategy. time.Sleep(time.Second) } @@ -198,6 +216,8 @@ func newConnArray(maxSize uint, addr string, security config.Security) (*connArr } func (a *connArray) Init(addr string, security config.Security) error { + a.target = addr + opt := grpc.WithInsecure() if len(security.ClusterSSLCA) != 0 { tlsConfig, err := security.ToTLSConfig() @@ -256,6 +276,7 @@ func (a *connArray) Init(addr string, security config.Security) error { return errors.Trace(err) } batchClient := &batchCommandsClient{ + target: a.target, conn: conn, client: streamClient, batched: sync.Map{}, @@ -423,7 +444,10 @@ func (a *connArray) batchSendLoop(cfg config.TiKVClient) { } } length := len(requests) - if uint(length) < bestBatchWaitSize && bestBatchWaitSize > 1 { + if uint(length) == 0 { + // The batch command channel is closed. + return + } else if uint(length) < bestBatchWaitSize && bestBatchWaitSize > 1 { // Waits too long to collect requests, reduce the target batch size. bestBatchWaitSize -= 1 } else if uint(length) > bestBatchWaitSize+4 && bestBatchWaitSize < cfg.MaxBatchSize { @@ -450,7 +474,11 @@ func (a *connArray) batchSendLoop(cfg config.TiKVClient) { err := batchCommandsClient.client.Send(request) batchCommandsClient.clientLock.Unlock() if err != nil { - logutil.Logger(context.Background()).Error("batch commands send error", zap.Error(err)) + logutil.Logger(context.Background()).Error( + "batch commands send error", + zap.String("target", a.target), + zap.Error(err), + ) batchCommandsClient.failPendingRequests(err) } } From 190990b352b4939c75a60722573ab3f29baa9351 Mon Sep 17 00:00:00 2001 From: lysu Date: Mon, 6 May 2019 13:19:37 +0800 Subject: [PATCH 45/49] bindinfo: reduce duplicate DigestHash call (#10352) --- bindinfo/bind_test.go | 16 ++++++++++------ bindinfo/handle.go | 7 +++---- executor/compiler.go | 12 +++++++----- go.mod | 2 +- go.sum | 4 ++-- session/session.go | 9 +++++---- 6 files changed, 28 insertions(+), 22 deletions(-) diff --git a/bindinfo/bind_test.go b/bindinfo/bind_test.go index 3c716fc39354d..d6e1a12a9a5f2 100644 --- a/bindinfo/bind_test.go +++ b/bindinfo/bind_test.go @@ -122,7 +122,8 @@ func (s *testSuite) TestBindParse(c *C) { c.Check(err, IsNil) c.Check(bindHandle.Size(), Equals, 1) - bindData := bindHandle.GetBindRecord("select * from t", "test") + sql, hash := parser.NormalizeDigest("select * from t") + bindData := bindHandle.GetBindRecord(hash, sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t") c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t)") @@ -151,7 +152,9 @@ func (s *testSuite) TestGlobalBinding(c *C) { _, err = tk.Exec("create global binding for select * from t where i>99 using select * from t use index(index_t) where i>99") c.Assert(err, IsNil) - bindData := s.domain.BindHandle().GetBindRecord("select * from t where i > ?", "test") + sql, hash := parser.NormalizeDigest("select * from t where i > ?") + + bindData := s.domain.BindHandle().GetBindRecord(hash, sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>99") @@ -183,7 +186,7 @@ func (s *testSuite) TestGlobalBinding(c *C) { c.Check(err, IsNil) c.Check(bindHandle.Size(), Equals, 1) - bindData = bindHandle.GetBindRecord("select * from t where i > ?", "test") + bindData = bindHandle.GetBindRecord(hash, sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>99") @@ -196,7 +199,7 @@ func (s *testSuite) TestGlobalBinding(c *C) { _, err = tk.Exec("DROP global binding for select * from t where i>100") c.Check(err, IsNil) - bindData = s.domain.BindHandle().GetBindRecord("select * from t where i > ?", "test") + bindData = s.domain.BindHandle().GetBindRecord(hash, sql, "test") c.Check(bindData, IsNil) bindHandle = bindinfo.NewBindHandle(tk.Se) @@ -204,7 +207,7 @@ func (s *testSuite) TestGlobalBinding(c *C) { c.Check(err, IsNil) c.Check(bindHandle.Size(), Equals, 0) - bindData = bindHandle.GetBindRecord("select * from t where i > ?", "test") + bindData = bindHandle.GetBindRecord(hash, sql, "test") c.Check(bindData, IsNil) rs, err = tk.Exec("show global bindings") @@ -400,7 +403,8 @@ func (s *testSuite) TestErrorBind(c *C) { _, err := tk.Exec("create global binding for select * from t where i>100 using select * from t use index(index_t) where i>100") c.Assert(err, IsNil, Commentf("err %v", err)) - bindData := s.domain.BindHandle().GetBindRecord("select * from t where i > ?", "test") + sql, hash := parser.NormalizeDigest("select * from t where i > ?") + bindData := s.domain.BindHandle().GetBindRecord(hash, sql, "test") c.Check(bindData, NotNil) c.Check(bindData.OriginalSQL, Equals, "select * from t where i > ?") c.Check(bindData.BindSQL, Equals, "select * from t use index(index_t) where i>100") diff --git a/bindinfo/handle.go b/bindinfo/handle.go index 140b6f8cd8884..2d0712bb5fdb8 100644 --- a/bindinfo/handle.go +++ b/bindinfo/handle.go @@ -284,8 +284,8 @@ func (h *BindHandle) Size() int { } // GetBindRecord return the bindMeta of the (normdOrigSQL,db) if bindMeta exist. -func (h *BindHandle) GetBindRecord(normdOrigSQL, db string) *BindMeta { - return h.bindInfo.Load().(cache).getBindRecord(normdOrigSQL, db) +func (h *BindHandle) GetBindRecord(hash, normdOrigSQL, db string) *BindMeta { + return h.bindInfo.Load().(cache).getBindRecord(hash, normdOrigSQL, db) } // GetAllBindRecord return all bind record in cache. @@ -387,8 +387,7 @@ func copyInvalidBindRecordMap(oldMap map[string]*invalidBindRecordMap) map[strin return newMap } -func (c cache) getBindRecord(normdOrigSQL, db string) *BindMeta { - hash := parser.DigestHash(normdOrigSQL) +func (c cache) getBindRecord(hash, normdOrigSQL, db string) *BindMeta { bindRecords := c[hash] if bindRecords != nil { for _, bindRecord := range bindRecords { diff --git a/executor/compiler.go b/executor/compiler.go index f233bdfe833cd..7f1c85c691011 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -396,17 +396,19 @@ func addHint(ctx sessionctx.Context, stmtNode ast.StmtNode) ast.StmtNode { normalizeExplainSQL := parser.Normalize(x.Text()) idx := strings.Index(normalizeExplainSQL, "select") normalizeSQL := normalizeExplainSQL[idx:] - x.Stmt = addHintForSelect(normalizeSQL, ctx, x.Stmt) + hash := parser.DigestHash(normalizeSQL) + x.Stmt = addHintForSelect(hash, normalizeSQL, ctx, x.Stmt) } return x case *ast.SelectStmt: - return addHintForSelect(parser.Normalize(x.Text()), ctx, x) + normalizeSQL, hash := parser.NormalizeDigest(x.Text()) + return addHintForSelect(hash, normalizeSQL, ctx, x) default: return stmtNode } } -func addHintForSelect(normdOrigSQL string, ctx sessionctx.Context, stmt ast.StmtNode) ast.StmtNode { +func addHintForSelect(hash, normdOrigSQL string, ctx sessionctx.Context, stmt ast.StmtNode) ast.StmtNode { sessionHandle := ctx.Value(bindinfo.SessionBindInfoKeyType).(*bindinfo.SessionHandle) bindRecord := sessionHandle.GetBindRecord(normdOrigSQL, ctx.GetSessionVars().CurrentDB) if bindRecord != nil { @@ -418,9 +420,9 @@ func addHintForSelect(normdOrigSQL string, ctx sessionctx.Context, stmt ast.Stmt } } globalHandle := domain.GetDomain(ctx).BindHandle() - bindRecord = globalHandle.GetBindRecord(normdOrigSQL, ctx.GetSessionVars().CurrentDB) + bindRecord = globalHandle.GetBindRecord(hash, normdOrigSQL, ctx.GetSessionVars().CurrentDB) if bindRecord == nil { - bindRecord = globalHandle.GetBindRecord(normdOrigSQL, "") + bindRecord = globalHandle.GetBindRecord(hash, normdOrigSQL, "") } if bindRecord != nil { return bindinfo.BindHint(stmt, bindRecord.Ast) diff --git a/go.mod b/go.mod index 56b76b68b2666..68f788985f536 100644 --- a/go.mod +++ b/go.mod @@ -39,7 +39,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190429120706-c378059f7f42 + github.com/pingcap/parser v0.0.0-20190505092803-4542e963c7f1 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 diff --git a/go.sum b/go.sum index f2543ff30249b..827c6392873c9 100644 --- a/go.sum +++ b/go.sum @@ -160,8 +160,8 @@ github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190429120706-c378059f7f42 h1:iuZ/y1DLC/4gLTxw/xtEbo9R2SMRq7CKzLXhffff/kc= -github.com/pingcap/parser v0.0.0-20190429120706-c378059f7f42/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190505092803-4542e963c7f1 h1:YvxFABfyD5Pnp80FUVV4w3zdlmkcwRhQbn7xpTjBwwU= +github.com/pingcap/parser v0.0.0-20190505092803-4542e963c7f1/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= diff --git a/session/session.go b/session/session.go index e25198f8b57e5..5ea7497547262 100644 --- a/session/session.go +++ b/session/session.go @@ -1046,7 +1046,7 @@ func (s *session) execute(ctx context.Context, sql string) (recordSets []sqlexec } func (s *session) handleInvalidBindRecord(ctx context.Context, stmtNode ast.StmtNode) { - var normdOrigSQL string + var normdOrigSQL, hash string switch x := stmtNode.(type) { case *ast.ExplainStmt: switch x.Stmt.(type) { @@ -1054,11 +1054,12 @@ func (s *session) handleInvalidBindRecord(ctx context.Context, stmtNode ast.Stmt normalizeExplainSQL := parser.Normalize(x.Text()) idx := strings.Index(normalizeExplainSQL, "select") normdOrigSQL = normalizeExplainSQL[idx:] + hash = parser.DigestHash(normdOrigSQL) default: return } case *ast.SelectStmt: - normdOrigSQL = parser.Normalize(x.Text()) + normdOrigSQL, hash = parser.NormalizeDigest(x.Text()) default: return } @@ -1070,9 +1071,9 @@ func (s *session) handleInvalidBindRecord(ctx context.Context, stmtNode ast.Stmt } globalHandle := domain.GetDomain(s).BindHandle() - bindMeta = globalHandle.GetBindRecord(normdOrigSQL, s.GetSessionVars().CurrentDB) + bindMeta = globalHandle.GetBindRecord(hash, normdOrigSQL, s.GetSessionVars().CurrentDB) if bindMeta == nil { - bindMeta = globalHandle.GetBindRecord(normdOrigSQL, "") + bindMeta = globalHandle.GetBindRecord(hash, normdOrigSQL, "") } if bindMeta != nil { record := &bindinfo.BindRecord{ From 7e8005324cd9835ae3e989f20e9384ddf64a9abb Mon Sep 17 00:00:00 2001 From: Song Guo Date: Mon, 6 May 2019 15:40:20 +0800 Subject: [PATCH 46/49] planner: fix wrong `DATE/DATETIME` comparison in `BETWEEN` function (#10313) --- expression/builtin_compare.go | 8 ++++---- planner/core/expression_rewriter.go | 13 +++++++++++-- planner/core/expression_test.go | 2 ++ 3 files changed, 17 insertions(+), 6 deletions(-) diff --git a/expression/builtin_compare.go b/expression/builtin_compare.go index 895cf133bc291..a6e12a6b2474c 100644 --- a/expression/builtin_compare.go +++ b/expression/builtin_compare.go @@ -379,8 +379,8 @@ func temporalWithDateAsNumEvalType(argTp *types.FieldType) (argEvalType types.Ev return } -// getCmpTp4MinMax gets compare type for GREATEST and LEAST. -func getCmpTp4MinMax(args []Expression) (argTp types.EvalType) { +// GetCmpTp4MinMax gets compare type for GREATEST and LEAST and BETWEEN (mainly for datetime). +func GetCmpTp4MinMax(args []Expression) (argTp types.EvalType) { datetimeFound, isAllStr := false, true cmpEvalType, isStr, isTemporalWithDate := temporalWithDateAsNumEvalType(args[0].GetType()) if !isStr { @@ -421,7 +421,7 @@ func (c *greatestFunctionClass) getFunction(ctx sessionctx.Context, args []Expre if err = c.verifyArgs(args); err != nil { return nil, err } - tp, cmpAsDatetime := getCmpTp4MinMax(args), false + tp, cmpAsDatetime := GetCmpTp4MinMax(args), false if tp == types.ETDatetime { cmpAsDatetime = true tp = types.ETString @@ -615,7 +615,7 @@ func (c *leastFunctionClass) getFunction(ctx sessionctx.Context, args []Expressi if err = c.verifyArgs(args); err != nil { return nil, err } - tp, cmpAsDatetime := getCmpTp4MinMax(args), false + tp, cmpAsDatetime := GetCmpTp4MinMax(args), false if tp == types.ETDatetime { cmpAsDatetime = true tp = types.ETString diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index 0f36b2bc32eb2..8383b18824b0c 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -1259,11 +1259,20 @@ func (er *expressionRewriter) betweenToExpression(v *ast.BetweenExpr) { if er.err != nil { return } + + expr, lexp, rexp := er.ctxStack[stkLen-3], er.ctxStack[stkLen-2], er.ctxStack[stkLen-1] + + if expression.GetCmpTp4MinMax([]expression.Expression{expr, lexp, rexp}) == types.ETDatetime { + expr = expression.WrapWithCastAsTime(er.ctx, expr, types.NewFieldType(mysql.TypeDatetime)) + lexp = expression.WrapWithCastAsTime(er.ctx, lexp, types.NewFieldType(mysql.TypeDatetime)) + rexp = expression.WrapWithCastAsTime(er.ctx, rexp, types.NewFieldType(mysql.TypeDatetime)) + } + var op string var l, r expression.Expression - l, er.err = er.newFunction(ast.GE, &v.Type, er.ctxStack[stkLen-3], er.ctxStack[stkLen-2]) + l, er.err = er.newFunction(ast.GE, &v.Type, expr, lexp) if er.err == nil { - r, er.err = er.newFunction(ast.LE, &v.Type, er.ctxStack[stkLen-3], er.ctxStack[stkLen-1]) + r, er.err = er.newFunction(ast.LE, &v.Type, expr, rexp) } op = ast.LogicAnd if er.err != nil { diff --git a/planner/core/expression_test.go b/planner/core/expression_test.go index 232afd3a62d70..c44633b0a752f 100644 --- a/planner/core/expression_test.go +++ b/planner/core/expression_test.go @@ -70,6 +70,8 @@ func (s *testExpressionSuite) TestBetween(c *C) { tests := []testCase{ {exprStr: "1 between 2 and 3", resultStr: "0"}, {exprStr: "1 not between 2 and 3", resultStr: "1"}, + {exprStr: "'2001-04-10 12:34:56' between cast('2001-01-01 01:01:01' as datetime) and '01-05-01'", resultStr: "1"}, + {exprStr: "20010410123456 between cast('2001-01-01 01:01:01' as datetime) and 010501", resultStr: "0"}, } s.runTests(c, tests) } From 0106361cd92da0a5f79557d33613841bab9271b3 Mon Sep 17 00:00:00 2001 From: Haibin Xie Date: Mon, 6 May 2019 16:22:19 +0800 Subject: [PATCH 47/49] expression: let `PushDownNot` does not change the argument (#10363) --- cmd/explaintest/r/explain_easy.result | 17 ++++++++++++ cmd/explaintest/t/explain_easy.test | 8 ++++++ expression/util.go | 40 ++++++++++++--------------- expression/util_test.go | 2 ++ planner/core/logical_plan_test.go | 2 +- 5 files changed, 46 insertions(+), 23 deletions(-) diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index 5255df34cea89..4806f6be3d3b7 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -643,3 +643,20 @@ id count task operator info Point_Get_1 1.00 root table:t, index:i j rollback; drop table if exists t; +create table t(a int); +begin; +insert into t values (1); +explain select * from t left outer join t t1 on t.a = t1.a where t.a not between 1 and 2; +id count task operator info +Projection_8 8320.83 root test.t.a, test.t1.a +└─HashLeftJoin_9 8320.83 root left outer join, inner:UnionScan_14, equal:[eq(test.t.a, test.t1.a)] + ├─UnionScan_10 6656.67 root not(and(ge(test.t.a, 1), le(test.t.a, 2))) + │ └─TableReader_13 6656.67 root data:Selection_12 + │ └─Selection_12 6656.67 cop or(lt(test.t.a, 1), gt(test.t.a, 2)) + │ └─TableScan_11 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo + └─UnionScan_14 6656.67 root not(and(ge(test.t1.a, 1), le(test.t1.a, 2))), not(isnull(test.t1.a)) + └─TableReader_17 6656.67 root data:Selection_16 + └─Selection_16 6656.67 cop not(isnull(test.t1.a)), or(lt(test.t1.a, 1), gt(test.t1.a, 2)) + └─TableScan_15 10000.00 cop table:t, range:[-inf,+inf], keep order:false, stats:pseudo +rollback; +drop table if exists t; diff --git a/cmd/explaintest/t/explain_easy.test b/cmd/explaintest/t/explain_easy.test index 8c2b399be3115..a8cd985164413 100644 --- a/cmd/explaintest/t/explain_easy.test +++ b/cmd/explaintest/t/explain_easy.test @@ -137,3 +137,11 @@ insert into t values (1, 1); explain update t set j = -j where i = 1 and j = 1; rollback; drop table if exists t; + +# https://github.com/pingcap/tidb/issues/10344 +create table t(a int); +begin; +insert into t values (1); +explain select * from t left outer join t t1 on t.a = t1.a where t.a not between 1 and 2; +rollback; +drop table if exists t; diff --git a/expression/util.go b/expression/util.go index 881a8baa19a9b..368a211d2c472 100644 --- a/expression/util.go +++ b/expression/util.go @@ -289,6 +289,14 @@ var symmetricOp = map[opcode.Op]opcode.Op{ opcode.NullEQ: opcode.NullEQ, } +func doPushDownNot(ctx sessionctx.Context, exprs []Expression, not bool) []Expression { + newExprs := make([]Expression, 0, len(exprs)) + for _, expr := range exprs { + newExprs = append(newExprs, PushDownNot(ctx, expr, not)) + } + return newExprs +} + // PushDownNot pushes the `not` function down to the expression's arguments. func PushDownNot(ctx sessionctx.Context, expr Expression, not bool) Expression { if f, ok := expr.(*ScalarFunction); ok { @@ -299,34 +307,22 @@ func PushDownNot(ctx sessionctx.Context, expr Expression, not bool) Expression { if not { return NewFunctionInternal(f.GetCtx(), oppositeOp[f.FuncName.L], f.GetType(), f.GetArgs()...) } - for i, arg := range f.GetArgs() { - f.GetArgs()[i] = PushDownNot(f.GetCtx(), arg, false) - } - return f + newArgs := doPushDownNot(f.GetCtx(), f.GetArgs(), false) + return NewFunctionInternal(f.GetCtx(), f.FuncName.L, f.GetType(), newArgs...) case ast.LogicAnd: if not { - args := f.GetArgs() - for i, a := range args { - args[i] = PushDownNot(f.GetCtx(), a, true) - } - return NewFunctionInternal(f.GetCtx(), ast.LogicOr, f.GetType(), args...) - } - for i, arg := range f.GetArgs() { - f.GetArgs()[i] = PushDownNot(f.GetCtx(), arg, false) + newArgs := doPushDownNot(f.GetCtx(), f.GetArgs(), true) + return NewFunctionInternal(f.GetCtx(), ast.LogicOr, f.GetType(), newArgs...) } - return f + newArgs := doPushDownNot(f.GetCtx(), f.GetArgs(), false) + return NewFunctionInternal(f.GetCtx(), f.FuncName.L, f.GetType(), newArgs...) case ast.LogicOr: if not { - args := f.GetArgs() - for i, a := range args { - args[i] = PushDownNot(f.GetCtx(), a, true) - } - return NewFunctionInternal(f.GetCtx(), ast.LogicAnd, f.GetType(), args...) - } - for i, arg := range f.GetArgs() { - f.GetArgs()[i] = PushDownNot(f.GetCtx(), arg, false) + newArgs := doPushDownNot(f.GetCtx(), f.GetArgs(), true) + return NewFunctionInternal(f.GetCtx(), ast.LogicAnd, f.GetType(), newArgs...) } - return f + newArgs := doPushDownNot(f.GetCtx(), f.GetArgs(), false) + return NewFunctionInternal(f.GetCtx(), f.FuncName.L, f.GetType(), newArgs...) } } if not { diff --git a/expression/util_test.go b/expression/util_test.go index 1d9f9b936860f..bf6ff7f459cb0 100644 --- a/expression/util_test.go +++ b/expression/util_test.go @@ -68,8 +68,10 @@ func (s *testUtilSuite) TestPushDownNot(c *check.C) { neFunc := newFunction(ast.NE, col, One) andFunc2 := newFunction(ast.LogicAnd, neFunc, neFunc) orFunc2 := newFunction(ast.LogicOr, andFunc2, neFunc) + notFuncCopy := notFunc.Clone() ret := PushDownNot(ctx, notFunc, false) c.Assert(ret.Equal(ctx, orFunc2), check.IsTrue) + c.Assert(notFunc.Equal(ctx, notFuncCopy), check.IsTrue) } func (s *testUtilSuite) TestFilter(c *check.C) { diff --git a/planner/core/logical_plan_test.go b/planner/core/logical_plan_test.go index 2af974ba048c8..ee92ee927a0f8 100644 --- a/planner/core/logical_plan_test.go +++ b/planner/core/logical_plan_test.go @@ -395,7 +395,7 @@ func (s *testPlanSuite) TestSimplifyOuterJoin(c *C) { }, { sql: "select * from t t1 left join t t2 on t1.b = t2.b where not (t1.c > 1 and t2.c > 1);", - best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.b,test.t2.b)->Sel([not(and(le(test.t1.c, 1), le(test.t2.c, 1)))])->Projection", + best: "Join{DataScan(t1)->DataScan(t2)}(test.t1.b,test.t2.b)->Sel([not(and(gt(test.t1.c, 1), gt(test.t2.c, 1)))])->Projection", joinType: "left outer join", }, { From 7ecb3159a6370d2c74f32887dc404023ee551812 Mon Sep 17 00:00:00 2001 From: crazycs Date: Mon, 6 May 2019 17:41:50 +0800 Subject: [PATCH 48/49] *: add split index region syntax (#10203) --- executor/builder.go | 13 ++++++ executor/executor.go | 4 ++ executor/executor_test.go | 12 ++++++ executor/split.go | 84 ++++++++++++++++++++++++++++++++++++ go.mod | 2 +- go.sum | 4 +- planner/core/common_plans.go | 10 +++++ planner/core/planbuilder.go | 42 ++++++++++++++++++ 8 files changed, 168 insertions(+), 3 deletions(-) create mode 100644 executor/split.go diff --git a/executor/builder.go b/executor/builder.go index 596b64c72b48c..2ccdc3584b865 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -193,6 +193,8 @@ func (b *executorBuilder) build(p plannercore.Plan) Executor { return b.buildWindow(v) case *plannercore.SQLBindPlan: return b.buildSQLBindExec(v) + case *plannercore.SplitIndexRegion: + return b.buildSplitIndexRegion(v) default: if mp, ok := p.(MockPhysicalPlan); ok { return mp.GetExecutor() @@ -1248,6 +1250,17 @@ func (b *executorBuilder) buildUnionAll(v *plannercore.PhysicalUnionAll) Executo return e } +func (b *executorBuilder) buildSplitIndexRegion(v *plannercore.SplitIndexRegion) Executor { + base := newBaseExecutor(b.ctx, nil, v.ExplainID()) + base.initCap = chunk.ZeroCapacity + return &SplitIndexRegionExec{ + baseExecutor: base, + table: v.Table, + indexInfo: v.IndexInfo, + valueLists: v.ValueLists, + } +} + func (b *executorBuilder) buildUpdate(v *plannercore.Update) Executor { tblID2table := make(map[int64]table.Table) for id := range v.SelectPlan.Schema().TblID2Handle { diff --git a/executor/executor.go b/executor/executor.go index eb32d3cc17f2b..afbc20045ba23 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1379,6 +1379,10 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.InShowWarning = true sc.SetWarnings(vars.StmtCtx.GetWarnings()) } + case *ast.SplitIndexRegionStmt: + sc.IgnoreTruncate = false + sc.IgnoreZeroInDate = true + sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() default: sc.IgnoreTruncate = true sc.IgnoreZeroInDate = true diff --git a/executor/executor_test.go b/executor/executor_test.go index cce1ea9b35432..fe2840ec7d852 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3693,6 +3693,18 @@ func (s *testSuite) TestReadPartitionedTable(c *C) { tk.MustQuery("select a from pt where b = 3").Check(testkit.Rows("3")) } +func (s *testSuite) TestSplitIndexRegion(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a varchar(100),b int, index idx1(b,a))") + tk.MustExec(`split table t index idx1 by (10000,"abcd"),(10000000);`) + _, err := tk.Exec(`split table t index idx1 by ("abcd");`) + c.Assert(err, NotNil) + terr := errors.Cause(err).(*terror.Error) + c.Assert(terr.Code(), Equals, terror.ErrCode(mysql.WarnDataTruncated)) +} + type testOOMSuite struct { store kv.Storage do *domain.Domain diff --git a/executor/split.go b/executor/split.go new file mode 100644 index 0000000000000..dc1b75e8018f3 --- /dev/null +++ b/executor/split.go @@ -0,0 +1,84 @@ +// Copyright 2019 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package executor + +import ( + "context" + "math" + + "github.com/pingcap/parser/model" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/table/tables" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/logutil" + "go.uber.org/zap" +) + +// SplitIndexRegionExec represents a split index regions executor. +type SplitIndexRegionExec struct { + baseExecutor + + table table.Table + indexInfo *model.IndexInfo + valueLists [][]types.Datum +} + +type splitableStore interface { + SplitRegionAndScatter(splitKey kv.Key) (uint64, error) + WaitScatterRegionFinish(regionID uint64) error +} + +// Next implements the Executor Next interface. +func (e *SplitIndexRegionExec) Next(ctx context.Context, _ *chunk.RecordBatch) error { + store := e.ctx.GetStore() + s, ok := store.(splitableStore) + if !ok { + return nil + } + regionIDs := make([]uint64, 0, len(e.valueLists)) + index := tables.NewIndex(e.table.Meta().ID, e.table.Meta(), e.indexInfo) + for _, values := range e.valueLists { + idxKey, _, err := index.GenIndexKey(e.ctx.GetSessionVars().StmtCtx, values, math.MinInt64, nil) + if err != nil { + return err + } + + regionID, err := s.SplitRegionAndScatter(idxKey) + if err != nil { + logutil.Logger(context.Background()).Warn("split table index region failed", + zap.String("table", e.table.Meta().Name.L), + zap.String("index", e.indexInfo.Name.L), + zap.Error(err)) + continue + } + regionIDs = append(regionIDs, regionID) + + } + if !e.ctx.GetSessionVars().WaitTableSplitFinish { + return nil + } + for _, regionID := range regionIDs { + err := s.WaitScatterRegionFinish(regionID) + if err != nil { + logutil.Logger(context.Background()).Warn("wait scatter region failed", + zap.Uint64("regionID", regionID), + zap.String("table", e.table.Meta().Name.L), + zap.String("index", e.indexInfo.Name.L), + zap.Error(err)) + } + } + return nil +} diff --git a/go.mod b/go.mod index 68f788985f536..70a6f40911b88 100644 --- a/go.mod +++ b/go.mod @@ -39,7 +39,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020122428-b9ff6c35079e github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190505092803-4542e963c7f1 + github.com/pingcap/parser v0.0.0-20190506092653-e336082eb825 github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330 diff --git a/go.sum b/go.sum index 827c6392873c9..288f6e6d080d2 100644 --- a/go.sum +++ b/go.sum @@ -160,8 +160,8 @@ github.com/pingcap/kvproto v0.0.0-20190327032727-3d8cb3a30d5d/go.mod h1:QMdbTAXC github.com/pingcap/log v0.0.0-20190214045112-b37da76f67a7/go.mod h1:xsfkWVaFVV5B8e1K9seWfyJWFrIhbtUTAD8NV1Pq3+w= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190505092803-4542e963c7f1 h1:YvxFABfyD5Pnp80FUVV4w3zdlmkcwRhQbn7xpTjBwwU= -github.com/pingcap/parser v0.0.0-20190505092803-4542e963c7f1/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190506092653-e336082eb825 h1:U9Kdnknj4n2v76Mg7wazevZ5N9U1OIaMwSNRVLEcLX0= +github.com/pingcap/parser v0.0.0-20190506092653-e336082eb825/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669 h1:ZoKjndm/Ig7Ru/wojrQkc/YLUttUdQXoH77gtuWCvL4= github.com/pingcap/pd v0.0.0-20190424024702-bd1e2496a669/go.mod h1:MUCxRzOkYiWZtlyi4MhxjCIj9PgQQ/j+BLNGm7aUsnM= github.com/pingcap/tidb-tools v2.1.3-0.20190321065848-1e8b48f5c168+incompatible h1:MkWCxgZpJBgY2f4HtwWMMFzSBb3+JPzeJgF3VrXE/bU= diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index ea467b3ef637d..a918ed17ff218 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/types" driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/kvcache" @@ -476,6 +477,15 @@ type LoadStats struct { Path string } +// SplitIndexRegion represents a split index regions plan. +type SplitIndexRegion struct { + baseSchemaProducer + + Table table.Table + IndexInfo *model.IndexInfo + ValueLists [][]types.Datum +} + // DDL represents a DDL statement plan. type DDL struct { baseSchemaProducer diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 1bde08d6f95b5..118b41c8bd9d3 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -279,6 +279,8 @@ func (b *PlanBuilder) Build(node ast.Node) (Plan, error) { return b.buildDropBindPlan(x) case *ast.ChangeStmt: return b.buildChange(x) + case *ast.SplitIndexRegionStmt: + return b.buildSplitIndexRegion(x) } return nil, ErrUnsupportedType.GenWithStack("Unsupported type %T", node) } @@ -1605,6 +1607,46 @@ func (b *PlanBuilder) buildLoadStats(ld *ast.LoadStatsStmt) Plan { return p } +func (b *PlanBuilder) buildSplitIndexRegion(node *ast.SplitIndexRegionStmt) (Plan, error) { + tblInfo := node.Table.TableInfo + indexInfo := tblInfo.FindIndexByName(strings.ToLower(node.IndexName)) + if indexInfo == nil { + return nil, ErrKeyDoesNotExist.GenWithStackByArgs(node.IndexName, tblInfo.Name) + } + + indexValues := make([][]types.Datum, 0, len(node.ValueLists)) + for i, valuesItem := range node.ValueLists { + if len(valuesItem) > len(indexInfo.Columns) { + return nil, ErrWrongValueCountOnRow.GenWithStackByArgs(i + 1) + } + valueList := make([]types.Datum, 0, len(valuesItem)) + for j, valueItem := range valuesItem { + x, ok := valueItem.(*driver.ValueExpr) + if !ok { + return nil, errors.New("expect constant values") + } + colOffset := indexInfo.Columns[j].Offset + value, err := x.Datum.ConvertTo(b.ctx.GetSessionVars().StmtCtx, &tblInfo.Columns[colOffset].FieldType) + if err != nil { + return nil, err + } + + valueList = append(valueList, value) + } + indexValues = append(indexValues, valueList) + } + tableInPlan, ok := b.is.TableByID(tblInfo.ID) + if !ok { + return nil, errors.Errorf("Can't get table %s.", tblInfo.Name.O) + } + return &SplitIndexRegion{ + Table: tableInPlan, + IndexInfo: indexInfo, + ValueLists: indexValues, + }, nil + +} + func (b *PlanBuilder) buildDDL(node ast.DDLNode) (Plan, error) { var authErr error switch v := node.(type) { From f17a1155e73011260d41789c69a7fb55f7971076 Mon Sep 17 00:00:00 2001 From: Kenan Yao Date: Mon, 6 May 2019 21:05:48 +0800 Subject: [PATCH 49/49] executor: split unit tests to speedup execution time (#10364) --- executor/executor_test.go | 51 +++++++++++++++++++++++++++ executor/metrics_test.go | 2 +- executor/sort_test.go | 2 +- executor/union_scan_test.go | 4 +-- executor/window_test.go | 2 +- executor/write_test.go | 70 ++++++++++++++++++------------------- 6 files changed, 91 insertions(+), 40 deletions(-) diff --git a/executor/executor_test.go b/executor/executor_test.go index fe2840ec7d852..ae1bd548eed47 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -3636,6 +3636,57 @@ func (s *testSuite3) TearDownTest(c *C) { } } +type testSuite4 struct { + cluster *mocktikv.Cluster + mvccStore mocktikv.MVCCStore + store kv.Storage + domain *domain.Domain + *parser.Parser + ctx *mock.Context +} + +func (s *testSuite4) SetUpSuite(c *C) { + s.Parser = parser.New() + flag.Lookup("mockTikv") + useMockTikv := *mockTikv + if useMockTikv { + s.cluster = mocktikv.NewCluster() + mocktikv.BootstrapWithSingleStore(s.cluster) + s.mvccStore = mocktikv.MustNewMVCCStore() + store, err := mockstore.NewMockTikvStore( + mockstore.WithCluster(s.cluster), + mockstore.WithMVCCStore(s.mvccStore), + ) + c.Assert(err, IsNil) + s.store = store + session.SetSchemaLease(0) + session.SetStatsLease(0) + } + d, err := session.BootstrapSession(s.store) + c.Assert(err, IsNil) + d.SetStatsUpdating(true) + s.domain = d +} + +func (s *testSuite4) TearDownSuite(c *C) { + s.domain.Close() + s.store.Close() +} + +func (s *testSuite4) TearDownTest(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + r := tk.MustQuery("show full tables") + for _, tb := range r.Rows() { + tableName := tb[0] + if tb[1] == "VIEW" { + tk.MustExec(fmt.Sprintf("drop view %v", tableName)) + } else { + tk.MustExec(fmt.Sprintf("drop table %v", tableName)) + } + } +} + func (s *testSuite) TestStrToDateBuiltin(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustQuery(`select str_to_date('18/10/22','%y/%m/%d') from dual`).Check(testkit.Rows("2018-10-22")) diff --git a/executor/metrics_test.go b/executor/metrics_test.go index 249b9989e0303..afb165038d8b6 100644 --- a/executor/metrics_test.go +++ b/executor/metrics_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite2) TestStmtLabel(c *C) { +func (s *testSuite4) TestStmtLabel(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table label (c1 int primary key, c2 int, c3 int, index (c2))") diff --git a/executor/sort_test.go b/executor/sort_test.go index 83da697db84ff..3cc18f5953c8f 100644 --- a/executor/sort_test.go +++ b/executor/sort_test.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite2) TestSortRand(c *C) { +func (s *testSuite4) TestSortRand(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/executor/union_scan_test.go b/executor/union_scan_test.go index 2590fd8b7d0c9..7fc90083cf144 100644 --- a/executor/union_scan_test.go +++ b/executor/union_scan_test.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite2) TestDirtyTransaction(c *C) { +func (s *testSuite4) TestDirtyTransaction(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -94,7 +94,7 @@ func (s *testSuite2) TestDirtyTransaction(c *C) { tk.MustExec("commit") } -func (s *testSuite2) TestUnionScanWithCastCondition(c *C) { +func (s *testSuite4) TestUnionScanWithCastCondition(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table ta (a varchar(20))") diff --git a/executor/window_test.go b/executor/window_test.go index 2929991cf5301..03abde8ec702f 100644 --- a/executor/window_test.go +++ b/executor/window_test.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/tidb/util/testkit" ) -func (s *testSuite2) TestWindowFunctions(c *C) { +func (s *testSuite4) TestWindowFunctions(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") diff --git a/executor/write_test.go b/executor/write_test.go index 7bb7906f9a214..1c1ba50866b2a 100644 --- a/executor/write_test.go +++ b/executor/write_test.go @@ -40,7 +40,7 @@ type testBypassSuite struct{} func (s *testBypassSuite) SetUpSuite(c *C) { } -func (s *testSuite2) TestInsert(c *C) { +func (s *testSuite4) TestInsert(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testSQL := `drop table if exists insert_test;create table insert_test (id int PRIMARY KEY AUTO_INCREMENT, c1 int, c2 int, c3 int default 1);` @@ -314,7 +314,7 @@ func (s *testSuite) TestMultiBatch(c *C) { tk.MustExec("admin check table t") } -func (s *testSuite2) TestInsertAutoInc(c *C) { +func (s *testSuite4) TestInsertAutoInc(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") createSQL := `drop table if exists insert_autoinc_test; create table insert_autoinc_test (id int primary key auto_increment, c1 int);` @@ -447,7 +447,7 @@ func (s *testSuite2) TestInsertAutoInc(c *C) { r.Check(testkit.Rows(rowStr4, rowStr1, rowStr2, rowStr3, rowStr5, rowStr6, rowStr7, rowStr8)) } -func (s *testSuite2) TestInsertIgnore(c *C) { +func (s *testSuite4) TestInsertIgnore(c *C) { var cfg kv.InjectionConfig tk := testkit.NewTestKit(c, kv.NewInjectedStore(s.store, &cfg)) tk.MustExec("use test") @@ -561,7 +561,7 @@ commit;` tk.MustQuery(testSQL).Check(testkit.Rows("0")) } -func (s *testSuite2) TestInsertOnDup(c *C) { +func (s *testSuite4) TestInsertOnDup(c *C) { var cfg kv.InjectionConfig tk := testkit.NewTestKit(c, kv.NewInjectedStore(s.store, &cfg)) tk.MustExec("use test") @@ -726,7 +726,7 @@ commit;` tk.MustQuery(`SELECT * FROM t1 order by f1;`).Check(testkit.Rows("1 0", "2 2")) } -func (s *testSuite2) TestInsertIgnoreOnDup(c *C) { +func (s *testSuite4) TestInsertIgnoreOnDup(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testSQL := `drop table if exists t; @@ -749,7 +749,7 @@ func (s *testSuite2) TestInsertIgnoreOnDup(c *C) { r.Check(testkit.Rows("1 1", "2 2")) } -func (s *testSuite2) TestReplace(c *C) { +func (s *testSuite4) TestReplace(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testSQL := `drop table if exists replace_test; @@ -900,7 +900,7 @@ func (s *testSuite2) TestReplace(c *C) { tk.CheckLastMessage("Records: 1 Duplicates: 1 Warnings: 0") } -func (s *testSuite2) TestPartitionedTableReplace(c *C) { +func (s *testSuite4) TestPartitionedTableReplace(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") testSQL := `drop table if exists replace_test; @@ -1039,7 +1039,7 @@ func (s *testSuite2) TestPartitionedTableReplace(c *C) { r.Check(testkit.Rows("111 2")) } -func (s *testSuite2) TestHashPartitionedTableReplace(c *C) { +func (s *testSuite4) TestHashPartitionedTableReplace(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("set @@session.tidb_enable_table_partition = '1';") @@ -1356,7 +1356,7 @@ func (s *testSuite) TestUpdate(c *C) { tk.MustExec("drop view v") } -func (s *testSuite2) TestPartitionedTableUpdate(c *C) { +func (s *testSuite4) TestPartitionedTableUpdate(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t") @@ -1464,7 +1464,7 @@ func (s *testSuite2) TestPartitionedTableUpdate(c *C) { } // TestUpdateCastOnlyModifiedValues for issue #4514. -func (s *testSuite2) TestUpdateCastOnlyModifiedValues(c *C) { +func (s *testSuite4) TestUpdateCastOnlyModifiedValues(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table update_modified (col_1 int, col_2 enum('a', 'b'))") @@ -1499,7 +1499,7 @@ func (s *testSuite2) TestUpdateCastOnlyModifiedValues(c *C) { r.Check(testkit.Rows(`{"a": "测试"}`)) } -func (s *testSuite2) fillMultiTableForUpdate(tk *testkit.TestKit) { +func (s *testSuite4) fillMultiTableForUpdate(tk *testkit.TestKit) { // Create and fill table items tk.MustExec("CREATE TABLE items (id int, price TEXT);") tk.MustExec(`insert into items values (11, "items_price_11"), (12, "items_price_12"), (13, "items_price_13");`) @@ -1510,7 +1510,7 @@ func (s *testSuite2) fillMultiTableForUpdate(tk *testkit.TestKit) { tk.CheckExecResult(3, 0) } -func (s *testSuite2) TestMultipleTableUpdate(c *C) { +func (s *testSuite4) TestMultipleTableUpdate(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") s.fillMultiTableForUpdate(tk) @@ -1618,7 +1618,7 @@ func (s *testSuite) TestDelete(c *C) { tk.MustExec("drop view v") } -func (s *testSuite2) TestPartitionedTableDelete(c *C) { +func (s *testSuite4) TestPartitionedTableDelete(c *C) { createTable := `CREATE TABLE test.t (id int not null default 1, name varchar(255), index(id)) PARTITION BY RANGE ( id ) ( PARTITION p0 VALUES LESS THAN (6), @@ -1671,7 +1671,7 @@ func (s *testSuite2) TestPartitionedTableDelete(c *C) { tk.MustExec(`drop table t1;`) } -func (s *testSuite2) fillDataMultiTable(tk *testkit.TestKit) { +func (s *testSuite4) fillDataMultiTable(tk *testkit.TestKit) { tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2, t3") // Create and fill table t1 @@ -1688,7 +1688,7 @@ func (s *testSuite2) fillDataMultiTable(tk *testkit.TestKit) { tk.CheckExecResult(3, 0) } -func (s *testSuite2) TestMultiTableDelete(c *C) { +func (s *testSuite4) TestMultiTableDelete(c *C) { tk := testkit.NewTestKit(c, s.store) s.fillDataMultiTable(tk) @@ -1700,7 +1700,7 @@ func (s *testSuite2) TestMultiTableDelete(c *C) { c.Assert(r.Rows(), HasLen, 3) } -func (s *testSuite2) TestQualifiedDelete(c *C) { +func (s *testSuite4) TestQualifiedDelete(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1") @@ -1736,7 +1736,7 @@ func (s *testSuite2) TestQualifiedDelete(c *C) { c.Assert(err, NotNil) } -func (s *testSuite2) TestLoadData(c *C) { +func (s *testSuite4) TestLoadData(c *C) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -1902,7 +1902,7 @@ func (s *testSuite2) TestLoadData(c *C) { checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) } -func (s *testSuite2) TestLoadDataEscape(c *C) { +func (s *testSuite4) TestLoadDataEscape(c *C) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test; drop table if exists load_data_test;") @@ -1931,7 +1931,7 @@ func (s *testSuite2) TestLoadDataEscape(c *C) { } // TestLoadDataSpecifiedColumns reuse TestLoadDataEscape's test case :-) -func (s *testSuite2) TestLoadDataSpecifiedColumns(c *C) { +func (s *testSuite4) TestLoadDataSpecifiedColumns(c *C) { trivialMsg := "Records: 1 Deleted: 0 Skipped: 0 Warnings: 0" tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test; drop table if exists load_data_test;") @@ -1958,7 +1958,7 @@ func (s *testSuite2) TestLoadDataSpecifiedColumns(c *C) { checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) } -func (s *testSuite2) TestLoadDataIgnoreLines(c *C) { +func (s *testSuite4) TestLoadDataIgnoreLines(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test; drop table if exists load_data_test;") tk.MustExec("CREATE TABLE load_data_test (id INT NOT NULL PRIMARY KEY, value TEXT NOT NULL) CHARACTER SET utf8") @@ -1978,7 +1978,7 @@ func (s *testSuite2) TestLoadDataIgnoreLines(c *C) { } // related to issue 6360 -func (s *testSuite2) TestLoadDataOverflowBigintUnsigned(c *C) { +func (s *testSuite4) TestLoadDataOverflowBigintUnsigned(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test; drop table if exists load_data_test;") tk.MustExec("CREATE TABLE load_data_test (a bigint unsigned);") @@ -1997,7 +1997,7 @@ func (s *testSuite2) TestLoadDataOverflowBigintUnsigned(c *C) { checkCases(tests, ld, c, tk, ctx, selectSQL, deleteSQL) } -func (s *testSuite2) TestBatchInsertDelete(c *C) { +func (s *testSuite4) TestBatchInsertDelete(c *C) { originLimit := atomic.LoadUint64(&kv.TxnEntryCountLimit) defer func() { atomic.StoreUint64(&kv.TxnEntryCountLimit, originLimit) @@ -2114,7 +2114,7 @@ func (s *testSuite2) TestBatchInsertDelete(c *C) { r.Check(testkit.Rows("0")) } -func (s *testSuite2) TestNullDefault(c *C) { +func (s *testSuite4) TestNullDefault(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test; drop table if exists test_null_default;") tk.MustExec("set timestamp = 1234") @@ -2126,7 +2126,7 @@ func (s *testSuite2) TestNullDefault(c *C) { tk.MustQuery("select * from test_null_default").Check(testkit.Rows("", "1970-01-01 08:20:34")) } -func (s *testSuite2) TestNotNullDefault(c *C) { +func (s *testSuite4) TestNotNullDefault(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test; drop table if exists t1,t2;") defer tk.MustExec("drop table t1,t2") @@ -2180,7 +2180,7 @@ func (s *testBypassSuite) TestLatch(c *C) { } // TestIssue4067 Test issue https://github.com/pingcap/tidb/issues/4067 -func (s *testSuite2) TestIssue4067(c *C) { +func (s *testSuite4) TestIssue4067(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("drop table if exists t1, t2") @@ -2194,7 +2194,7 @@ func (s *testSuite2) TestIssue4067(c *C) { tk.MustQuery("select * from t1").Check(nil) } -func (s *testSuite2) TestInsertCalculatedValue(c *C) { +func (s *testSuite4) TestInsertCalculatedValue(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") @@ -2310,7 +2310,7 @@ func (s *testSuite2) TestInsertCalculatedValue(c *C) { tk.MustQuery("select * from t").Check(testkit.Rows("4 0 2")) } -func (s *testSuite2) TestDataTooLongErrMsg(c *C) { +func (s *testSuite4) TestDataTooLongErrMsg(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table t(a varchar(2));") @@ -2323,7 +2323,7 @@ func (s *testSuite2) TestDataTooLongErrMsg(c *C) { c.Assert(err.Error(), Equals, "[types:1406]Data too long for column 'a' at row 1") } -func (s *testSuite2) TestUpdateSelect(c *C) { +func (s *testSuite4) TestUpdateSelect(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table msg (id varchar(8), b int, status int, primary key (id, b))") @@ -2335,7 +2335,7 @@ func (s *testSuite2) TestUpdateSelect(c *C) { tk.MustExec("admin check table msg") } -func (s *testSuite2) TestUpdateDelete(c *C) { +func (s *testSuite4) TestUpdateDelete(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("CREATE TABLE ttt (id bigint(20) NOT NULL, host varchar(30) NOT NULL, PRIMARY KEY (id), UNIQUE KEY i_host (host));") @@ -2355,7 +2355,7 @@ func (s *testSuite2) TestUpdateDelete(c *C) { tk.MustExec("drop table ttt") } -func (s *testSuite2) TestUpdateAffectRowCnt(c *C) { +func (s *testSuite4) TestUpdateAffectRowCnt(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table a(id int auto_increment, a int default null, primary key(id))") @@ -2374,7 +2374,7 @@ func (s *testSuite2) TestUpdateAffectRowCnt(c *C) { tk.CheckLastMessage("Rows matched: 2 Changed: 2 Warnings: 0") } -func (s *testSuite2) TestReplaceLog(c *C) { +func (s *testSuite4) TestReplaceLog(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table testLog (a int not null primary key, b int unique key);`) @@ -2409,7 +2409,7 @@ func (s *testSuite2) TestReplaceLog(c *C) { // TestRebaseIfNeeded is for issue 7422. // There is no need to do the rebase when updating a record if the auto-increment ID not changed. // This could make the auto ID increasing speed slower. -func (s *testSuite2) TestRebaseIfNeeded(c *C) { +func (s *testSuite4) TestRebaseIfNeeded(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec(`create table t (a int not null primary key auto_increment, b int unique key);`) @@ -2441,7 +2441,7 @@ func (s *testSuite2) TestRebaseIfNeeded(c *C) { tk.MustQuery(`select a from t where b = 6;`).Check(testkit.Rows("30003")) } -func (s *testSuite2) TestDeferConstraintCheckForInsert(c *C) { +func (s *testSuite4) TestDeferConstraintCheckForInsert(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) tk.MustExec(`drop table if exists t;create table t (i int key);`) @@ -2455,7 +2455,7 @@ func (s *testSuite2) TestDeferConstraintCheckForInsert(c *C) { tk.MustQuery(`select * from t;`).Check(testkit.Rows("2")) } -func (s *testSuite2) TestDefEnumInsert(c *C) { +func (s *testSuite4) TestDefEnumInsert(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") tk.MustExec("create table test (id int, prescription_type enum('a','b','c','d','e','f') NOT NULL, primary key(id));") @@ -2463,7 +2463,7 @@ func (s *testSuite2) TestDefEnumInsert(c *C) { tk.MustQuery("select prescription_type from test").Check(testkit.Rows("a")) } -func (s *testSuite2) TestAutoIDInRetry(c *C) { +func (s *testSuite4) TestAutoIDInRetry(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t (id int not null auto_increment primary key)")