From dc9b6917141510e0760429c6a89ecc9927c94afb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Mon, 7 Nov 2022 16:03:51 +0800 Subject: [PATCH 01/22] extension: add extension API to listen stmt events (#38789) close pingcap/tidb#38788 --- extension/BUILD.bazel | 10 + extension/event_listener_test.go | 403 +++++++++++++++++++++++++++++++ extension/session.go | 69 ++++++ server/BUILD.bazel | 1 + server/conn.go | 18 +- server/conn_stmt.go | 5 +- server/extension.go | 247 +++++++++++++++++++ server/mock_conn.go | 8 +- server/server.go | 27 --- sessionctx/variable/session.go | 16 +- 10 files changed, 766 insertions(+), 38 deletions(-) create mode 100644 extension/event_listener_test.go create mode 100644 server/extension.go diff --git a/extension/BUILD.bazel b/extension/BUILD.bazel index a21f6ea2f4130..67b3351e012a4 100644 --- a/extension/BUILD.bazel +++ b/extension/BUILD.bazel @@ -13,7 +13,10 @@ go_library( importpath = "github.com/pingcap/tidb/extension", visibility = ["//visibility:public"], deps = [ + "//parser", + "//parser/ast", "//parser/auth", + "//sessionctx/stmtctx", "//sessionctx/variable", "//types", "//util/chunk", @@ -27,6 +30,7 @@ go_test( name = "extension_test", srcs = [ "bootstrap_test.go", + "event_listener_test.go", "function_test.go", "main_test.go", "registry_test.go", @@ -34,8 +38,14 @@ go_test( embed = [":extension"], deps = [ "//expression", + "//parser/ast", "//parser/auth", + "//parser/mysql", "//privilege/privileges", + "//server", + "//sessionctx", + "//sessionctx/sessionstates", + "//sessionctx/stmtctx", "//sessionctx/variable", "//testkit", "//testkit/testsetup", diff --git a/extension/event_listener_test.go b/extension/event_listener_test.go new file mode 100644 index 0000000000000..ece046f820b64 --- /dev/null +++ b/extension/event_listener_test.go @@ -0,0 +1,403 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package extension_test + +import ( + "context" + "encoding/binary" + "sort" + "strings" + "testing" + "time" + + "github.com/pingcap/tidb/extension" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/server" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/sessionstates" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/types" + "github.com/stretchr/testify/require" +) + +type stmtEventRecord struct { + tp extension.StmtEventTp + user *auth.UserIdentity + originalText string + redactText string + params []types.Datum + connInfo *variable.ConnectionInfo + err string + tables []stmtctx.TableEntry + affectedRows uint64 + stmtNode ast.StmtNode + executeStmtNode *ast.ExecuteStmt + preparedNode ast.StmtNode +} + +type sessionHandler struct { + records []stmtEventRecord +} + +func (h *sessionHandler) OnStmtEvent(tp extension.StmtEventTp, info extension.StmtEventInfo) { + tables := make([]stmtctx.TableEntry, len(info.RelatedTables())) + copy(tables, info.RelatedTables()) + + redactText, _ := info.SQLDigest() + r := stmtEventRecord{ + tp: tp, + user: info.User(), + originalText: info.OriginalText(), + redactText: redactText, + params: info.PreparedParams(), + connInfo: info.ConnectionInfo(), + tables: tables, + affectedRows: info.AffectedRows(), + stmtNode: info.StmtNode(), + executeStmtNode: info.ExecuteStmtNode(), + preparedNode: info.ExecutePreparedStmt(), + } + + if err := info.GetError(); err != nil { + r.err = err.Error() + } + + h.records = append(h.records, r) +} + +func (h *sessionHandler) Reset() { + h.records = nil +} + +func (h *sessionHandler) GetHandler() *extension.SessionHandler { + return &extension.SessionHandler{ + OnStmtEvent: h.OnStmtEvent, + } +} + +func registerHandler(t *testing.T) *sessionHandler { + h := &sessionHandler{} + err := extension.Register( + "test", + extension.WithSessionHandlerFactory(h.GetHandler), + ) + require.NoError(t, err) + return h +} + +func getPreparedID(t *testing.T, sctx sessionctx.Context) uint32 { + sessStates := &sessionstates.SessionStates{} + require.NoError(t, sctx.GetSessionVars().EncodeSessionStates(context.Background(), sessStates)) + return sessStates.PreparedStmtID +} + +type stmtEventCase struct { + sql string + binaryExecute uint32 + executeParams []paramInfo + + err string + originalText string + redactText string + affectedRows uint64 + tables []stmtctx.TableEntry + parseError bool + prepareNotFound bool + multiQueryCases []stmtEventCase +} + +func TestExtensionStmtEvents(t *testing.T) { + defer extension.Reset() + extension.Reset() + h := registerHandler(t) + require.NoError(t, extension.Setup()) + + store := testkit.CreateMockStore(t) + serv := server.CreateMockServer(t, store) + defer serv.Close() + conn := server.CreateMockConn(t, serv) + defer conn.Close() + + require.NoError(t, conn.HandleQuery(context.Background(), "SET tidb_multi_statement_mode='ON'")) + require.NoError(t, conn.HandleQuery(context.Background(), "use test")) + require.NoError(t, conn.HandleQuery(context.Background(), "create table t1(a int, b int)")) + require.NoError(t, conn.HandleQuery(context.Background(), "create database test2")) + require.NoError(t, conn.HandleQuery(context.Background(), "create table test2.t1(c int, d int)")) + require.NoError(t, conn.HandleQuery(context.Background(), "set @a=1")) + require.NoError(t, conn.HandleQuery(context.Background(), "set @b=2")) + + cmd := append([]byte{mysql.ComStmtPrepare}, []byte("select ?")...) + require.NoError(t, conn.Dispatch(context.Background(), cmd)) + stmtID1 := getPreparedID(t, conn.Context()) + + cmd = append( + []byte{mysql.ComStmtPrepare}, + []byte("select a, b from t1 left join test2.t1 as t2 on t2.c = t1.a where t1.a = 3 and t1.b = ? and t2.d = ?")...) + require.NoError(t, conn.Dispatch(context.Background(), cmd)) + stmtID2 := getPreparedID(t, conn.Context()) + + require.NoError(t, conn.HandleQuery(context.Background(), "create table tnoexist(n int)")) + cmd = append([]byte{mysql.ComStmtPrepare}, []byte("select * from tnoexist where n=?")...) + require.NoError(t, conn.Dispatch(context.Background(), cmd)) + stmtID3 := getPreparedID(t, conn.Context()) + require.NoError(t, conn.HandleQuery(context.Background(), "drop table tnoexist")) + + connID := conn.Context().Session.GetSessionVars().ConnectionID + require.NotEqual(t, uint64(0), connID) + + cases := []stmtEventCase{ + { + sql: "select 1", + redactText: "select ?", + }, + { + sql: "invalid sql", + parseError: true, + err: "[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use line 1 column 7 near \"invalid sql\" ", + }, + { + binaryExecute: stmtID1, + executeParams: []paramInfo{ + {value: 7}, + }, + originalText: "select ?", + redactText: "select ?", + }, + { + sql: "select a, b from t1 where a > 1 and b < 2", + redactText: "select `a` , `b` from `t1` where `a` > ? and `b` < ?", + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t1"}, + }, + }, + { + binaryExecute: stmtID2, + executeParams: []paramInfo{ + {value: 3}, + {value: 4}, + }, + originalText: "select a, b from t1 left join test2.t1 as t2 on t2.c = t1.a where t1.a = 3 and t1.b = ? and t2.d = ?", + redactText: "select `a` , `b` from `t1` left join `test2` . `t1` as `t2` on `t2` . `c` = `t1` . `a` where `t1` . `a` = ? and `t1` . `b` = ? and `t2` . `d` = ?", + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t1"}, + {DB: "test2", Table: "t1"}, + }, + }, + { + binaryExecute: stmtID3, + executeParams: []paramInfo{ + {value: 5}, + }, + originalText: "select * from tnoexist where n=?", + redactText: "select * from `tnoexist` where `n` = ?", + err: "select * from tnoexist where n=? [arguments: 5]: [planner:8113]Schema change caused error: [schema:1146]Table 'test.tnoexist' doesn't exist", + }, + { + sql: "prepare s from 'select * from t1 where a=1 and b>? and b? and b ? and `b` < ?", + executeParams: []paramInfo{ + {value: 1}, + {value: 2}, + }, + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t1"}, + }, + }, + { + sql: "execute sn using @a, @b", + redactText: "execute `sn` using @a , @b", + executeParams: []paramInfo{ + {value: 1}, + {value: 2}, + }, + prepareNotFound: true, + err: "[planner:8111]Prepared statement not found", + }, + { + sql: "insert into t1 values(1, 10), (2, 20)", + redactText: "insert into `t1` values ( ... ) , ( ... )", + affectedRows: 2, + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t1"}, + }, + }, + { + sql: "select 1;select * from t1 where a > 1", + multiQueryCases: []stmtEventCase{ + { + originalText: "select 1;", + redactText: "select ? ;", + }, + { + originalText: "select * from t1 where a > 1", + redactText: "select * from `t1` where `a` > ?", + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t1"}, + }, + }, + }, + }, + } + + for i, c := range cases { + h.Reset() + conn.Context().SetProcessInfo("", time.Now(), mysql.ComSleep, 0) + + var err error + switch { + case c.sql != "": + err = conn.HandleQuery(context.Background(), c.sql) + if c.originalText == "" { + c.originalText = c.sql + } + if c.redactText == "" { + c.redactText = c.sql + } + case c.binaryExecute != 0: + err = conn.Dispatch(context.Background(), getExecuteBytes(c.binaryExecute, false, true, c.executeParams...)) + } + + if c.err != "" { + require.EqualError(t, err, c.err) + } else { + require.NoError(t, err) + } + + subCases := c.multiQueryCases + if subCases == nil { + subCases = []stmtEventCase{c} + } + + require.Equal(t, len(subCases), len(h.records), "%d", i) + for j, subCase := range subCases { + record := h.records[j] + if subCase.err != "" { + require.Equal(t, subCase.err, record.err) + require.Equal(t, extension.StmtError, record.tp) + } else { + require.Empty(t, record.err) + require.Equal(t, extension.StmtSuccess, record.tp) + } + + require.NotNil(t, record.connInfo) + if subCase.parseError { + require.Nil(t, record.stmtNode) + require.Nil(t, record.executeStmtNode) + require.Nil(t, record.preparedNode) + } else { + require.NotNil(t, record.stmtNode) + if subCase.binaryExecute != 0 || strings.HasPrefix(strings.ToLower(subCase.sql), "execute ") { + require.NotNil(t, record.executeStmtNode) + require.Equal(t, record.stmtNode, record.executeStmtNode) + if c.prepareNotFound { + require.Nil(t, record.preparedNode) + } else { + require.NotNil(t, record.preparedNode) + require.NotEqual(t, record.preparedNode, record.executeStmtNode) + } + } else { + require.Nil(t, record.executeStmtNode) + require.Nil(t, record.preparedNode) + } + } + + require.Equal(t, connID, record.connInfo.ConnectionID) + require.Equal(t, "root", record.user.Username) + require.Equal(t, "localhost", record.user.Hostname) + require.Equal(t, "root", record.user.AuthUsername) + require.Equal(t, "%", record.user.AuthHostname) + + require.Equal(t, subCase.originalText, record.originalText) + require.Equal(t, subCase.redactText, record.redactText) + require.Equal(t, subCase.affectedRows, record.affectedRows) + if subCase.tables == nil { + subCase.tables = []stmtctx.TableEntry{} + } + sort.Slice(subCase.tables, func(i, j int) bool { + l := subCase.tables[i] + r := subCase.tables[j] + return l.DB < r.DB || (l.DB == r.DB && l.Table < r.Table) + }) + sort.Slice(record.tables, func(i, j int) bool { + l := subCase.tables[i] + r := subCase.tables[j] + return l.DB < r.DB || (l.DB == r.DB && l.Table < r.Table) + }) + require.Equal(t, subCase.tables, record.tables) + + require.Equal(t, len(subCase.executeParams), len(record.params)) + for k, param := range subCase.executeParams { + require.Equal(t, uint64(param.value), record.params[k].GetUint64()) + } + } + } +} + +type paramInfo struct { + value uint32 + isNull bool +} + +// create bytes for COM_STMT_EXECUTE. It only supports int type for convenience. +func getExecuteBytes(stmtID uint32, useCursor bool, newParam bool, params ...paramInfo) []byte { + nullBitmapLen := (len(params) + 7) >> 3 + buf := make([]byte, 11+nullBitmapLen+len(params)*6) + pos := 0 + buf[pos] = mysql.ComStmtExecute + pos++ + binary.LittleEndian.PutUint32(buf[pos:], stmtID) + pos += 4 + if useCursor { + buf[pos] = 1 + } + pos++ + binary.LittleEndian.PutUint32(buf[pos:], 1) + pos += 4 + for i, param := range params { + if param.isNull { + buf[pos+(i>>3)] |= 1 << (i % 8) + } + } + pos += nullBitmapLen + if newParam { + buf[pos] = 1 + pos++ + for i := 0; i < len(params); i++ { + buf[pos] = mysql.TypeLong + pos++ + buf[pos] = 0 + pos++ + } + } else { + buf[pos] = 0 + pos++ + } + for _, param := range params { + if !param.isNull { + binary.LittleEndian.PutUint32(buf[pos:], param.value) + pos += 4 + } + } + return buf[:pos] +} diff --git a/extension/session.go b/extension/session.go index 65fddb753163a..e35f31ec68920 100644 --- a/extension/session.go +++ b/extension/session.go @@ -15,8 +15,12 @@ package extension import ( + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" ) // ConnEventInfo is the connection info for the event @@ -42,9 +46,54 @@ const ( ConnDisconnected ) +// StmtEventTp is the type of the statement event +type StmtEventTp uint8 + +const ( + // StmtError means the stmt is failed + StmtError StmtEventTp = iota + // StmtSuccess means the stmt is successfully executed + StmtSuccess +) + +// StmtEventInfo is the information of stmt event +type StmtEventInfo interface { + // User returns the user of the session + User() *auth.UserIdentity + // ActiveRoles returns the active roles of the user + ActiveRoles() []*auth.RoleIdentity + // CurrentDB returns the current database + CurrentDB() string + // ConnectionInfo returns the connection info of the current session + ConnectionInfo() *variable.ConnectionInfo + // StmtNode returns the parsed ast of the statement + // When parse error, this method will return a nil value + StmtNode() ast.StmtNode + // ExecuteStmtNode will return the `ast.ExecuteStmt` node when the current statement is EXECUTE, + // otherwise a nil value will be returned + ExecuteStmtNode() *ast.ExecuteStmt + // ExecutePreparedStmt will return the prepared stmt node for the EXECUTE statement. + // If the current statement is not EXECUTE or prepared statement is not found, a nil value will be returned + ExecutePreparedStmt() ast.StmtNode + // PreparedParams will return the params for the EXECUTE statement + PreparedParams() []types.Datum + // OriginalText will return the text of the statement. + // Notice that for the EXECUTE statement, the prepared statement text will be used as the return value + OriginalText() string + // SQLDigest will return the normalized and redact text of the `OriginalText()` + SQLDigest() (normalized string, digest *parser.Digest) + // AffectedRows will return the affected rows of the current statement + AffectedRows() uint64 + // RelatedTables will return the related tables of the current statement + RelatedTables() []stmtctx.TableEntry + // GetError will return the error when the current statement is failed + GetError() error +} + // SessionHandler is used to listen session events type SessionHandler struct { OnConnectionEvent func(ConnEventTp, *ConnEventInfo) + OnStmtEvent func(StmtEventTp, StmtEventInfo) } func newSessionExtensions(es *Extensions) *SessionExtensions { @@ -55,6 +104,9 @@ func newSessionExtensions(es *Extensions) *SessionExtensions { if fn := handler.OnConnectionEvent; fn != nil { connExtensions.connectionEventFuncs = append(connExtensions.connectionEventFuncs, fn) } + if fn := handler.OnStmtEvent; fn != nil { + connExtensions.stmtEventFuncs = append(connExtensions.stmtEventFuncs, fn) + } } } } @@ -64,6 +116,7 @@ func newSessionExtensions(es *Extensions) *SessionExtensions { // SessionExtensions is the extensions type SessionExtensions struct { connectionEventFuncs []func(ConnEventTp, *ConnEventInfo) + stmtEventFuncs []func(StmtEventTp, StmtEventInfo) } // OnConnectionEvent will be called when a connection event happens @@ -76,3 +129,19 @@ func (es *SessionExtensions) OnConnectionEvent(tp ConnEventTp, event *ConnEventI fn(tp, event) } } + +// HasStmtEventListeners returns a bool that indicates if any stmt event listener exists +func (es *SessionExtensions) HasStmtEventListeners() bool { + return es != nil && len(es.stmtEventFuncs) > 0 +} + +// OnStmtEvent will be called when a stmt event happens +func (es *SessionExtensions) OnStmtEvent(tp StmtEventTp, event StmtEventInfo) { + if es == nil { + return + } + + for _, fn := range es.stmtEventFuncs { + fn(tp, event) + } +} diff --git a/server/BUILD.bazel b/server/BUILD.bazel index 8dc0efd947901..abdcf02bdd7a1 100644 --- a/server/BUILD.bazel +++ b/server/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "conn_stmt.go", "driver.go", "driver_tidb.go", + "extension.go", "http_handler.go", "http_status.go", "mock_conn.go", diff --git a/server/conn.go b/server/conn.go index f48e616577830..57da6c02f9e0a 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1864,11 +1864,13 @@ func (cc *clientConn) audit(eventType plugin.GeneralEvent) { // Query `load stats` does not return result either. func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { defer trace.StartRegion(ctx, "handleQuery").End() - sc := cc.ctx.GetSessionVars().StmtCtx + sessVars := cc.ctx.GetSessionVars() + sc := sessVars.StmtCtx prevWarns := sc.GetWarnings() var stmts []ast.StmtNode cc.ctx.GetSessionVars().SetAlloc(cc.chunkAlloc) if stmts, err = cc.ctx.Parse(ctx, sql); err != nil { + cc.onExtensionSQLParseFailed(sql, err) return err } @@ -1905,6 +1907,9 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { // Only pre-build point plans for multi-statement query pointPlans, err = cc.prefetchPointPlanKeys(ctx, stmts) if err != nil { + for _, stmt := range stmts { + cc.onExtensionStmtEnd(stmt, err) + } return err } } @@ -1912,7 +1917,13 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { defer cc.ctx.ClearValue(plannercore.PointPlanKey) } var retryable bool + var lastStmt ast.StmtNode for i, stmt := range stmts { + if lastStmt != nil { + cc.onExtensionStmtEnd(lastStmt, nil) + } + + lastStmt = stmt if len(pointPlans) > 0 { // Save the point plan in Session, so we don't need to build the point plan again. cc.ctx.SetValue(plannercore.PointPlanKey, plannercore.PointPlanVal{Plan: pointPlans[i]}) @@ -1952,6 +1963,11 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { } } } + + if lastStmt != nil { + cc.onExtensionStmtEnd(lastStmt, err) + } + return err } diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 65ad113d6ad5a..26dfd9f38d439 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -204,7 +204,10 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e return errors.Annotate(err, cc.preparedStmt2String(stmtID)) } } - return cc.executePlanCacheStmt(ctx, stmt, args, useCursor) + + err = cc.executePlanCacheStmt(ctx, stmt, args, useCursor) + cc.onExtensionBinaryExecuteEnd(stmt, args, err) + return err } func (cc *clientConn) executePlanCacheStmt(ctx context.Context, stmt interface{}, args []expression.Expression, useCursor bool) (err error) { diff --git a/server/extension.go b/server/extension.go new file mode 100644 index 0000000000000..9973226cff841 --- /dev/null +++ b/server/extension.go @@ -0,0 +1,247 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package server + +import ( + "fmt" + + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/extension" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" + "github.com/pingcap/tidb/parser/auth" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx/stmtctx" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/types" +) + +func (cc *clientConn) onExtensionConnEvent(tp extension.ConnEventTp, err error) { + if cc.extensions == nil { + return + } + + var connInfo *variable.ConnectionInfo + var activeRoles []*auth.RoleIdentity + if ctx := cc.getCtx(); ctx != nil { + sessVars := ctx.GetSessionVars() + connInfo = sessVars.ConnectionInfo + activeRoles = sessVars.ActiveRoles + } + + if connInfo == nil { + connInfo = cc.connectInfo() + } + + info := &extension.ConnEventInfo{ + ConnectionInfo: connInfo, + ActiveRoles: activeRoles, + Error: err, + } + + cc.extensions.OnConnectionEvent(tp, info) +} + +func (cc *clientConn) onExtensionStmtEnd(node interface{}, err error, args ...expression.Expression) { + if !cc.extensions.HasStmtEventListeners() { + return + } + + ctx := cc.getCtx() + if ctx == nil { + return + } + + tp := extension.StmtSuccess + if err != nil { + tp = extension.StmtError + } + + sessVars := ctx.GetSessionVars() + info := &stmtEventInfo{ + sessVars: sessVars, + err: err, + } + + switch stmt := node.(type) { + case *ast.ExecuteStmt: + info.executeStmt = stmt + info.stmtNode = stmt + case PreparedStatement: + info.executeStmtID = uint32(stmt.ID()) + prepared, _ := sessVars.GetPreparedStmtByID(info.executeStmtID) + info.executeStmt = &ast.ExecuteStmt{ + PrepStmt: prepared, + BinaryArgs: args, + } + info.stmtNode = info.executeStmt + case ast.StmtNode: + info.stmtNode = stmt + } + + if sc := sessVars.StmtCtx; sc != nil && err == nil { + info.sc = sc + } else { + info.sc = &stmtctx.StatementContext{} + } + cc.extensions.OnStmtEvent(tp, info) +} + +// onSQLParseFailed will be called when sql parse failed +func (cc *clientConn) onExtensionSQLParseFailed(sql string, err error) { + if !cc.extensions.HasStmtEventListeners() { + return + } + + cc.extensions.OnStmtEvent(extension.StmtError, &stmtEventInfo{ + sessVars: cc.getCtx().GetSessionVars(), + err: err, + failedParseText: sql, + }) +} + +func (cc *clientConn) onExtensionBinaryExecuteEnd(prep PreparedStatement, args []expression.Expression, err error) { + cc.onExtensionStmtEnd(prep, err, args...) +} + +type stmtEventInfo struct { + sessVars *variable.SessionVars + sc *stmtctx.StatementContext + stmtNode ast.StmtNode + // execute info + executeStmtID uint32 + executeStmt *ast.ExecuteStmt + executePreparedCached bool + executePreparedCache *core.PlanCacheStmt + // error will only be valid when the stmt is failed + err error + // failedParseText will only present on parse failed + failedParseText string +} + +func (e *stmtEventInfo) ConnectionInfo() *variable.ConnectionInfo { + return e.sessVars.ConnectionInfo +} + +func (e *stmtEventInfo) StmtNode() ast.StmtNode { + return e.stmtNode +} + +func (e *stmtEventInfo) ExecuteStmtNode() *ast.ExecuteStmt { + return e.executeStmt +} + +func (e *stmtEventInfo) ExecutePreparedStmt() ast.StmtNode { + if cache := e.ensureExecutePreparedCache(); cache != nil { + return cache.PreparedAst.Stmt + } + return nil +} + +func (e *stmtEventInfo) PreparedParams() []types.Datum { + return e.sessVars.PreparedParams +} + +func (e *stmtEventInfo) OriginalText() string { + if sql := e.ensureStmtContextOriginalSQL(); sql != "" { + return sql + } + + if e.executeStmtID != 0 { + return binaryExecuteStmtText(e.executeStmtID) + } + + return e.failedParseText +} + +func (e *stmtEventInfo) SQLDigest() (normalized string, digest *parser.Digest) { + if sql := e.ensureStmtContextOriginalSQL(); sql != "" { + return e.sc.SQLDigest() + } + + if e.executeStmtID != 0 { + return binaryExecuteStmtText(e.executeStmtID), nil + } + + return e.failedParseText, nil +} + +func (e *stmtEventInfo) User() *auth.UserIdentity { + return e.sessVars.User +} + +func (e *stmtEventInfo) ActiveRoles() []*auth.RoleIdentity { + return e.sessVars.ActiveRoles +} + +func (e *stmtEventInfo) CurrentDB() string { + return e.sessVars.CurrentDB +} + +func (e *stmtEventInfo) AffectedRows() uint64 { + if e.sc == nil { + return 0 + } + return e.sc.AffectedRows() +} + +func (e *stmtEventInfo) RelatedTables() []stmtctx.TableEntry { + if e.sc == nil { + return nil + } + return e.sc.Tables +} + +func (e *stmtEventInfo) GetError() error { + return e.err +} + +func (e *stmtEventInfo) ensureExecutePreparedCache() *core.PlanCacheStmt { + if e.executeStmt == nil { + return nil + } + + if !e.executePreparedCached { + e.executePreparedCache, _ = core.GetPreparedStmt(e.executeStmt, e.sessVars) + e.executePreparedCached = true + } + + return e.executePreparedCache +} + +func (e *stmtEventInfo) ensureStmtContextOriginalSQL() string { + if e.sc == nil { + return "" + } + + if sql := e.sc.OriginalSQL; sql != "" { + return sql + } + + if planCache := e.ensureExecutePreparedCache(); planCache != nil { + e.sc.OriginalSQL = planCache.PreparedAst.Stmt.Text() + e.sc.InitSQLDigest(planCache.NormalizedSQL, planCache.SQLDigest) + } + + if e.sc.OriginalSQL == "" && e.executeStmtID == 0 { + e.sc.OriginalSQL = e.stmtNode.Text() + } + + return e.sc.OriginalSQL +} + +func binaryExecuteStmtText(id uint32) string { + return fmt.Sprintf("BINARY EXECUTE (ID %d)", id) +} diff --git a/server/mock_conn.go b/server/mock_conn.go index 93455f6483031..36ad6503db812 100644 --- a/server/mock_conn.go +++ b/server/mock_conn.go @@ -23,6 +23,7 @@ import ( "testing" "github.com/pingcap/tidb/config" + "github.com/pingcap/tidb/extension" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/auth" tmysql "github.com/pingcap/tidb/parser/mysql" @@ -94,8 +95,11 @@ func CreateMockServer(t *testing.T, store kv.Storage) *Server { // CreateMockConn creates a mock connection together with a session. func CreateMockConn(t *testing.T, server *Server) MockConn { + extensions, err := extension.GetExtensions() + require.NoError(t, err) + connID := rand.Uint64() - tc, err := server.driver.OpenCtx(connID, 0, uint8(tmysql.DefaultCollationID), "", nil, nil) + tc, err := server.driver.OpenCtx(connID, 0, uint8(tmysql.DefaultCollationID), "", nil, extensions.NewSessionExtensions()) require.NoError(t, err) cc := &clientConn{ @@ -108,12 +112,14 @@ func CreateMockConn(t *testing.T, server *Server) MockConn { pkt: &packetIO{ bufWriter: bufio.NewWriter(bytes.NewBuffer(nil)), }, + extensions: tc.GetExtensions(), } cc.setCtx(tc) cc.server.rwlock.Lock() server.clients[cc.connectionID] = cc cc.server.rwlock.Unlock() tc.Session.SetSessionManager(server) + tc.Session.GetSessionVars().ConnectionInfo = cc.connectInfo() err = tc.Session.Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost"}, nil, nil) require.NoError(t, err) return &mockConn{ diff --git a/server/server.go b/server/server.go index 24172f22c8824..1b1cce5e882a5 100644 --- a/server/server.go +++ b/server/server.go @@ -57,7 +57,6 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/parser/ast" - "github.com/pingcap/tidb/parser/auth" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/planner/core" @@ -667,32 +666,6 @@ func (cc *clientConn) connectInfo() *variable.ConnectionInfo { return connInfo } -func (cc *clientConn) onExtensionConnEvent(tp extension.ConnEventTp, err error) { - if cc.extensions == nil { - return - } - - var connInfo *variable.ConnectionInfo - var activeRoles []*auth.RoleIdentity - if ctx := cc.getCtx(); ctx != nil { - sessVars := ctx.GetSessionVars() - connInfo = sessVars.ConnectionInfo - activeRoles = sessVars.ActiveRoles - } - - if connInfo == nil { - connInfo = cc.connectInfo() - } - - info := &extension.ConnEventInfo{ - ConnectionInfo: connInfo, - ActiveRoles: activeRoles, - Error: err, - } - - cc.extensions.OnConnectionEvent(tp, info) -} - func (s *Server) checkConnectionCount() error { // When the value of Instance.MaxConnections is 0, the number of connections is unlimited. if int(s.cfg.Instance.MaxConnections) == 0 { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 1065fe62d28d0..e046d28fdfc6f 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -1164,11 +1164,8 @@ type SessionVars struct { // ReadStaleness indicates the staleness duration for the following query ReadStaleness time.Duration - // cached is used to optimze the object allocation. - cached struct { - curr int8 - data [2]stmtctx.StatementContext - } + // cachedStmtCtx is used to optimze the object allocation. + cachedStmtCtx [2]stmtctx.StatementContext // Rng stores the rand_seed1 and rand_seed2 for Rand() function Rng *mathutil.MysqlRng @@ -1381,9 +1378,12 @@ func (s *SessionVars) GetPreparedStmtByID(stmtID uint32) (interface{}, error) { // InitStatementContext initializes a StatementContext, the object is reused to reduce allocation. func (s *SessionVars) InitStatementContext() *stmtctx.StatementContext { - s.cached.curr = (s.cached.curr + 1) % 2 - s.cached.data[s.cached.curr] = stmtctx.StatementContext{} - return &s.cached.data[s.cached.curr] + sc := &s.cachedStmtCtx[0] + if sc == s.StmtCtx { + sc = &s.cachedStmtCtx[1] + } + *sc = stmtctx.StatementContext{} + return sc } // AllocMPPTaskID allocates task id for mpp tasks. It will reset the task id if the query's From 54ff6e3f2f96b2c3a8681c1c5a1319379924ecea Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Mon, 7 Nov 2022 16:19:51 +0800 Subject: [PATCH 02/22] planner: update some UTs from cost model1 to model2 (#38925) ref pingcap/tidb#35240 --- planner/core/physical_plan_test.go | 3 + planner/core/plan_cost_ver2.go | 8 + planner/core/plan_test.go | 1 + planner/core/rule_join_reorder_test.go | 4 + .../core/testdata/join_reorder_suite_out.json | 372 +++++++++--------- planner/core/testdata/plan_suite_out.json | 91 ++--- 6 files changed, 250 insertions(+), 229 deletions(-) diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index 5bf4d3ab7cf3f..b35f73c260535 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -867,6 +867,7 @@ func TestJoinHints(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") var input []string var output []struct { @@ -1096,6 +1097,7 @@ func TestLimitToCopHint(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists tn") tk.MustExec("create table tn(a int, b int, c int, d int, key (a, b, c, d))") tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) @@ -2409,6 +2411,7 @@ func TestNoDecorrelateHint(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int)") tk.MustExec("create table t2(a int primary key, b int)") diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 27960e9c10091..0242777ae6ef5 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/util/paging" "github.com/pingcap/tipb/go-tipb" ) @@ -256,6 +257,13 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, doubleReadCost := sumCostVer2(doubleReadCPUCost, doubleReadRequestCost) p.planCostVer2 = sumCostVer2(indexSideCost, divCostVer2(sumCostVer2(tableSideCost, doubleReadCost), doubleReadConcurrency)) + + if p.ctx.GetSessionVars().EnablePaging && p.expectedCnt > 0 && p.expectedCnt <= paging.Threshold { + // if the expectCnt is below the paging threshold, using paging API + p.Paging = true // TODO: move this operation from cost model to physical optimization + p.planCostVer2 = mulCostVer2(p.planCostVer2, 0.6) + } + p.planCostInit = true return p.planCostVer2.label(p), nil } diff --git a/planner/core/plan_test.go b/planner/core/plan_test.go index 8f8c5342794ec..a31844f5fe722 100644 --- a/planner/core/plan_test.go +++ b/planner/core/plan_test.go @@ -787,6 +787,7 @@ func TestCopPaging(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("set session tidb_enable_paging = 1") tk.MustExec("create table t(id int, c1 int, c2 int, primary key (id), key i(c1))") diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index a0c136d8d4a2f..609b2dde4499a 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -52,6 +52,7 @@ func TestStraightJoinHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, t1, t2, t3, t4;") tk.MustExec("create table t(a int, b int, key(a));") tk.MustExec("create table t1(a int, b int, key(a));") @@ -66,6 +67,7 @@ func TestLeadingJoinHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") tk.MustExec("create table t(a int, b int, key(a));") tk.MustExec("create table t1(a int, b int, key(a));") @@ -312,6 +314,7 @@ func TestJoinOrderHint4Subquery(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") tk.MustExec("create table t(a int, b int, key(a));") tk.MustExec("create table t1(a int, b int, key(a));") @@ -333,6 +336,7 @@ func TestLeadingJoinHint4OuterJoin(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") tk.MustExec("create table t(a int, b int, key(a));") tk.MustExec("create table t1(a int, b int, key(a));") diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 1e385b764f07e..159862538cbe9 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -381,11 +381,11 @@ " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", - " ├─TableReader(Build) 9980.01 root data:Selection", - " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": null }, @@ -413,11 +413,11 @@ " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t.a, test.t1.a)]", - " ├─TableReader(Build) 9980.01 root data:Selection", - " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - " └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo" + " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", + " └─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": null }, @@ -508,12 +508,12 @@ " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t1.b, test.t2.b)]", - " ├─TableReader(Build) 9980.01 root data:Selection", - " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": null }, @@ -540,12 +540,12 @@ " ├─IndexReader(Build) 9990.00 root index:IndexFullScan", " │ └─IndexFullScan 9990.00 cop[tikv] table:t, index:a(a) keep order:false, stats:pseudo", " └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t2.b, test.t1.b)]", - " ├─TableReader(Build) 9980.01 root data:Selection", - " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + " ├─TableReader(Build) 9990.00 root data:Selection", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9980.01 root data:Selection", + " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": null }, @@ -1561,12 +1561,12 @@ " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - " │ ├─TableReader(Build) 9980.01 root data:Selection", - " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - " │ └─TableReader(Probe) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1594,12 +1594,12 @@ " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - " │ ├─TableReader(Build) 9980.01 root data:Selection", - " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - " │ └─TableReader(Probe) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t4.a, test.t8.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", @@ -1626,12 +1626,12 @@ "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1661,12 +1661,12 @@ " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - " │ ├─TableReader(Build) 9980.01 root data:Selection", - " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - " │ └─TableReader(Probe) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1694,12 +1694,12 @@ " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - " │ ├─TableReader(Build) 9980.01 root data:Selection", - " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - " │ └─TableReader(Probe) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t4.a, test.t8.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t8.a))", @@ -1726,12 +1726,12 @@ "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1761,12 +1761,12 @@ " │ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", " │ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - " │ ├─TableReader(Build) 9980.01 root data:Selection", - " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - " │ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - " │ └─TableReader(Probe) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ ├─TableReader(Build) 9990.00 root data:Selection", + " │ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + " │ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + " │ └─TableReader(Probe) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", " └─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1793,12 +1793,12 @@ "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1828,11 +1828,11 @@ "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t5.a, test.t6.a)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─IndexReader(Probe) 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1859,12 +1859,12 @@ "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1955,12 +1955,12 @@ "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -1989,12 +1989,12 @@ "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -2023,12 +2023,12 @@ "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -2057,12 +2057,12 @@ "│ ├─IndexReader(Build) 9990.00 root index:IndexFullScan", "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t5, index:a(a) keep order:false, stats:pseudo", "│ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t6.b, test.t7.b)]", - "│ ├─TableReader(Build) 9980.01 root data:Selection", - "│ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", - "│ │ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", - "│ └─TableReader(Probe) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ ├─TableReader(Build) 9990.00 root data:Selection", + "│ │ └─Selection 9990.00 cop[tikv] not(isnull(test.t7.b))", + "│ │ └─TableFullScan 10000.00 cop[tikv] table:t7 keep order:false, stats:pseudo", + "│ └─TableReader(Probe) 9980.01 root data:Selection", + "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t6.a)), not(isnull(test.t6.b))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t6 keep order:false, stats:pseudo", "└─HashJoin(Probe) 19511.72 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -6316,8 +6316,9 @@ "Plan": [ "HashJoin 4.69 root inner join, equal:[eq(test.t1.a, test.t3.a)]", "├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ └─IndexReader 3.00 root index:IndexFullScan", - "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ └─IndexReader 3.00 root index:StreamAgg", + "│ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7264,8 +7265,9 @@ "└─IndexJoin 5.86 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", " ├─HashJoin(Build) 4.69 root inner join, equal:[eq(test.t1.a, test.t3.a)]", " │ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - " │ │ └─IndexReader 3.00 root index:IndexFullScan", - " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + " │ │ └─IndexReader 3.00 root index:StreamAgg", + " │ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + " │ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", " │ └─HashJoin(Probe) 12475.01 root inner join, equal:[eq(test.t4.b, test.t1.b)]", " │ ├─TableReader(Build) 9980.01 root data:Selection", " │ │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", @@ -7286,8 +7288,9 @@ "IndexJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "├─IndexJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ │ └─IndexReader 3.00 root index:IndexFullScan", - "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", "│ └─IndexLookUp(Probe) 3.75 root ", "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", @@ -7305,8 +7308,9 @@ "IndexJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "├─IndexJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ │ └─IndexReader 3.00 root index:IndexFullScan", - "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", "│ └─IndexLookUp(Probe) 3.75 root ", "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", @@ -7326,8 +7330,9 @@ "IndexJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "├─IndexJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ │ └─IndexReader 3.00 root index:IndexFullScan", - "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", "│ └─IndexLookUp(Probe) 3.75 root ", "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", @@ -7347,8 +7352,9 @@ "IndexJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "├─IndexJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ │ └─IndexReader 3.00 root index:IndexFullScan", - "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", "│ └─IndexLookUp(Probe) 3.75 root ", "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", @@ -7368,8 +7374,9 @@ "IndexJoin 4.69 root inner join, inner:IndexLookUp, outer key:test.t1.a, inner key:test.t2.a, equal cond:eq(test.t1.a, test.t2.a)", "├─IndexJoin(Build) 3.75 root inner join, inner:IndexLookUp, outer key:test.t3.a, inner key:test.t1.a, equal cond:eq(test.t3.a, test.t1.a)", "│ ├─StreamAgg(Build) 3.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ │ └─IndexReader 3.00 root index:IndexFullScan", - "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", + "│ │ └─IndexReader 3.00 root index:StreamAgg", + "│ │ └─StreamAgg 3.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:true", "│ └─IndexLookUp(Probe) 3.75 root ", "│ ├─Selection(Build) 3.75 cop[tikv] not(isnull(test.t1.a))", "│ │ └─IndexRangeScan 3.75 cop[tikv] table:t1, index:a(a) range: decided by [eq(test.t1.a, test.t3.a)], keep order:false, stats:pseudo", @@ -8729,11 +8736,11 @@ " └─Sort(Probe) 12487.50 root test.t1.b", " └─HashJoin 12487.50 root right outer join, equal:[eq(test.t.a, test.t1.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warning": null }, @@ -8753,11 +8760,11 @@ " └─Sort(Probe) 12487.50 root test.t1.b", " └─HashJoin 12487.50 root right outer join, equal:[eq(test.t.a, test.t1.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warning": null }, @@ -8776,11 +8783,11 @@ " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─HashJoin(Probe) 12487.50 root right outer join, equal:[eq(test.t.a, test.t1.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 leading hint is inapplicable, check the join type or the join algorithm hint" @@ -8866,12 +8873,12 @@ "│ └─Selection 9980.01 cop[tikv] not(isnull(test.t3.a)), not(isnull(test.t3.b))", "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "└─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" @@ -9194,12 +9201,12 @@ " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", " └─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" @@ -9220,12 +9227,12 @@ " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", " └─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3, t1) */. Maybe you can use the table alias name", @@ -9247,12 +9254,12 @@ " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", " └─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t4, t3) */. Maybe you can use the table alias name", @@ -9299,12 +9306,12 @@ " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.b))", " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", " └─HashJoin(Probe) 12475.01 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - " └─TableReader(Probe) 9980.01 root data:Selection", - " └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + " ├─TableReader(Build) 9980.01 root data:Selection", + " │ └─Selection 9980.01 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " └─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 There are no matching table names for (t3) in optimizer hint /*+ LEADING(t3) */. Maybe you can use the table alias name" @@ -9342,9 +9349,10 @@ "Plan": [ "Projection 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", "└─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t3.a)]", - " ├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - " │ └─IndexReader 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", + " ├─StreamAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + " │ └─IndexReader 7992.00 root index:StreamAgg", + " │ └─StreamAgg 7992.00 cop[tikv] group by:test.t3.a, ", + " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", " └─HashJoin(Probe) 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -9363,16 +9371,17 @@ "SQL": "select /*+ leading(t3@sel_2) */ * from t1 left join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", "Plan": [ "HashJoin 12487.50 root left outer join, equal:[eq(test.t1.a, test.t2.a)]", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─HashJoin(Probe) 9990.00 root inner join, equal:[eq(test.t3.a, test.t1.a)]", - " ├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - " │ └─IndexReader 9990.00 root index:IndexFullScan", - " │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", - " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "├─HashJoin(Build) 9990.00 root inner join, equal:[eq(test.t3.a, test.t1.a)]", + "│ ├─StreamAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ │ └─IndexReader 7992.00 root index:StreamAgg", + "│ │ └─StreamAgg 7992.00 cop[tikv] group by:test.t3.a, ", + "│ │ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", + "│ └─TableReader(Probe) 9990.00 root data:Selection", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ], "Warning": null }, @@ -9380,9 +9389,10 @@ "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a in (select t3.a from t3)", "Plan": [ "HashJoin 12487.50 root inner join, equal:[eq(test.t1.a, test.t3.a)]", - "├─HashAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", - "│ └─IndexReader 9990.00 root index:IndexFullScan", - "│ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:false, stats:pseudo", + "├─StreamAgg(Build) 7992.00 root group by:test.t3.a, funcs:firstrow(test.t3.a)->test.t3.a", + "│ └─IndexReader 7992.00 root index:StreamAgg", + "│ └─StreamAgg 7992.00 cop[tikv] group by:test.t3.a, ", + "│ └─IndexFullScan 9990.00 cop[tikv] table:t3, index:a(a) keep order:true, stats:pseudo", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -9550,11 +9560,11 @@ "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "└─HashJoin(Probe) 12487.50 root right outer join, equal:[eq(test.t4.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t4.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t4 keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 There are no matching table names for (t4) in optimizer hint /*+ LEADING(t2, t4) */. Maybe you can use the table alias name" diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 595fc7235ecfc..50315a9784697 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -1343,15 +1343,15 @@ "Cases": [ { "SQL": "select /*+ TIDB_INLJ(t1) */ t1.a, t2.a, t3.a from t t1, t t2, t t3 where t1.a = t2.a and t2.a = t3.a;", - "Best": "RightHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.f)[[NULL,+inf]])}(test.t.a,test.t.a)}(test.t.a,test.t.a)->Projection", + "Best": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.t.a)}(test.t.a,test.t.a)->Projection", "Warning": "", - "Hints": "use_index(@`sel_1` `test`.`t3` `f`), use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` `f`), inl_join(@`sel_1` `test`.`t1`), hash_join(@`sel_1` `test`.`t3`)" + "Hints": "use_index(@`sel_1` `test`.`t3` ), use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), inl_join(@`sel_1` `test`.`t1`), merge_join(@`sel_1` `test`.`t3`)" }, { "SQL": "select /*+ TIDB_INLJ(test.t1) */ t1.a, t2.a, t3.a from t t1, t t2, t t3 where t1.a = t2.a and t2.a = t3.a;", - "Best": "RightHashJoin{IndexReader(Index(t.f)[[NULL,+inf]])->IndexJoin{TableReader(Table(t))->IndexReader(Index(t.f)[[NULL,+inf]])}(test.t.a,test.t.a)}(test.t.a,test.t.a)->Projection", + "Best": "MergeInnerJoin{TableReader(Table(t))->IndexJoin{TableReader(Table(t))->TableReader(Table(t))}(test.t.a,test.t.a)}(test.t.a,test.t.a)->Projection", "Warning": "", - "Hints": "use_index(@`sel_1` `test`.`t3` `f`), use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` `f`), inl_join(@`sel_1` `test`.`t1`), hash_join(@`sel_1` `test`.`t3`)" + "Hints": "use_index(@`sel_1` `test`.`t3` ), use_index(@`sel_1` `test`.`t1` ), use_index(@`sel_1` `test`.`t2` ), inl_join(@`sel_1` `test`.`t1`), merge_join(@`sel_1` `test`.`t3`)" }, { "SQL": "select /*+ TIDB_INLJ(t1) */ t1.b, t2.a from t t1, t t2 where t1.b = t2.a;", @@ -1660,9 +1660,10 @@ "SQL": "select * from tn where a = 1 and b > 10 and b < 20 and c > 50 order by d limit 1", "Plan": [ "TopN 0.83 root test.tn.d, offset:0, count:1", - "└─IndexReader 0.83 root index:Selection", - " └─Selection 0.83 cop[tikv] gt(test.tn.c, 50)", - " └─IndexRangeScan 2.50 cop[tikv] table:tn, index:a(a, b, c, d) range:(1 10,1 20), keep order:false, stats:pseudo" + "└─IndexReader 0.83 root index:TopN", + " └─TopN 0.83 cop[tikv] test.tn.d, offset:0, count:1", + " └─Selection 0.83 cop[tikv] gt(test.tn.c, 50)", + " └─IndexRangeScan 2.50 cop[tikv] table:tn, index:a(a, b, c, d) range:(1 10,1 20), keep order:false, stats:pseudo" ], "Warning": null }, @@ -6166,9 +6167,9 @@ "SQL": "SELECT ta.NAME,(SELECT sum(tb.CODE) FROM tb WHERE ta.id = tb.id) tb_sum_code FROM ta WHERE ta.NAME LIKE 'chad999%'", "Plan": [ "HashJoin 250.00 root left outer join, equal:[eq(test.ta.id, test.tb.id)]", - "├─IndexLookUp(Build) 250.00 root ", - "│ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:ta, index:idx_ta_name(name) range:[\"chad999\",\"chad99:\"), keep order:false, stats:pseudo", - "│ └─TableRowIDScan(Probe) 250.00 cop[tikv] table:ta keep order:false, stats:pseudo", + "├─TableReader(Build) 250.00 root data:Selection", + "│ └─Selection 250.00 cop[tikv] like(test.ta.name, \"chad999%\", 92)", + "│ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", "└─HashAgg(Probe) 7992.00 root group by:test.tb.id, funcs:sum(Column#19)->Column#13, funcs:firstrow(test.tb.id)->test.tb.id", " └─TableReader 7992.00 root data:HashAgg", " └─HashAgg 7992.00 cop[tikv] group by:test.tb.id, funcs:sum(test.tb.code)->Column#19", @@ -6183,9 +6184,9 @@ "Plan": [ "Projection 250.00 root test.ta.name, Column#13", "└─Apply 250.00 root CARTESIAN left outer join", - " ├─IndexLookUp(Build) 250.00 root ", - " │ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:ta, index:idx_ta_name(name) range:[\"chad999\",\"chad99:\"), keep order:false, stats:pseudo", - " │ └─TableRowIDScan(Probe) 250.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " ├─TableReader(Build) 250.00 root data:Selection", + " │ └─Selection 250.00 cop[tikv] like(test.ta.name, \"chad999%\", 92)", + " │ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", " └─MaxOneRow(Probe) 250.00 root ", " └─StreamAgg 250.00 root funcs:sum(Column#21)->Column#13", " └─Projection 2500.00 root cast(test.tb.code, decimal(10,0) BINARY)->Column#21", @@ -6200,9 +6201,9 @@ "SQL": "SELECT ta.NAME,(SELECT sum(tb.CODE) FROM tb WHERE ta.id = tb.id and exists (select 1 from tc where tb.name=tc.name and tc.`code` like '999%')) tb_sum_code FROM ta WHERE ta.NAME LIKE 'chad999%'", "Plan": [ "HashJoin 250.00 root left outer join, equal:[eq(test.ta.id, test.tb.id)]", - "├─IndexLookUp(Build) 250.00 root ", - "│ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:ta, index:idx_ta_name(name) range:[\"chad999\",\"chad99:\"), keep order:false, stats:pseudo", - "│ └─TableRowIDScan(Probe) 250.00 cop[tikv] table:ta keep order:false, stats:pseudo", + "├─TableReader(Build) 250.00 root data:Selection", + "│ └─Selection 250.00 cop[tikv] like(test.ta.name, \"chad999%\", 92)", + "│ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", "└─HashAgg(Probe) 6387.21 root group by:Column#39, funcs:sum(Column#37)->Column#18, funcs:firstrow(Column#38)->test.tb.id", " └─Projection 7984.01 root cast(test.tb.code, decimal(10,0) BINARY)->Column#37, test.tb.id, test.tb.id", " └─HashJoin 7984.01 root semi join, equal:[eq(test.tb.name, test.tc.name)]", @@ -6221,9 +6222,9 @@ "Plan": [ "Projection 250.00 root test.ta.name, Column#18", "└─Apply 250.00 root CARTESIAN left outer join", - " ├─IndexLookUp(Build) 250.00 root ", - " │ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:ta, index:idx_ta_name(name) range:[\"chad999\",\"chad99:\"), keep order:false, stats:pseudo", - " │ └─TableRowIDScan(Probe) 250.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " ├─TableReader(Build) 250.00 root data:Selection", + " │ └─Selection 250.00 cop[tikv] like(test.ta.name, \"chad999%\", 92)", + " │ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", " └─MaxOneRow(Probe) 250.00 root ", " └─StreamAgg 250.00 root funcs:sum(Column#33)->Column#18", " └─Projection 1998.00 root cast(test.tb.code, decimal(10,0) BINARY)->Column#33", @@ -6245,9 +6246,9 @@ "SQL": "SELECT ta.NAME,(SELECT sum(tb.CODE) FROM tb WHERE ta.id = tb.id and exists (select /*+ no_decorrelate() */ 1 from tc where tb.name=tc.name and tc.`code` like '999%')) tb_sum_code FROM ta WHERE ta.NAME LIKE 'chad999%'", "Plan": [ "HashJoin 250.00 root left outer join, equal:[eq(test.ta.id, test.tb.id)]", - "├─IndexLookUp(Build) 250.00 root ", - "│ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:ta, index:idx_ta_name(name) range:[\"chad999\",\"chad99:\"), keep order:false, stats:pseudo", - "│ └─TableRowIDScan(Probe) 250.00 cop[tikv] table:ta keep order:false, stats:pseudo", + "├─TableReader(Build) 250.00 root data:Selection", + "│ └─Selection 250.00 cop[tikv] like(test.ta.name, \"chad999%\", 92)", + "│ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", "└─HashAgg(Probe) 7992.00 root group by:Column#28, funcs:sum(Column#26)->Column#18, funcs:firstrow(Column#27)->test.tb.id", " └─Projection 9990.00 root cast(test.tb.code, decimal(10,0) BINARY)->Column#26, test.tb.id, test.tb.id", " └─Apply 9990.00 root CARTESIAN semi join", @@ -6267,9 +6268,9 @@ "Plan": [ "Projection 250.00 root test.ta.name, Column#18", "└─Apply 250.00 root CARTESIAN left outer join", - " ├─IndexLookUp(Build) 250.00 root ", - " │ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:ta, index:idx_ta_name(name) range:[\"chad999\",\"chad99:\"), keep order:false, stats:pseudo", - " │ └─TableRowIDScan(Probe) 250.00 cop[tikv] table:ta keep order:false, stats:pseudo", + " ├─TableReader(Build) 250.00 root data:Selection", + " │ └─Selection 250.00 cop[tikv] like(test.ta.name, \"chad999%\", 92)", + " │ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", " └─MaxOneRow(Probe) 250.00 root ", " └─StreamAgg 250.00 root funcs:sum(Column#22)->Column#18", " └─Projection 2500.00 root cast(test.tb.code, decimal(10,0) BINARY)->Column#22", @@ -6330,24 +6331,20 @@ "Plan": [ "HashJoin 159.84 root inner join, equal:[eq(test.ta.id, test.td.id)]", "├─Selection(Build) 159.84 root gt(Column#19, 100)", - "│ └─HashAgg 199.80 root group by:test.td.id, funcs:max(Column#36)->Column#19, funcs:firstrow(test.td.id)->test.td.id", - "│ └─IndexLookUp 199.80 root ", - "│ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:td, index:idx_tc_name(name) range:[\"chad999\",\"chad99:\"), keep order:false, stats:pseudo", - "│ └─HashAgg(Probe) 199.80 cop[tikv] group by:test.td.id, funcs:max(test.td.id)->Column#36", - "│ └─Selection 249.75 cop[tikv] not(isnull(test.td.id))", - "│ └─TableRowIDScan 250.00 cop[tikv] table:td keep order:false, stats:pseudo", + "│ └─HashAgg 199.80 root group by:test.td.id, funcs:max(test.td.id)->Column#19, funcs:firstrow(test.td.id)->test.td.id", + "│ └─TableReader 249.75 root data:Selection", + "│ └─Selection 249.75 cop[tikv] like(test.td.name, \"chad999%\", 92), not(isnull(test.td.id))", + "│ └─TableFullScan 10000.00 cop[tikv] table:td keep order:false, stats:pseudo", "└─HashJoin(Probe) 200.00 root inner join, equal:[eq(test.ta.name, test.tc.name)]", " ├─Selection(Build) 160.00 root gt(Column#14, 100)", - " │ └─HashAgg 200.00 root group by:test.tc.name, funcs:max(Column#27)->Column#14, funcs:firstrow(test.tc.name)->test.tc.name", - " │ └─IndexLookUp 200.00 root ", - " │ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:tc, index:idx_tc_name(name) range:[\"chad99\",\"chad9:\"), keep order:false, stats:pseudo", - " │ └─HashAgg(Probe) 200.00 cop[tikv] group by:test.tc.name, funcs:max(test.tc.id)->Column#27", - " │ └─TableRowIDScan 250.00 cop[tikv] table:tc keep order:false, stats:pseudo", + " │ └─HashAgg 200.00 root group by:test.tc.name, funcs:max(test.tc.id)->Column#14, funcs:firstrow(test.tc.name)->test.tc.name", + " │ └─TableReader 250.00 root data:Selection", + " │ └─Selection 250.00 cop[tikv] like(test.tc.name, \"chad99%\", 92), not(isnull(test.tc.name))", + " │ └─TableFullScan 10000.00 cop[tikv] table:tc keep order:false, stats:pseudo", " └─HashJoin(Probe) 7976.02 root semi join, equal:[eq(test.ta.code, test.tb.code)]", - " ├─IndexLookUp(Build) 249.75 root ", - " │ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:tb, index:idx_tb_name(name) range:[\"chad9\",\"chad:\"), keep order:false, stats:pseudo", - " │ └─Selection(Probe) 249.75 cop[tikv] not(isnull(test.tb.code))", - " │ └─TableRowIDScan 250.00 cop[tikv] table:tb keep order:false, stats:pseudo", + " ├─TableReader(Build) 249.75 root data:Selection", + " │ └─Selection 249.75 cop[tikv] like(test.tb.name, \"chad9%\", 92), not(isnull(test.tb.code))", + " │ └─TableFullScan 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo", " └─TableReader(Probe) 9970.03 root data:Selection", " └─Selection 9970.03 cop[tikv] not(isnull(test.ta.code)), not(isnull(test.ta.id)), not(isnull(test.ta.name))", " └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo" @@ -6363,10 +6360,9 @@ " ├─Apply(Build) 249.75 root CARTESIAN inner join", " │ ├─IndexJoin(Build) 249.75 root inner join, inner:IndexLookUp, outer key:test.tb.code, inner key:test.ta.code, equal cond:eq(test.tb.code, test.ta.code)", " │ │ ├─HashAgg(Build) 199.80 root group by:test.tb.code, funcs:firstrow(test.tb.code)->test.tb.code", - " │ │ │ └─IndexLookUp 249.75 root ", - " │ │ │ ├─IndexRangeScan(Build) 250.00 cop[tikv] table:tb, index:idx_tb_name(name) range:[\"chad9\",\"chad:\"), keep order:false, stats:pseudo", - " │ │ │ └─Selection(Probe) 249.75 cop[tikv] not(isnull(test.tb.code))", - " │ │ │ └─TableRowIDScan 250.00 cop[tikv] table:tb keep order:false, stats:pseudo", + " │ │ │ └─TableReader 249.75 root data:Selection", + " │ │ │ └─Selection 249.75 cop[tikv] like(test.tb.name, \"chad9%\", 92), not(isnull(test.tb.code))", + " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:tb keep order:false, stats:pseudo", " │ │ └─IndexLookUp(Probe) 249.75 root ", " │ │ ├─Selection(Build) 249.75 cop[tikv] not(isnull(test.ta.code))", " │ │ │ └─IndexRangeScan 250.00 cop[tikv] table:ta, index:idx_ta_code(code) range: decided by [eq(test.ta.code, test.tb.code)], keep order:false, stats:pseudo", @@ -6404,10 +6400,9 @@ " │ ├─Apply(Build) 10000.00 root CARTESIAN semi join", " │ │ ├─TableReader(Build) 10000.00 root data:TableFullScan", " │ │ │ └─TableFullScan 10000.00 cop[tikv] table:ta keep order:false, stats:pseudo", - " │ │ └─IndexLookUp(Probe) 2500.00 root ", - " │ │ ├─IndexRangeScan(Build) 2500000.00 cop[tikv] table:tb, index:idx_tb_name(name) range:[\"chad9\",\"chad:\"), keep order:false, stats:pseudo", - " │ │ └─Selection(Probe) 2500.00 cop[tikv] eq(test.ta.code, test.tb.code)", - " │ │ └─TableRowIDScan 2500000.00 cop[tikv] table:tb keep order:false, stats:pseudo", + " │ │ └─TableReader(Probe) 2500.00 root data:Selection", + " │ │ └─Selection 2500.00 cop[tikv] eq(test.ta.code, test.tb.code), like(test.tb.name, \"chad9%\", 92)", + " │ │ └─TableFullScan 100000000.00 cop[tikv] table:tb keep order:false, stats:pseudo", " │ └─Selection(Probe) 8000.00 root gt(Column#14, 100)", " │ └─MaxOneRow 10000.00 root ", " │ └─StreamAgg 10000.00 root funcs:max(test.tc.id)->Column#14", From f79e67f98b90960da64fd3a62ff4f41ff2fc9168 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Mon, 7 Nov 2022 19:55:51 +0800 Subject: [PATCH 03/22] executor: insert ignore return errAutoincReadFailed warning (#38486) close pingcap/tidb#38483 --- executor/executor.go | 1 + executor/insert.go | 16 ++++++++- executor/insert_test.go | 8 +++++ sessionctx/stmtctx/stmtctx.go | 63 ++++++++++++++++++----------------- 4 files changed, 56 insertions(+), 32 deletions(-) diff --git a/executor/executor.go b/executor/executor.go index 6fa7481290c6f..542734e4b1a75 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -2042,6 +2042,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { sc.DupKeyAsWarning = stmt.IgnoreErr sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr sc.IgnoreNoPartition = stmt.IgnoreErr + sc.ErrAutoincReadFailedAsWarning = stmt.IgnoreErr sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr sc.AllowInvalidDate = vars.SQLMode.HasAllowInvalidDatesMode() diff --git a/executor/insert.go b/executor/insert.go index 4f78d0b2809ac..9b286297351b9 100644 --- a/executor/insert.go +++ b/executor/insert.go @@ -22,11 +22,14 @@ import ( "time" "github.com/opentracing/opentracing-go" + "github.com/pingcap/errors" + "github.com/pingcap/tidb/errno" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -310,7 +313,18 @@ func (e *InsertExec) Next(ctx context.Context, req *chunk.Chunk) error { if len(e.children) > 0 && e.children[0] != nil { return insertRowsFromSelect(ctx, e) } - return insertRows(ctx, e) + err := insertRows(ctx, e) + if err != nil { + terr, ok := errors.Cause(err).(*terror.Error) + if ok && len(e.OnDuplicate) == 0 && + e.ctx.GetSessionVars().StmtCtx.ErrAutoincReadFailedAsWarning && + terr.Code() == errno.ErrAutoincReadFailed { + e.ctx.GetSessionVars().StmtCtx.AppendWarning(err) + return nil + } + return err + } + return nil } // Close implements the Executor Close interface. diff --git a/executor/insert_test.go b/executor/insert_test.go index f81be28e43c70..53d0c62cc9776 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -398,6 +398,14 @@ func TestInsertWrongValueForField(t *testing.T) { tk.MustExec(`SET @@sql_mode='STRICT_TRANS_TABLES'`) tk.MustGetErrMsg(`INSERT INTO ts (id, time1) VALUES (2, TIMESTAMP '1018-12-24 00:00:00')`, `[table:1292]Incorrect timestamp value: '1018-12-24 00:00:00' for column 'time1' at row 1`) tk.MustExec(`DROP TABLE ts`) + + tk.MustExec(`CREATE TABLE t0(c0 SMALLINT AUTO_INCREMENT PRIMARY KEY);`) + tk.MustExec(`INSERT IGNORE INTO t0(c0) VALUES (194626268);`) + tk.MustExec(`INSERT IGNORE INTO t0(c0) VALUES ('*')`) + tk.MustQuery(`SHOW WARNINGS`).Check(testkit.Rows( + `Warning 1366 Incorrect smallint value: '*' for column 'c0' at row 1`, + `Warning 1690 constant 32768 overflows smallint`, + `Warning 1467 Failed to read auto-increment value from storage engine`)) } func TestInsertValueForCastDecimalField(t *testing.T) { diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 86e3b36dc1f73..bd716e8509938 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -109,37 +109,38 @@ type StatementContext struct { // IsDDLJobInQueue is used to mark whether the DDL job is put into the queue. // If IsDDLJobInQueue is true, it means the DDL job is in the queue of storage, and it can be handled by the DDL worker. - IsDDLJobInQueue bool - DDLJobID int64 - InInsertStmt bool - InUpdateStmt bool - InDeleteStmt bool - InSelectStmt bool - InLoadDataStmt bool - InExplainStmt bool - InCreateOrAlterStmt bool - InSetSessionStatesStmt bool - InPreparedPlanBuilding bool - IgnoreTruncate bool - IgnoreZeroInDate bool - NoZeroDate bool - DupKeyAsWarning bool - BadNullAsWarning bool - DividedByZeroAsWarning bool - TruncateAsWarning bool - OverflowAsWarning bool - InShowWarning bool - UseCache bool - BatchCheck bool - InNullRejectCheck bool - AllowInvalidDate bool - IgnoreNoPartition bool - SkipPlanCache bool - IgnoreExplainIDSuffix bool - SkipUTF8Check bool - SkipASCIICheck bool - SkipUTF8MB4Check bool - MultiSchemaInfo *model.MultiSchemaInfo + IsDDLJobInQueue bool + DDLJobID int64 + InInsertStmt bool + InUpdateStmt bool + InDeleteStmt bool + InSelectStmt bool + InLoadDataStmt bool + InExplainStmt bool + InCreateOrAlterStmt bool + InSetSessionStatesStmt bool + InPreparedPlanBuilding bool + IgnoreTruncate bool + IgnoreZeroInDate bool + NoZeroDate bool + DupKeyAsWarning bool + BadNullAsWarning bool + DividedByZeroAsWarning bool + TruncateAsWarning bool + OverflowAsWarning bool + ErrAutoincReadFailedAsWarning bool + InShowWarning bool + UseCache bool + BatchCheck bool + InNullRejectCheck bool + AllowInvalidDate bool + IgnoreNoPartition bool + SkipPlanCache bool + IgnoreExplainIDSuffix bool + SkipUTF8Check bool + SkipASCIICheck bool + SkipUTF8MB4Check bool + MultiSchemaInfo *model.MultiSchemaInfo // If the select statement was like 'select * from t as of timestamp ...' or in a stale read transaction // or is affected by the tidb_read_staleness session variable, then the statement will be makred as isStaleness // in stmtCtx From 32248e25f2a955473b1bab18e9a5988f5aa4f2d4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Mon, 7 Nov 2022 22:57:50 +0800 Subject: [PATCH 04/22] *: fix panic when reusing `OptimisticTxnContextProvider` (#38952) close pingcap/tidb#38951 --- session/txnmanager.go | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/session/txnmanager.go b/session/txnmanager.go index 65652428f7481..0d72c89461237 100644 --- a/session/txnmanager.go +++ b/session/txnmanager.go @@ -51,7 +51,7 @@ type txnManager struct { ctxProvider sessiontxn.TxnContextProvider // We always reuse the same OptimisticTxnContextProvider in one session to reduce memory allocation cost for every new txn. - reservedOptimisticProvider isolation.OptimisticTxnContextProvider + reservedOptimisticProviders [2]isolation.OptimisticTxnContextProvider } func newTxnManager(sctx sessionctx.Context) *txnManager { @@ -241,8 +241,13 @@ func (m *txnManager) newProviderWithRequest(r *sessiontxn.EnterNewTxnRequest) (s switch txnMode { case "", ast.Optimistic: // When txnMode is 'OPTIMISTIC' or '', the transaction should be optimistic - m.reservedOptimisticProvider.ResetForNewTxn(m.sctx, r.CausalConsistencyOnly) - return &m.reservedOptimisticProvider, nil + provider := &m.reservedOptimisticProviders[0] + if old, ok := m.ctxProvider.(*isolation.OptimisticTxnContextProvider); ok && old == provider { + // We should make sure the new provider is not the same with the old one + provider = &m.reservedOptimisticProviders[1] + } + provider.ResetForNewTxn(m.sctx, r.CausalConsistencyOnly) + return provider, nil case ast.Pessimistic: // When txnMode is 'PESSIMISTIC', the provider should be determined by the isolation level switch sessVars.IsolationLevelForNewTxn() { From 3ad6f932c0a2220954807616a300a50b08d69d2a Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 8 Nov 2022 10:21:50 +0800 Subject: [PATCH 05/22] planner: update some UTs from cost model1 to model2 (#38947) ref pingcap/tidb#35240 --- planner/core/integration_test.go | 10 +- planner/core/physical_plan_test.go | 1 + planner/core/rule_result_reorder_test.go | 1 + .../core/testdata/integration_suite_out.json | 730 +++++++++--------- .../ordered_result_mode_suite_out.json | 10 +- planner/core/testdata/plan_suite_out.json | 12 +- 6 files changed, 388 insertions(+), 376 deletions(-) diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index 44a9101613d1e..a89b85b736d62 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -858,6 +858,7 @@ func TestMPPShuffledJoin(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists d1_t") tk.MustExec("create table d1_t(d1_k int, value int)") tk.MustExec("insert into d1_t values(1,2),(2,3)") @@ -1275,6 +1276,7 @@ func TestReadFromStorageHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, tt, ttt") tk.MustExec("set session tidb_allow_mpp=OFF") tk.MustExec("create table t(a int, b int, index ia(a))") @@ -4267,6 +4269,7 @@ func TestPushDownProjectionForTiKV(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t (a int, b real, i int, id int, value decimal(6,3), name char(128), d decimal(6,3), s char(128), t datetime, c bigint as ((a+1)) virtual, e real as ((b+a)))") tk.MustExec("analyze table t") @@ -4550,6 +4553,7 @@ func TestPushDownAggForMPP(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3))") tk.MustExec("analyze table t") @@ -4631,6 +4635,7 @@ func TestMppJoinDecimal(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("drop table if exists tt") tk.MustExec("create table t (c1 decimal(8, 5), c2 decimal(9, 5), c3 decimal(9, 4) NOT NULL, c4 decimal(8, 4) NOT NULL, c5 decimal(40, 20))") @@ -4723,6 +4728,7 @@ func TestMppAggTopNWithJoin(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3))") tk.MustExec("analyze table t") @@ -5658,6 +5664,7 @@ func TestHeuristicIndexSelection(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int, c int, d int, e int, f int, g int, primary key (a), unique key c_d_e (c, d, e), unique key f (f), unique key f_g (f, g), key g (g))") tk.MustExec("create table t2(a int, b int, c int, d int, unique index idx_a (a), unique index idx_b_c (b, c), unique index idx_b_c_a_d (b, c, a, d))") @@ -7029,6 +7036,7 @@ func TestIssue36194(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") // create virtual tiflash replica. @@ -7043,7 +7051,7 @@ func TestIssue36194(t *testing.T) { } } } - tk.MustQuery("explain format = 'brief' select * from t where a + 1 > 20 limit 100;;").Check(testkit.Rows( + tk.MustQuery("explain format = 'brief' select /*+ read_from_storage(tiflash[t]) */ * from t where a + 1 > 20 limit 100;;").Check(testkit.Rows( "Limit 100.00 root offset:0, count:100", "└─TableReader 100.00 root data:ExchangeSender", " └─ExchangeSender 100.00 mpp[tiflash] ExchangeType: PassThrough", diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index b35f73c260535..c78ef7e8fd27d 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -1552,6 +1552,7 @@ func TestIndexMergeHint(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") var input []string var output []struct { diff --git a/planner/core/rule_result_reorder_test.go b/planner/core/rule_result_reorder_test.go index e6a248bee4071..060b44da245aa 100644 --- a/planner/core/rule_result_reorder_test.go +++ b/planner/core/rule_result_reorder_test.go @@ -106,6 +106,7 @@ func TestOrderedResultMode(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec(`set tidb_opt_limit_push_down_threshold=0`) tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t") diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index bc36df6a8f5e4..0a4b38ba0b05b 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1834,7 +1834,7 @@ { "SQL": "select * from t1 where a = 3 or a = 5", "Plan": [ - "Batch_Point_Get_5 2.00 12.53 root table:t1 handle:[3 5], keep order:false, desc:false" + "Batch_Point_Get_5 2.00 887.04 root table:t1 handle:[3 5], keep order:false, desc:false" ], "Warnings": [ "Note 1105 handle of t1 is selected since the path only has point ranges" @@ -1843,7 +1843,7 @@ { "SQL": "select f, g from t1 where f = 2 and g in (3, 4, 5)", "Plan": [ - "Batch_Point_Get_5 3.00 11.40 root table:t1, index:f_g(f, g) keep order:false, desc:false" + "Batch_Point_Get_5 3.00 380.16 root table:t1, index:f_g(f, g) keep order:false, desc:false" ], "Warnings": [ "Note 1105 unique index f_g of t1 is selected since the path only has point ranges with single scan" @@ -1852,7 +1852,7 @@ { "SQL": "select * from t1 where c = 1 and (d = 2 or d = 3) and e in (4, 5)", "Plan": [ - "Batch_Point_Get_5 4.00 27.20 root table:t1, index:c_d_e(c, d, e) keep order:false, desc:false" + "Batch_Point_Get_5 4.00 1774.08 root table:t1, index:c_d_e(c, d, e) keep order:false, desc:false" ], "Warnings": [ "Note 1105 unique index c_d_e of t1 is selected since the path only has point ranges with double scan" @@ -1861,8 +1861,8 @@ { "SQL": "select f, g from t1 where f = 2 and g > 3", "Plan": [ - "IndexReader_6 33.33 160.78 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 33.33 1850.00 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" + "IndexReader_6 33.33 752.77 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 33.33 7067.49 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 unique index f_g of t1 is selected since the path only fetches limited number of rows with single scan" @@ -1871,8 +1871,8 @@ { "SQL": "select a, b, c from t2 where a = 1 and b = 2 and c in (1, 2, 3, 4, 5)", "Plan": [ - "Selection_6 0.01 8.60 root eq(test.t2.b, 2), in(test.t2.c, 1, 2, 3, 4, 5)", - "└─Point_Get_5 1.00 5.60 root table:t2, index:idx_a(a) " + "Selection_6 0.01 289.88 root eq(test.t2.b, 2), in(test.t2.c, 1, 2, 3, 4, 5)", + "└─Point_Get_5 1.00 190.08 root table:t2, index:idx_a(a) " ], "Warnings": [ "Note 1105 unique index idx_a of t2 is selected since the path only has point ranges with double scan" @@ -1881,7 +1881,7 @@ { "SQL": "select * from t3 where (a = 1 or a = 3) and b = 'xx'", "Plan": [ - "Batch_Point_Get_5 2.00 7.60 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" + "Batch_Point_Get_5 2.00 1449.36 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" ], "Warnings": [ "Note 1105 handle of t3 is selected since the path only has point ranges" @@ -1890,7 +1890,7 @@ { "SQL": "select * from t4 where (a = 1 or a = 3) and b = 'xx'", "Plan": [ - "Batch_Point_Get_5 2.00 10.00 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" + "Batch_Point_Get_5 2.00 1449.36 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" ], "Warnings": [ "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with double scan" @@ -1899,7 +1899,7 @@ { "SQL": "select a, b from t3 where (a = 1 or a = 3) and b = 'xx'", "Plan": [ - "Batch_Point_Get_5 2.00 7.60 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" + "Batch_Point_Get_5 2.00 1322.64 root table:t3, clustered index:PRIMARY(a, b) keep order:false, desc:false" ], "Warnings": [ "Note 1105 handle of t3 is selected since the path only has point ranges" @@ -1908,7 +1908,7 @@ { "SQL": "select a, b from t4 where (a = 1 or a = 3) and b = 'xx'", "Plan": [ - "Batch_Point_Get_5 2.00 7.60 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" + "Batch_Point_Get_5 2.00 1322.64 root table:t4, index:PRIMARY(a, b) keep order:false, desc:false" ], "Warnings": [ "Note 1105 unique index PRIMARY of t4 is selected since the path only has point ranges with single scan" @@ -1918,7 +1918,7 @@ "SQL": "update t1 set b = 2 where a = 4 or a = 6", "Plan": [ "Update_4 N/A N/A root N/A", - "└─Batch_Point_Get_6 2.00 12.53 root table:t1 handle:[4 6], keep order:false, desc:false" + "└─Batch_Point_Get_6 2.00 887.04 root table:t1 handle:[4 6], keep order:false, desc:false" ], "Warnings": [ "Note 1105 handle of t1 is selected since the path only has point ranges" @@ -1928,8 +1928,8 @@ "SQL": "delete from t1 where f = 2 and g in (3, 4)", "Plan": [ "Delete_4 N/A N/A root N/A", - "└─Selection_7 2.00 9.80 root in(test.t1.g, 3, 4)", - " └─Point_Get_6 1.00 6.80 root table:t1, index:f(f) " + "└─Selection_7 2.00 493.42 root in(test.t1.g, 3, 4)", + " └─Point_Get_6 1.00 443.52 root table:t1, index:f(f) " ], "Warnings": [ "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" @@ -1939,8 +1939,8 @@ "SQL": "insert into t3 select a, b, c from t1 where f = 2", "Plan": [ "Insert_1 N/A N/A root N/A", - "└─Projection_6 1.00 25.40 root test.t1.a, test.t1.b, test.t1.c", - " └─Point_Get_7 1.00 6.80 root table:t1, index:f(f) " + "└─Projection_6 1.00 253.74 root test.t1.a, test.t1.b, test.t1.c", + " └─Point_Get_7 1.00 253.44 root table:t1, index:f(f) " ], "Warnings": [ "Note 1105 unique index f of t1 is selected since the path only has point ranges with double scan" @@ -1950,7 +1950,7 @@ "SQL": "replace into t3 select a, b, c from t1 where a = 3", "Plan": [ "Insert_1 N/A N/A root N/A", - "└─Point_Get_7 1.00 6.27 root table:t1 handle:3" + "└─Point_Get_7 1.00 190.08 root table:t1 handle:3" ], "Warnings": [ "Note 1105 handle of t1 is selected since the path only has point ranges" @@ -3605,310 +3605,294 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d2_k = d2_t.d2_k and fact_t.d3_k = d3_t.d3_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 128.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.d3_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d3_t.d3_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d3_t keep order:false", - " └─ExchangeReceiver(Probe) 64.00 mpp[tiflash] ", - " └─ExchangeSender 64.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d3_k, collate: binary]", - " └─HashJoin 64.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.d2_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d2_t.d2_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d2_t keep order:false", - " └─ExchangeReceiver(Probe) 32.00 mpp[tiflash] ", - " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d2_k, collate: binary]", - " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#17", + "└─TableReader 128.00 root data:ExchangeSender", + " └─ExchangeSender 128.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 128.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d3_k, test.d3_t.d3_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.d3_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d3_t.d3_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d3_t keep order:false", + " └─ExchangeReceiver(Probe) 64.00 mpp[tiflash] ", + " └─ExchangeSender 64.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d3_k, collate: binary]", + " └─HashJoin 64.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d2_k, test.d2_t.d2_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.d2_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d2_t.d2_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d2_t keep order:false", + " └─ExchangeReceiver(Probe) 32.00 mpp[tiflash] ", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d2_k, collate: binary]", + " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k)), not(isnull(test.fact_t.d2_k)), not(isnull(test.fact_t.d3_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t where fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t, d1_t, d2_t, d3_t where fact_t.d1_k = d1_t.d1_k and fact_t.d1_k = d2_t.value and fact_t.d1_k = d3_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#18)->Column#17", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#18", - " └─HashJoin 128.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d3_t.value)]", + "StreamAgg 1.00 root funcs:count(1)->Column#17", + "└─TableReader 128.00 root data:ExchangeSender", + " └─ExchangeSender 128.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 128.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d3_t.value)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.value, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d3_t.value))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d3_t keep order:false", + " └─HashJoin(Probe) 64.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d2_t.value)]", " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d3_t.value, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d3_t.value))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d3_t keep order:false", - " └─HashJoin(Probe) 64.00 mpp[tiflash] inner join, equal:[eq(test.fact_t.d1_k, test.d2_t.value)]", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.value, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d2_t.value))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d2_t keep order:false", + " └─HashJoin(Probe) 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d2_t.value, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d2_t.value))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d2_t keep order:false", - " └─HashJoin(Probe) 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > d1_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 10", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col1, 10)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from (select case when t1.col1 is null then t2.col1 + 5 else 10 end as col1, t2.d1_k as d1_k from fact_t t1 right join fact_t t2 on t1.d1_k = t2.d1_k) fact_t join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col1 > 5", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#22)->Column#19", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#22", - " └─HashJoin 204.80 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─Projection(Probe) 102.40 mpp[tiflash] test.fact_t.d1_k", - " └─Selection 102.40 mpp[tiflash] gt(case(isnull(test.fact_t.col1), plus(test.fact_t.col1, 5), 10), 5)", - " └─HashJoin 128.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.fact_t.d1_k)]", - " ├─ExchangeReceiver(Build) 16.00 mpp[tiflash] ", - " │ └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " │ └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 mpp[tiflash] table:t1 keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:t2 keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#19", + "└─TableReader 204.80 root data:ExchangeSender", + " └─ExchangeSender 204.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 204.80 mpp[tiflash] inner join, equal:[eq(test.d1_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─Projection(Probe) 102.40 mpp[tiflash] test.fact_t.d1_k", + " └─Selection 102.40 mpp[tiflash] gt(case(isnull(test.fact_t.col1), plus(test.fact_t.col1, 5), 10), 5)", + " └─HashJoin 128.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.fact_t.d1_k)]", + " ├─ExchangeReceiver(Build) 16.00 mpp[tiflash] ", + " │ └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " │ └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 16.00 mpp[tiflash] table:t1 keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k = d1_t.d1_k and fact_t.col2 > 10 and fact_t.col1 > d1_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] left outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], left cond:[gt(test.fact_t.col2, 10)], other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10)", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k = d1_t.d1_k and d1_t.value > 10 and fact_t.col1 > d1_t.value", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 32.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", - " ├─ExchangeReceiver(Build) 16.00 mpp[tiflash] ", - " │ └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " │ └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false", - " └─ExchangeReceiver(Probe) 4.00 mpp[tiflash] ", - " └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 32.00 root data:ExchangeSender", + " └─ExchangeSender 32.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 32.00 mpp[tiflash] right outer join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], right cond:gt(test.d1_t.value, 10), other cond:gt(test.fact_t.col1, test.d1_t.value)", + " ├─ExchangeReceiver(Build) 16.00 mpp[tiflash] ", + " │ └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " │ └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false", + " └─ExchangeReceiver(Probe) 4.00 mpp[tiflash] ", + " └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 12.80 root data:ExchangeSender", + " └─ExchangeSender 12.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12.80 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 12.80 root data:ExchangeSender", + " └─ExchangeSender 12.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12.80 mpp[tiflash] semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─Selection 4.00 mpp[tiflash] not(isnull(test.d1_t.d1_k)), not(isnull(test.d1_t.value))", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─Selection 16.00 mpp[tiflash] not(isnull(test.fact_t.col1)), not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 12.80 root data:ExchangeSender", + " └─ExchangeSender 12.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12.80 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)]", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t where not exists (select 1 from d1_t where d1_k = fact_t.d1_k and value > fact_t.col1)", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#13)->Column#12", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#13", - " └─HashJoin 12.80 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", - " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", - " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", - " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", - " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", - " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", - " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#12", + "└─TableReader 12.80 root data:ExchangeSender", + " └─ExchangeSender 12.80 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 12.80 mpp[tiflash] anti semi join, equal:[eq(test.fact_t.d1_k, test.d1_t.d1_k)], other cond:gt(test.d1_t.value, test.fact_t.col1)", + " ├─ExchangeReceiver(Build) 4.00 mpp[tiflash] ", + " │ └─ExchangeSender 4.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.d1_t.d1_k, collate: binary]", + " │ └─TableFullScan 4.00 mpp[tiflash] table:d1_t keep order:false", + " └─ExchangeReceiver(Probe) 16.00 mpp[tiflash] ", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.fact_t.d1_k, collate: binary]", + " └─TableFullScan 16.00 mpp[tiflash] table:fact_t keep order:false" ] } ] @@ -4187,11 +4171,10 @@ { "SQL": "desc format = 'brief' select avg(a) from t", "Plan": [ - "StreamAgg 1.00 root funcs:avg(Column#7, Column#8)->Column#4", - "└─TableReader 1.00 root data:StreamAgg", - " └─StreamAgg 1.00 batchCop[tiflash] funcs:count(Column#9)->Column#7, funcs:sum(Column#10)->Column#8", - " └─Projection 10000.00 batchCop[tiflash] test.t.a, cast(test.t.a, decimal(10,0) BINARY)->Column#10", - " └─TableFullScan 10000.00 batchCop[tiflash] table:t keep order:false, stats:pseudo" + "HashAgg 1.00 root funcs:avg(Column#5, Column#6)->Column#4", + "└─IndexReader 1.00 root index:HashAgg", + " └─HashAgg 1.00 cop[tikv] funcs:count(test.t.a)->Column#5, funcs:sum(test.t.a)->Column#6", + " └─IndexFullScan 10000.00 cop[tikv] table:t, index:ia(a) keep order:false, stats:pseudo" ], "Warn": null }, @@ -4269,7 +4252,7 @@ "SQL": "desc format = 'brief' select * from tt where (tt.a > 1 and tt.a < 20) or (tt.a >= 30 and tt.a < 55)", "Plan": [ "TableReader 44.00 root data:TableRangeScan", - "└─TableRangeScan 44.00 cop[tiflash] table:tt range:(1,20), [30,55), keep order:false, stats:pseudo" + "└─TableRangeScan 44.00 cop[tikv] table:tt range:(1,20), [30,55), keep order:false, stats:pseudo" ], "Warn": null }, @@ -4675,11 +4658,11 @@ "SQL": "desc format = 'brief' select * from t right join (select id-2 as b from t) A on A.b=t.id", "Plan": [ "HashJoin 12487.50 root right outer join, equal:[eq(test.t.id, Column#25)]", - "├─TableReader(Build) 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.id))", + "├─TableReader(Build) 10000.00 root data:Projection", + "│ └─Projection 10000.00 cop[tikv] minus(test.t.id, 2)->Column#25", "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", - "└─TableReader(Probe) 10000.00 root data:Projection", - " └─Projection 10000.00 cop[tikv] minus(test.t.id, 2)->Column#25", + "└─TableReader(Probe) 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.id))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ] }, @@ -5430,20 +5413,22 @@ " │ └─ExchangeSender 7976.02 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: Column#58, collate: binary], [name: test.t.c5, collate: binary]", " │ └─Projection 7976.02 mpp[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5, cast(test.t.c2, decimal(10,5))->Column#58", " │ └─Projection 7976.02 mpp[tiflash] Column#7, test.t.c1, test.t.c2, test.t.c5", - " │ └─HashAgg 7976.02 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:count(1)->Column#7, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c5)->test.t.c5", - " │ └─ExchangeReceiver 9970.03 mpp[tiflash] ", - " │ └─ExchangeSender 9970.03 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c5, collate: binary]", - " │ └─Selection 9970.03 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7976.02 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:sum(Column#15)->Column#7, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c5)->test.t.c5", + " │ └─ExchangeReceiver 7976.02 mpp[tiflash] ", + " │ └─ExchangeSender 7976.02 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c1, collate: binary], [name: test.t.c2, collate: binary], [name: test.t.c5, collate: binary]", + " │ └─HashAgg 7976.02 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c5, funcs:count(1)->Column#15", + " │ └─Selection 9970.03 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2)), not(isnull(test.t.c5))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 7984.01 mpp[tiflash] ", " └─ExchangeSender 7984.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: Column#59, collate: binary], [name: Column#60, collate: binary]", " └─Projection 7984.01 mpp[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3, cast(test.t.c3, decimal(10,5))->Column#59, cast(test.t.c1, decimal(40,20))->Column#60", " └─Projection 7984.01 mpp[tiflash] Column#14, test.t.c1, test.t.c2, test.t.c3", - " └─HashAgg 7984.01 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:count(1)->Column#14, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c3)->test.t.c3", - " └─ExchangeReceiver 9980.01 mpp[tiflash] ", - " └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c1, collate: binary]", - " └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 7984.01 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:sum(Column#23)->Column#14, funcs:firstrow(test.t.c1)->test.t.c1, funcs:firstrow(test.t.c2)->test.t.c2, funcs:firstrow(test.t.c3)->test.t.c3", + " └─ExchangeReceiver 7984.01 mpp[tiflash] ", + " └─ExchangeSender 7984.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.c2, collate: binary], [name: test.t.c3, collate: binary], [name: test.t.c1, collate: binary]", + " └─HashAgg 7984.01 mpp[tiflash] group by:test.t.c1, test.t.c2, test.t.c3, funcs:count(1)->Column#23", + " └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.c1)), not(isnull(test.t.c2))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5649,10 +5634,11 @@ "TableReader 8000.00 root data:ExchangeSender", "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#4, test.t.id", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#11", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5677,11 +5663,12 @@ "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 mpp[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#8", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", @@ -5707,11 +5694,12 @@ "Plan": [ "TableReader 8000.00 root data:ExchangeSender", "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#16, 0), 1, Column#16), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#16, funcs:sum(test.t.value)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#18)->Column#17, funcs:sum(Column#19)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#18, funcs:sum(test.t.value)->Column#19", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5721,11 +5709,12 @@ "└─TableReader 1.00 root data:ExchangeSender", " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#4)->Column#18", - " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#14, 0), 1, Column#14), decimal(20,0) BINARY))->Column#4", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#14, funcs:sum(test.t.value)->Column#4", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#15, 0), 1, Column#15), decimal(20,0) BINARY))->Column#4", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#16)->Column#15, funcs:sum(Column#17)->Column#4", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#16, funcs:sum(test.t.value)->Column#17", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5735,11 +5724,12 @@ "└─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 6400.00 mpp[tiflash] test.t.id", " └─Selection 6400.00 mpp[tiflash] gt(Column#4, 0)", - " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#17, 0), 1, Column#17), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#17, funcs:sum(test.t.value)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#18, 0), 1, Column#18), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#19)->Column#18, funcs:sum(Column#20)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#19, funcs:sum(test.t.value)->Column#20", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5748,11 +5738,12 @@ "TableReader 6400.00 root data:ExchangeSender", "└─ExchangeSender 6400.00 mpp[tiflash] ExchangeType: PassThrough", " └─Selection 6400.00 mpp[tiflash] gt(Column#4, 0)", - " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#18, 0), 1, Column#18), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#18, funcs:sum(test.t.value)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#20)->Column#19, funcs:sum(Column#21)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#20, funcs:sum(test.t.value)->Column#21", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5761,11 +5752,12 @@ "TableReader 8000.00 root data:ExchangeSender", "└─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] plus(Column#4, 1)->Column#5, test.t.id", - " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#18, 0), 1, Column#18), decimal(20,0) BINARY))->Column#4, test.t.id", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#18, funcs:sum(test.t.value)->Column#4, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─Projection 8000.00 mpp[tiflash] div(Column#4, cast(case(eq(Column#19, 0), 1, Column#19), decimal(20,0) BINARY))->Column#4, test.t.id", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#20)->Column#19, funcs:sum(Column#21)->Column#4, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(test.t.value)->Column#20, funcs:sum(test.t.value)->Column#21", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5795,20 +5787,22 @@ " └─HashJoin 7992.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 mpp[tiflash] Column#11, test.t.id", " └─HashAgg 7992.00 mpp[tiflash] group by:Column#39, funcs:sum(Column#37)->Column#11, funcs:firstrow(Column#38)->test.t.id", " └─Projection 9990.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#37, test.t.id, test.t.id", " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 mpp[tiflash] test.t.id, Column#13", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:count(1)->Column#13", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#17)->Column#13", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#17", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", @@ -5862,12 +5856,13 @@ "└─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: PassThrough", " └─HashJoin 9990.00 mpp[tiflash] inner join, equal:[eq(test.t.id, test.t.id)]", " ├─Projection(Build) 7992.00 mpp[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#20, funcs:count(Column#18)->Column#7, funcs:firstrow(Column#19)->test.t.id", - " │ └─Projection 9990.00 mpp[tiflash] div(1, test.t.value)->Column#18, test.t.id, test.t.id", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:Column#19, funcs:count(Column#18)->Column#8", + " │ └─Projection 9990.00 mpp[tiflash] div(1, test.t.value)->Column#18, test.t.id", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 9990.00 mpp[tiflash] ", " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", @@ -5901,11 +5896,12 @@ " └─Selection 8000.00 mpp[tiflash] gt(plus(test.t.id, 1), ifnull(Column#7, 0))", " └─HashJoin 10000.00 mpp[tiflash] left outer join, equal:[eq(test.t.id, test.t.id) eq(test.t.value, test.t.value)]", " ├─Projection(Build) 7984.01 mpp[tiflash] Column#7, test.t.id, test.t.value", - " │ └─HashAgg 7984.01 mpp[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " │ └─ExchangeReceiver 9980.01 mpp[tiflash] ", - " │ └─ExchangeSender 9980.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.id)), not(isnull(test.t.value))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7984.01 mpp[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#24)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " │ └─ExchangeReceiver 7984.01 mpp[tiflash] ", + " │ └─ExchangeSender 7984.01 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7984.01 mpp[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#24", + " │ └─Selection 9980.01 mpp[tiflash] not(isnull(test.t.id)), not(isnull(test.t.value))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─ExchangeReceiver(Probe) 10000.00 mpp[tiflash] ", " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", " └─TableFullScan 10000.00 mpp[tiflash] table:B keep order:false, stats:pseudo" @@ -5930,19 +5926,18 @@ { "SQL": "desc format = 'brief' select count(distinct x ) from (select count(distinct value) x from t) t", "Plan": [ - "TableReader 1.00 root data:ExchangeSender", - "└─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection 1.00 mpp[tiflash] Column#5", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct Column#4)->Column#5", - " └─Projection 1.00 mpp[tiflash] Column#4", - " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#7)->Column#4", - " └─ExchangeReceiver 1.00 mpp[tiflash] ", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.t.value)->Column#7", - " └─ExchangeReceiver 1.00 mpp[tiflash] ", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary]", - " └─HashAgg 1.00 mpp[tiflash] group by:test.t.value, ", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg 1.00 root funcs:count(distinct Column#4)->Column#5", + "└─TableReader 1.00 root data:ExchangeSender", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection 1.00 mpp[tiflash] Column#4", + " └─HashAgg 1.00 mpp[tiflash] funcs:sum(Column#6)->Column#4", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg 1.00 mpp[tiflash] funcs:count(distinct test.t.value)->Column#6", + " └─ExchangeReceiver 1.00 mpp[tiflash] ", + " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary]", + " └─HashAgg 1.00 mpp[tiflash] group by:test.t.value, ", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -5975,11 +5970,12 @@ " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] Column#7, test.t.id", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#8)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#8", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] @@ -5995,10 +5991,11 @@ " │ └─Projection 6400.00 mpp[tiflash] plus(Column#7, test.t.id)->Column#8", " │ └─Selection 6400.00 mpp[tiflash] not(isnull(plus(Column#7, test.t.id)))", " │ └─Projection 8000.00 mpp[tiflash] Column#7, test.t.id", - " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#11)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#11", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] @@ -6014,11 +6011,12 @@ " │ └─ExchangeSender 6393.60 mpp[tiflash] ExchangeType: Broadcast", " │ └─Selection 6393.60 mpp[tiflash] lt(plus(test.t.value, cast(Column#7, decimal(20,0) BINARY)), 10)", " │ └─Projection 7992.00 mpp[tiflash] Column#7, test.t.id, test.t.value", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary], [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, test.t.value, funcs:sum(Column#10)->Column#7, funcs:firstrow(test.t.id)->test.t.id, funcs:firstrow(test.t.value)->test.t.value", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.value, collate: binary], [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, test.t.value, funcs:count(1)->Column#10", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] @@ -6066,16 +6064,18 @@ " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 mpp[tiflash] Column#7, test.t.id", - " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#7, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(Column#9)->Column#7, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:sum(test.t.value)->Column#9", + " └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -6088,10 +6088,11 @@ " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Projection(Probe) 7992.00 mpp[tiflash] Column#11, test.t.id", " └─HashAgg 7992.00 mpp[tiflash] group by:Column#34, funcs:sum(Column#32)->Column#11, funcs:firstrow(Column#33)->test.t.id", " └─Projection 9990.00 mpp[tiflash] cast(test.t.id, decimal(10,0) BINARY)->Column#32, test.t.id, test.t.id", @@ -6101,11 +6102,12 @@ " ├─ExchangeReceiver(Build) 7992.00 mpp[tiflash] ", " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] test.t.id, Column#13", - " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:count(1)->Column#13", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id, funcs:sum(Column#16)->Column#13", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:count(1)->Column#16", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] @@ -6149,7 +6151,7 @@ { "SQL": "desc format = 'brief' select count(*) from (select t.id, t.value v1 from t join t t1 on t.id = t1.id order by t.value limit 20) v group by v.v1", "Plan": [ - "HashAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", + "StreamAgg 20.00 root group by:test.t.value, funcs:count(1)->Column#7", "└─TopN 20.00 root test.t.value, offset:0, count:20", " └─TableReader 20.00 root data:ExchangeSender", " └─ExchangeSender 20.00 mpp[tiflash] ExchangeType: PassThrough", diff --git a/planner/core/testdata/ordered_result_mode_suite_out.json b/planner/core/testdata/ordered_result_mode_suite_out.json index 1d724eae8eca5..f6acccadce758 100644 --- a/planner/core/testdata/ordered_result_mode_suite_out.json +++ b/planner/core/testdata/ordered_result_mode_suite_out.json @@ -86,11 +86,11 @@ { "Plan": [ "Sort_9 12500.00 root test.t.a, test.t.a", - "└─HashJoin_30 12500.00 root inner join, equal:[eq(test.t.a, test.t.a)]", - " ├─IndexReader_43(Build) 10000.00 root index:IndexFullScan_42", - " │ └─IndexFullScan_42 10000.00 cop[tikv] table:t2, index:b(b) keep order:false, stats:pseudo", - " └─IndexReader_39(Probe) 10000.00 root index:IndexFullScan_38", - " └─IndexFullScan_38 10000.00 cop[tikv] table:t1, index:b(b) keep order:false, stats:pseudo" + "└─MergeJoin_11 12500.00 root inner join, left key:test.t.a, right key:test.t.a", + " ├─TableReader_35(Build) 10000.00 root data:TableFullScan_34", + " │ └─TableFullScan_34 10000.00 cop[tikv] table:t2 keep order:true, stats:pseudo", + " └─TableReader_33(Probe) 10000.00 root data:TableFullScan_32", + " └─TableFullScan_32 10000.00 cop[tikv] table:t1 keep order:true, stats:pseudo" ] }, { diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 50315a9784697..572f81f7b0b50 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -350,9 +350,9 @@ }, { "SQL": "select /*+ USE_INDEX_MERGE(t1, c_d_e, f_g) */ * from t where c < 1 or f > 2", - "Best": "IndexMergeReader(PartialPlans->[Index(t.c_d_e)[[-inf,1)], Index(t.f)[(2,+inf]]], TablePlan->Table(t))", + "Best": "TableReader(Table(t)->Sel([or(lt(test.t.c, 1), gt(test.t.f, 2))]))", "HasWarn": true, - "Hints": "use_index_merge(@`sel_1` `t` `c_d_e`, `f`)" + "Hints": "use_index(@`sel_1` `test`.`t` )" }, { "SQL": "select /*+ NO_INDEX_MERGE(), USE_INDEX_MERGE(t, primary, f_g, c_d_e) */ * from t where a < 1 or f > 2", @@ -368,15 +368,15 @@ }, { "SQL": "select /*+ USE_INDEX_MERGE(db2.t) */ * from t where c < 1 or f > 2", - "Best": "IndexMergeReader(PartialPlans->[Index(t.c_d_e)[[-inf,1)], Index(t.f)[(2,+inf]]], TablePlan->Table(t))", + "Best": "TableReader(Table(t)->Sel([or(lt(test.t.c, 1), gt(test.t.f, 2))]))", "HasWarn": true, - "Hints": "use_index_merge(@`sel_1` `t` `c_d_e`, `f`)" + "Hints": "use_index(@`sel_1` `test`.`t` )" }, { "SQL": "select /*+ USE_INDEX_MERGE(db2.t, c_d_e, f_g) */ * from t where c < 1 or f > 2", - "Best": "IndexMergeReader(PartialPlans->[Index(t.c_d_e)[[-inf,1)], Index(t.f)[(2,+inf]]], TablePlan->Table(t))", + "Best": "TableReader(Table(t)->Sel([or(lt(test.t.c, 1), gt(test.t.f, 2))]))", "HasWarn": true, - "Hints": "use_index_merge(@`sel_1` `t` `c_d_e`, `f`)" + "Hints": "use_index(@`sel_1` `test`.`t` )" } ] }, From 97b01517732cb47d6ff9f9f1fe6744328aad9ca4 Mon Sep 17 00:00:00 2001 From: Jack Yu Date: Tue, 8 Nov 2022 10:33:50 +0800 Subject: [PATCH 06/22] metrics: add metrics for multi-statement (#38931) close pingcap/tidb#38930 --- metrics/grafana/tidb.json | 111 ++++++++++++++++++++++++++++++++++++++ metrics/metrics.go | 1 + metrics/server.go | 9 ++++ server/conn.go | 1 + 4 files changed, 122 insertions(+) diff --git a/metrics/grafana/tidb.json b/metrics/grafana/tidb.json index 757953d97b966..bd686ffd05c92 100644 --- a/metrics/grafana/tidb.json +++ b/metrics/grafana/tidb.json @@ -1979,6 +1979,117 @@ "align": false, "alignLevel": null } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of queries contained in a multi-query statement per second.", + "editable": true, + "error": false, + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 23 + }, + "hiddenSeries": false, + "id": 272, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": false, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tidb_server_multi_query_num_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s]))/sum(rate(tidb_server_multi_query_num_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s]))", + "format": "time_series", + "interval": "", + "intervalFactor": 2, + "legendFormat": "avg", + "refId": "A", + "step": 10 + }, + { + "exemplar": true, + "expr": "sum(rate(tidb_server_multi_query_num_sum{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[30s]))", + "hide": false, + "interval": "", + "legendFormat": "sum", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Queries in Multi-Statement", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } } ], "repeat": null, diff --git a/metrics/metrics.go b/metrics/metrics.go index 706da97f356b7..a843e794cff1b 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -114,6 +114,7 @@ func RegisterMetrics() { prometheus.MustRegister(ExecuteErrorCounter) prometheus.MustRegister(ExecutorCounter) prometheus.MustRegister(GetTokenDurationHistogram) + prometheus.MustRegister(NumOfMultiQueryHistogram) prometheus.MustRegister(HandShakeErrorCounter) prometheus.MustRegister(HandleJobHistogram) prometheus.MustRegister(SignificantFeedbackCounter) diff --git a/metrics/server.go b/metrics/server.go index 4f0d147a4b27e..116b02eb122b6 100644 --- a/metrics/server.go +++ b/metrics/server.go @@ -187,6 +187,15 @@ var ( Buckets: prometheus.ExponentialBuckets(1, 2, 30), // 1us ~ 528s }) + NumOfMultiQueryHistogram = prometheus.NewHistogram( + prometheus.HistogramOpts{ + Namespace: "tidb", + Subsystem: "server", + Name: "multi_query_num", + Help: "The number of queries contained in a multi-query statement.", + Buckets: prometheus.ExponentialBuckets(1, 2, 20), // 1 ~ 1048576 + }) + TotalQueryProcHistogram = prometheus.NewHistogramVec( prometheus.HistogramOpts{ Namespace: "tidb", diff --git a/server/conn.go b/server/conn.go index 57da6c02f9e0a..6a466fbffae5f 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1912,6 +1912,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { } return err } + metrics.NumOfMultiQueryHistogram.Observe(float64(len(stmts))) } if len(pointPlans) > 0 { defer cc.ctx.ClearValue(plannercore.PointPlanKey) From 2643550d3bc12c9a048b79f25e8d4e442b9c4a4d Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 8 Nov 2022 10:51:50 +0800 Subject: [PATCH 07/22] *: upgrade golang 1.19.3 (#38805) ref pingcap/tidb#36934, close pingcap/tidb#37022 --- WORKSPACE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/WORKSPACE b/WORKSPACE index 35ae55b7388a3..e6df7760f5d5e 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -29,7 +29,7 @@ go_rules_dependencies() go_register_toolchains( nogo = "@//build:tidb_nogo", - version = "1.19.2", + version = "1.19.3", ) gazelle_dependencies() From 6eb9379d431240f4c837e6cb4a90a1e5e69342a8 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Tue, 8 Nov 2022 11:25:50 +0800 Subject: [PATCH 08/22] planner: sync load shouldn't load items which isn't intiailized before (#38949) close pingcap/tidb#38919 --- planner/core/plan_stats.go | 2 +- statistics/column.go | 5 ----- statistics/handle/handle_hist.go | 15 +++++++++++---- statistics/handle/handle_hist_test.go | 22 ++++++++++++++++++++++ 4 files changed, 34 insertions(+), 10 deletions(-) diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 681b5c1e8a78e..9575c3e8d79b3 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -154,7 +154,7 @@ func collectSyncIndices(ctx sessionctx.Context, histNeededColumns []model.TableI continue } idxStats, ok := tblStats.Indices[idx.Meta().ID] - if !ok || idxStats == nil || !idxStats.IsFullLoad() { + if ok && idxStats.IsStatsInitialized() && !idxStats.IsFullLoad() { histNeededIndices[model.TableItemID{TableID: column.TableID, ID: idxID, IsIndex: true}] = struct{}{} } } diff --git a/statistics/column.go b/statistics/column.go index 197fc2f5a6794..ccd541f276bcf 100644 --- a/statistics/column.go +++ b/statistics/column.go @@ -141,11 +141,6 @@ func (c *Column) IsInvalid(sctx sessionctx.Context, collPseudo bool) bool { return c.TotalRowCount() == 0 || (!c.IsEssentialStatsLoaded() && c.Histogram.NDV > 0) } -// IsHistNeeded checks if this column needs histogram to be loaded -func (c *Column) IsHistNeeded(collPseudo bool) bool { - return (!collPseudo || !c.NotAccurate()) && c.IsLoadNeeded() -} - func (c *Column) equalRowCount(sctx sessionctx.Context, val types.Datum, encodedVal []byte, realtimeRowCount int64) (float64, error) { if val.IsNull() { return float64(c.NullCount), nil diff --git a/statistics/handle/handle_hist.go b/statistics/handle/handle_hist.go index 1392590130438..4d0128095e071 100644 --- a/statistics/handle/handle_hist.go +++ b/statistics/handle/handle_hist.go @@ -59,6 +59,16 @@ type NeededItemTask struct { // SendLoadRequests send neededColumns requests func (h *Handle) SendLoadRequests(sc *stmtctx.StatementContext, neededHistItems []model.TableItemID, timeout time.Duration) error { remainedItems := h.removeHistLoadedColumns(neededHistItems) + + failpoint.Inject("assertSyncLoadItems", func(val failpoint.Value) { + if sc.OptimizeTracer != nil { + count := val.(int) + if len(remainedItems) != count { + panic("remained items count wrong") + } + } + }) + if len(remainedItems) <= 0 { return nil } @@ -137,10 +147,7 @@ func (h *Handle) removeHistLoadedColumns(neededItems []model.TableItemID) []mode continue } colHist, ok := tbl.Columns[item.ID] - if !ok { - continue - } - if colHist.IsHistNeeded(tbl.Pseudo) { + if ok && colHist.IsStatsInitialized() && !colHist.IsFullLoad() { remainedItems = append(remainedItems, item) } } diff --git a/statistics/handle/handle_hist_test.go b/statistics/handle/handle_hist_test.go index cf9f708345d32..f2b1fa87ba37f 100644 --- a/statistics/handle/handle_hist_test.go +++ b/statistics/handle/handle_hist_test.go @@ -29,6 +29,28 @@ import ( "github.com/stretchr/testify/require" ) +func TestSyncLoadSkipUnAnalyzedItems(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t(a int)") + tk.MustExec("create table t1(a int)") + h := dom.StatsHandle() + h.SetLease(1) + + // no item would be loaded + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/handle/assertSyncLoadItems", `return(0)`)) + tk.MustQuery("trace plan select * from t where a > 10") + failpoint.Disable("github.com/pingcap/tidb/statistics/handle/assertSyncLoadItems") + tk.MustExec("analyze table t1") + // one column would be loaded + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/statistics/handle/assertSyncLoadItems", `return(1)`)) + tk.MustQuery("trace plan select * from t1 where a > 10") + failpoint.Disable("github.com/pingcap/tidb/statistics/handle/assertSyncLoadItems") +} + func TestConcurrentLoadHist(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) From 473a03c7d81df5e3ab499d294c5370e333785dfe Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Tue, 8 Nov 2022 15:27:50 +0800 Subject: [PATCH 09/22] gcworker: update gc time format (#38942) close pingcap/tidb#37497 --- DEPS.bzl | 4 ++-- ddl/BUILD.bazel | 1 + ddl/attributes_sql_test.go | 6 +++--- executor/recover_test.go | 6 +++--- go.mod | 2 +- go.sum | 4 ++-- store/gcworker/gc_worker_test.go | 5 +++++ tests/realtikvtest/brietest/BUILD.bazel | 1 + tests/realtikvtest/brietest/flashback_test.go | 6 +++--- 9 files changed, 21 insertions(+), 14 deletions(-) diff --git a/DEPS.bzl b/DEPS.bzl index 2a9004c21455b..f05385986ad79 100644 --- a/DEPS.bzl +++ b/DEPS.bzl @@ -3495,8 +3495,8 @@ def go_deps(): name = "com_github_tikv_client_go_v2", build_file_proto_mode = "disable_global", importpath = "github.com/tikv/client-go/v2", - sum = "h1:x5L32xr5KE2C0UhlKfrsDTWQ/8fpFYQLq5xn+ph0sQ0=", - version = "v2.0.3-0.20221103084528-ec1202a9d26f", + sum = "h1:nFVdyTXcQYZwQQCdSJcFI1vBFyzG1hVuZ39MAK6wqK4=", + version = "v2.0.3-0.20221108030801-9c0835c80eba", ) go_repository( name = "com_github_tikv_pd_client", diff --git a/ddl/BUILD.bazel b/ddl/BUILD.bazel index 1e6357664afeb..6d44cd6265bfa 100644 --- a/ddl/BUILD.bazel +++ b/ddl/BUILD.bazel @@ -256,6 +256,7 @@ go_test( "@com_github_tikv_client_go_v2//oracle", "@com_github_tikv_client_go_v2//testutils", "@com_github_tikv_client_go_v2//tikv", + "@com_github_tikv_client_go_v2//util", "@io_etcd_go_etcd_client_v3//:client", "@org_golang_google_grpc//:grpc", "@org_golang_x_exp//slices", diff --git a/ddl/attributes_sql_test.go b/ddl/attributes_sql_test.go index de00a63a7a661..95f881e6fb3fe 100644 --- a/ddl/attributes_sql_test.go +++ b/ddl/attributes_sql_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/util/gcutil" "github.com/stretchr/testify/require" + tikvutil "github.com/tikv/client-go/v2/util" ) // MockGC is used to make GC work in the test environment. @@ -44,9 +45,8 @@ func MockGC(tk *testkit.TestKit) (string, string, string, func()) { // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. util.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(tikvutil.GCTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(tikvutil.GCTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` diff --git a/executor/recover_test.go b/executor/recover_test.go index 26ed1a765f58f..aad1c93d9fb87 100644 --- a/executor/recover_test.go +++ b/executor/recover_test.go @@ -31,6 +31,7 @@ import ( "github.com/pingcap/tidb/util/gcutil" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" + tikvutil "github.com/tikv/client-go/v2/util" ) func TestRecoverTable(t *testing.T) { @@ -472,9 +473,8 @@ func MockGC(tk *testkit.TestKit) (string, string, string, func()) { // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddlutil.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(tikvutil.GCTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(tikvutil.GCTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` diff --git a/go.mod b/go.mod index 200010fa4ac82..952503212d3ee 100644 --- a/go.mod +++ b/go.mod @@ -86,7 +86,7 @@ require ( github.com/stretchr/testify v1.8.0 github.com/tdakkota/asciicheck v0.1.1 github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 - github.com/tikv/client-go/v2 v2.0.3-0.20221103084528-ec1202a9d26f + github.com/tikv/client-go/v2 v2.0.3-0.20221108030801-9c0835c80eba github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 github.com/twmb/murmur3 v1.1.3 diff --git a/go.sum b/go.sum index deccee72b3f3d..eb7dd447f9d5e 100644 --- a/go.sum +++ b/go.sum @@ -926,8 +926,8 @@ github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3 h1:f+jULpR github.com/tenntenn/text/transform v0.0.0-20200319021203-7eef512accb3/go.mod h1:ON8b8w4BN/kE1EOhwT0o+d62W65a6aPw1nouo9LMgyY= 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/tikv/client-go/v2 v2.0.3-0.20221103084528-ec1202a9d26f h1:x5L32xr5KE2C0UhlKfrsDTWQ/8fpFYQLq5xn+ph0sQ0= -github.com/tikv/client-go/v2 v2.0.3-0.20221103084528-ec1202a9d26f/go.mod h1:X9s4ct/MLk1sFqe5mU79KClKegLFDTa/FCx3hzexGtk= +github.com/tikv/client-go/v2 v2.0.3-0.20221108030801-9c0835c80eba h1:nFVdyTXcQYZwQQCdSJcFI1vBFyzG1hVuZ39MAK6wqK4= +github.com/tikv/client-go/v2 v2.0.3-0.20221108030801-9c0835c80eba/go.mod h1:X9s4ct/MLk1sFqe5mU79KClKegLFDTa/FCx3hzexGtk= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07 h1:ckPpxKcl75mO2N6a4cJXiZH43hvcHPpqc9dh1TmH1nc= github.com/tikv/pd/client v0.0.0-20221031025758-80f0d8ca4d07/go.mod h1:CipBxPfxPUME+BImx9MUYXCnAVLS3VJUr3mnSJwh40A= github.com/timakin/bodyclose v0.0.0-20210704033933-f49887972144 h1:kl4KhGNsJIbDHS9/4U9yQo1UcPQM0kOMJHn29EoH/Ro= diff --git a/store/gcworker/gc_worker_test.go b/store/gcworker/gc_worker_test.go index 8d9ae4c9ae377..4fde3dd44cd12 100644 --- a/store/gcworker/gc_worker_test.go +++ b/store/gcworker/gc_worker_test.go @@ -1117,6 +1117,11 @@ func TestRunGCJob(t *testing.T) { pdSafePoint := s.mustGetSafePointFromPd(t) require.Equal(t, safePoint, pdSafePoint) + require.NoError(t, s.gcWorker.saveTime(gcSafePointKey, oracle.GetTimeFromTS(safePoint))) + tikvSafePoint, err := s.gcWorker.loadTime(gcSafePointKey) + require.NoError(t, err) + require.Equal(t, *tikvSafePoint, oracle.GetTimeFromTS(safePoint)) + etcdSafePoint := s.loadEtcdSafePoint(t) require.Equal(t, safePoint, etcdSafePoint) diff --git a/tests/realtikvtest/brietest/BUILD.bazel b/tests/realtikvtest/brietest/BUILD.bazel index fc9ff667cfbf6..e4520a855a941 100644 --- a/tests/realtikvtest/brietest/BUILD.bazel +++ b/tests/realtikvtest/brietest/BUILD.bazel @@ -24,6 +24,7 @@ go_test( "@com_github_pingcap_tipb//go-binlog", "@com_github_stretchr_testify//require", "@com_github_tikv_client_go_v2//oracle", + "@com_github_tikv_client_go_v2//util", "@org_golang_google_grpc//:grpc", "@org_uber_go_goleak//:goleak", ], diff --git a/tests/realtikvtest/brietest/flashback_test.go b/tests/realtikvtest/brietest/flashback_test.go index 9c15845a108ad..322359fff411a 100644 --- a/tests/realtikvtest/brietest/flashback_test.go +++ b/tests/realtikvtest/brietest/flashback_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/tests/realtikvtest" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" + tikvutil "github.com/tikv/client-go/v2/util" ) // MockGC is used to make GC work in the test environment. @@ -42,9 +43,8 @@ func MockGC(tk *testkit.TestKit) (string, string, string, func()) { // disable emulator GC. // Otherwise emulator GC will delete table record as soon as possible after execute drop table ddl. ddlutil.EmulatorGCDisable() - gcTimeFormat := "20060102-15:04:05 -0700 MST" - timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(gcTimeFormat) - timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(gcTimeFormat) + timeBeforeDrop := time.Now().Add(0 - 48*60*60*time.Second).Format(tikvutil.GCTimeFormat) + timeAfterDrop := time.Now().Add(48 * 60 * 60 * time.Second).Format(tikvutil.GCTimeFormat) safePointSQL := `INSERT HIGH_PRIORITY INTO mysql.tidb VALUES ('tikv_gc_safe_point', '%[1]s', '') ON DUPLICATE KEY UPDATE variable_value = '%[1]s'` From 3208719a63971156a978e98866c54a04b5f59ada Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Tue, 8 Nov 2022 16:01:50 +0800 Subject: [PATCH 10/22] planner: update some UTs from cost model1 to model2 (#38959) ref pingcap/tidb#35240 --- executor/explainfor_test.go | 25 ++-- executor/showtest/show_test.go | 5 +- planner/core/cbo_test.go | 1 + planner/core/expression_rewriter_test.go | 1 + planner/core/integration_test.go | 3 + planner/core/physical_plan_test.go | 6 + planner/core/plan_cost_ver2.go | 7 +- planner/core/rule_join_reorder_test.go | 1 + planner/core/rule_result_reorder_test.go | 1 + planner/core/testdata/analyze_suite_out.json | 2 +- .../expression_rewriter_suite_out.json | 14 +- .../core/testdata/integration_suite_out.json | 132 +++++++++--------- .../core/testdata/join_reorder_suite_out.json | 8 +- .../ordered_result_mode_suite_out.json | 13 +- planner/core/testdata/plan_suite_out.json | 56 ++++---- 15 files changed, 149 insertions(+), 126 deletions(-) diff --git a/executor/explainfor_test.go b/executor/explainfor_test.go index 6d7cc1f11f7dd..db80c89f8d26e 100644 --- a/executor/explainfor_test.go +++ b/executor/explainfor_test.go @@ -443,6 +443,7 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "localhost", CurrentUser: true, AuthUsername: "root", AuthHostname: "%"}, nil, []byte("012345678901234567890")) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set @@tidb_enable_collect_execution_info=0;") tk.Session().GetSessionVars().EnableClusteredIndex = variable.ClusteredIndexDefModeOn tk.MustExec("drop table if exists t1") @@ -461,12 +462,12 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use idx_a `Projection_9 1.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b`, - `└─IndexJoin_17 1.00 root inner join, inner:TableReader_13, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)`, - ` ├─Selection_44(Build) 0.80 root not(isnull(test.t2.a))`, - ` │ └─Point_Get_43 1.00 root table:t2, index:idx_a(a) `, - ` └─TableReader_13(Probe) 0.00 root data:Selection_12`, - ` └─Selection_12 0.00 cop[tikv] eq(test.t1.a, 1)`, - ` └─TableRangeScan_11 0.80 cop[tikv] table:t1 range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo`)) + `└─MergeJoin_10 1.00 root inner join, left key:test.t2.a, right key:test.t1.a`, + ` ├─Selection_42(Build) 10.00 root eq(test.t1.a, 1)`, + ` │ └─TableReader_41 10.00 root data:TableRangeScan_40`, + ` │ └─TableRangeScan_40 10.00 cop[tikv] table:t1 range:[1,1], keep order:true, stats:pseudo`, + ` └─Selection_39(Probe) 0.80 root not(isnull(test.t2.a))`, + ` └─Point_Get_38 1.00 root table:t2, index:idx_a(a) `)) tk.MustExec("set @a=2") tk.MustQuery("execute stmt using @a").Check(testkit.Rows( @@ -477,12 +478,12 @@ func TestPointGetUserVarPlanCache(t *testing.T) { tk.Session().SetSessionManager(&testkit.MockSessionManager{PS: ps}) tk.MustQuery(fmt.Sprintf("explain for connection %d", tkProcess.ID)).Check(testkit.Rows( // can use idx_a `Projection_9 1.00 root test.t1.a, test.t1.b, test.t2.a, test.t2.b`, - `└─IndexJoin_17 1.00 root inner join, inner:TableReader_13, outer key:test.t2.a, inner key:test.t1.a, equal cond:eq(test.t2.a, test.t1.a)`, - ` ├─Selection_44(Build) 0.80 root not(isnull(test.t2.a))`, - ` │ └─Point_Get_43 1.00 root table:t2, index:idx_a(a) `, - ` └─TableReader_13(Probe) 0.00 root data:Selection_12`, - ` └─Selection_12 0.00 cop[tikv] eq(test.t1.a, 2)`, - ` └─TableRangeScan_11 0.80 cop[tikv] table:t1 range: decided by [eq(test.t1.a, test.t2.a)], keep order:false, stats:pseudo`)) + `└─MergeJoin_10 1.00 root inner join, left key:test.t2.a, right key:test.t1.a`, + ` ├─Selection_42(Build) 10.00 root eq(test.t1.a, 2)`, + ` │ └─TableReader_41 10.00 root data:TableRangeScan_40`, + ` │ └─TableRangeScan_40 10.00 cop[tikv] table:t1 range:[2,2], keep order:true, stats:pseudo`, + ` └─Selection_39(Probe) 0.80 root not(isnull(test.t2.a))`, + ` └─Point_Get_38 1.00 root table:t2, index:idx_a(a) `)) tk.MustQuery("execute stmt using @a").Check(testkit.Rows( "2 4 2 2", )) diff --git a/executor/showtest/show_test.go b/executor/showtest/show_test.go index cd06bd56b2229..11fa95dc0bb8c 100644 --- a/executor/showtest/show_test.go +++ b/executor/showtest/show_test.go @@ -775,6 +775,7 @@ func TestShowWarningsForExprPushdown(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec(`set tidb_cost_model_version=2`) testSQL := `create table if not exists show_warnings_expr_pushdown (a int, value date)` tk.MustExec(testSQL) @@ -796,10 +797,10 @@ func TestShowWarningsForExprPushdown(t *testing.T) { tk.MustExec("explain select * from show_warnings_expr_pushdown t where md5(value) = '2020-01-01'") require.Equal(t, uint16(1), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.")) - tk.MustExec("explain select max(md5(value)) from show_warnings_expr_pushdown group by a") + tk.MustExec("explain select /*+ read_from_storage(tiflash[show_warnings_expr_pushdown]) */ max(md5(value)) from show_warnings_expr_pushdown group by a") require.Equal(t, uint16(2), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", "Warning|1105|Aggregation can not be pushed to tiflash because arguments of AggFunc `max` contains unsupported exprs")) - tk.MustExec("explain select max(a) from show_warnings_expr_pushdown group by md5(value)") + tk.MustExec("explain select /*+ read_from_storage(tiflash[show_warnings_expr_pushdown]) */ max(a) from show_warnings_expr_pushdown group by md5(value)") require.Equal(t, uint16(2), tk.Session().GetSessionVars().StmtCtx.WarningCount()) tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Warning|1105|Scalar function 'md5'(signature: MD5, return type: var_string(32)) is not supported to push down to tiflash now.", "Warning|1105|Aggregation can not be pushed to tiflash because groupByItems contain unsupported exprs")) tk.MustExec("set tidb_opt_distinct_agg_push_down=0") diff --git a/planner/core/cbo_test.go b/planner/core/cbo_test.go index c7e663f43ff1e..d9c4dba7b676d 100644 --- a/planner/core/cbo_test.go +++ b/planner/core/cbo_test.go @@ -264,6 +264,7 @@ func TestEmptyTable(t *testing.T) { store := testkit.CreateMockStore(t) testKit := testkit.NewTestKit(t, store) testKit.MustExec("use test") + testKit.MustExec("set tidb_cost_model_version=2") testKit.MustExec("drop table if exists t, t1") testKit.MustExec("create table t (c1 int)") testKit.MustExec("create table t1 (c1 int)") diff --git a/planner/core/expression_rewriter_test.go b/planner/core/expression_rewriter_test.go index 3de3509c3e093..2e4f9e036b830 100644 --- a/planner/core/expression_rewriter_test.go +++ b/planner/core/expression_rewriter_test.go @@ -381,6 +381,7 @@ func TestMultiColInExpression(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test;") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t1, t2") tk.MustExec("create table t1(a int, b int)") tk.MustExec("insert into t1 values(1,1),(2,null),(null,3),(4,4)") diff --git a/planner/core/integration_test.go b/planner/core/integration_test.go index a89b85b736d62..889983c09f0ef 100644 --- a/planner/core/integration_test.go +++ b/planner/core/integration_test.go @@ -615,6 +615,7 @@ func TestMPPJoin(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists d1_t") tk.MustExec("create table d1_t(d1_k int, value int)") tk.MustExec("insert into d1_t values(1,2),(2,3)") @@ -6083,6 +6084,7 @@ func TestRejectSortForMPP(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int, value decimal(6,3), name char(128))") tk.MustExec("analyze table t") @@ -6709,6 +6711,7 @@ func TestTiFlashFineGrainedShuffle(t *testing.T) { store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set @@tidb_isolation_read_engines = 'tiflash'") tk.MustExec("set @@tidb_enforce_mpp = on") tk.MustExec("drop table if exists t1;") diff --git a/planner/core/physical_plan_test.go b/planner/core/physical_plan_test.go index c78ef7e8fd27d..dece0b960bf21 100644 --- a/planner/core/physical_plan_test.go +++ b/planner/core/physical_plan_test.go @@ -54,6 +54,7 @@ func TestDAGPlanBuilderSimpleCase(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set tidb_opt_limit_push_down_threshold=0") var input []string var output []struct { @@ -599,6 +600,7 @@ func TestIndexJoinUnionScan(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("create table t (a int primary key, b int, index idx(a))") tk.MustExec("create table tt (a int primary key) partition by range (a) (partition p0 values less than (100), partition p1 values less than (200))") tk.MustExec(`set @@tidb_partition_prune_mode='` + string(variable.Static) + `'`) @@ -720,6 +722,7 @@ func TestSemiJoinToInner(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") var input []string var output []struct { @@ -976,6 +979,7 @@ func TestSemiJoinRewriteHints(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("create table t(a int, b int, c int)") sessionVars := tk.Session().GetSessionVars() @@ -1218,6 +1222,7 @@ func TestForceInlineCTE(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t;") tk.MustExec("CREATE TABLE `t` (`a` int(11));") tk.MustExec("insert into t values (1), (5), (10), (15), (20), (30), (50);") @@ -2111,6 +2116,7 @@ func TestSkewDistinctAgg(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t") tk.MustExec("CREATE TABLE `t` (`a` int(11), `b` int(11), `c` int(11), `d` date)") tk.MustExec("insert into t (a,b,c,d) value(1,4,5,'2019-06-01')") diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 0242777ae6ef5..1afa6237386ae 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -246,13 +246,14 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, } tableSideCost := divCostVer2(sumCostVer2(tableNetCost, tableChildCost), distConcurrency) - // double-read + // double-read: assume at least 1 row to double-read to avoid 0 double-read cost. + doubleReadRows := math.Max(indexRows, 1) doubleReadCPUCost := newCostVer2(option, cpuFactor, indexRows*cpuFactor.Value, - "double-read-cpu(%v*%v)", indexRows, cpuFactor) + "double-read-cpu(%v*%v)", doubleReadRows, cpuFactor) batchSize := float64(p.ctx.GetSessionVars().IndexLookupSize) taskPerBatch := 32.0 // TODO: remove this magic number - doubleReadTasks := indexRows / batchSize * taskPerBatch + doubleReadTasks := doubleReadRows / batchSize * taskPerBatch doubleReadRequestCost := doubleReadCostVer2(option, doubleReadTasks, requestFactor) doubleReadCost := sumCostVer2(doubleReadCPUCost, doubleReadRequestCost) diff --git a/planner/core/rule_join_reorder_test.go b/planner/core/rule_join_reorder_test.go index 609b2dde4499a..d4d35f5de80a4 100644 --- a/planner/core/rule_join_reorder_test.go +++ b/planner/core/rule_join_reorder_test.go @@ -261,6 +261,7 @@ func TestJoinOrderHint4DifferentJoinType(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("drop table if exists t, t1, t2, t3, t4, t5, t6, t7, t8;") tk.MustExec("create table t(a int, b int, key(a));") tk.MustExec("create table t1(a int, b int, key(a));") diff --git a/planner/core/rule_result_reorder_test.go b/planner/core/rule_result_reorder_test.go index 060b44da245aa..0880f6426c151 100644 --- a/planner/core/rule_result_reorder_test.go +++ b/planner/core/rule_result_reorder_test.go @@ -130,6 +130,7 @@ func TestOrderedResultModeOnSubQuery(t *testing.T) { tk := testkit.NewTestKit(t, store) tk.MustExec("use test") + tk.MustExec("set tidb_cost_model_version=2") tk.MustExec("set tidb_enable_ordered_result_mode=1") tk.MustExec("drop table if exists t1") tk.MustExec("drop table if exists t2") diff --git a/planner/core/testdata/analyze_suite_out.json b/planner/core/testdata/analyze_suite_out.json index d9345a1a5bcce..750fc736a7f6e 100644 --- a/planner/core/testdata/analyze_suite_out.json +++ b/planner/core/testdata/analyze_suite_out.json @@ -424,7 +424,7 @@ "Name": "TestEmptyTable", "Cases": [ "TableReader(Table(t)->Sel([le(test.t.c1, 50)]))", - "LeftHashJoin{TableReader(Table(t)->Sel([not(isnull(test.t.c1))]))->TableReader(Table(t1)->Sel([not(isnull(test.t1.c1))]))->HashAgg}(test.t.c1,test.t1.c1)", + "LeftHashJoin{TableReader(Table(t)->Sel([not(isnull(test.t.c1))]))->TableReader(Table(t1)->Sel([not(isnull(test.t1.c1))])->HashAgg)->HashAgg}(test.t.c1,test.t1.c1)", "LeftHashJoin{TableReader(Table(t)->Sel([not(isnull(test.t.c1))]))->TableReader(Table(t1)->Sel([not(isnull(test.t1.c1))]))}(test.t.c1,test.t1.c1)", "Dual" ] diff --git a/planner/core/testdata/expression_rewriter_suite_out.json b/planner/core/testdata/expression_rewriter_suite_out.json index 3c850e434fc9b..72b86884a1412 100644 --- a/planner/core/testdata/expression_rewriter_suite_out.json +++ b/planner/core/testdata/expression_rewriter_suite_out.json @@ -7,10 +7,9 @@ "Plan": [ "HashJoin 2.25 root inner join, equal:[eq(test.t1.a, test.t2.a) eq(test.t1.b, test.t2.b)]", "├─HashAgg(Build) 1.69 root group by:test.t2.a, test.t2.b, funcs:firstrow(test.t2.a)->test.t2.a, funcs:firstrow(test.t2.b)->test.t2.b", - "│ └─TableReader 1.69 root data:HashAgg", - "│ └─HashAgg 1.69 cop[tikv] group by:test.t2.a, test.t2.b, ", - "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", + "│ └─TableReader 2.25 root data:Selection", + "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", "└─TableReader(Probe) 2.25 root data:Selection", " └─Selection 2.25 cop[tikv] not(isnull(test.t1.a)), not(isnull(test.t1.b))", " └─TableFullScan 4.00 cop[tikv] table:t1 keep order:false" @@ -37,10 +36,9 @@ "Plan": [ "HashJoin 1.69 root inner join, equal:[eq(test.t2.a, test.t1.a) eq(test.t2.b, Column#7)]", "├─HashAgg(Build) 1.69 root group by:test.t2.a, test.t2.b, funcs:firstrow(test.t2.a)->test.t2.a, funcs:firstrow(test.t2.b)->test.t2.b", - "│ └─TableReader 1.69 root data:HashAgg", - "│ └─HashAgg 1.69 cop[tikv] group by:test.t2.a, test.t2.b, ", - "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", - "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", + "│ └─TableReader 2.25 root data:Selection", + "│ └─Selection 2.25 cop[tikv] not(isnull(test.t2.a)), not(isnull(test.t2.b))", + "│ └─TableFullScan 4.00 cop[tikv] table:t2 keep order:false", "└─HashAgg(Probe) 2.25 root group by:test.t1.a, funcs:count(1)->Column#7, funcs:firstrow(test.t1.a)->test.t1.a", " └─TableReader 3.00 root data:Selection", " └─Selection 3.00 cop[tikv] not(isnull(test.t1.a))", diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 0a4b38ba0b05b..0a04a8f55f24d 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -2558,8 +2558,8 @@ { "SQL": "explain format = 'verbose' select /*+ use_index(t3, c) */ count(a) from t3 where b = 0", "Plan": [ - "StreamAgg_10 1.00 0.00 root funcs:count(test.t3.a)->Column#4", - "└─IndexLookUp_17 0.00 0.00 root ", + "StreamAgg_10 1.00 1920.00 root funcs:count(test.t3.a)->Column#4", + "└─IndexLookUp_17 0.00 1920.00 root ", " ├─IndexRangeScan_15(Build) 0.00 0.00 cop[tikv] table:t3, index:c(b) range:[0,0], keep order:false", " └─TableRowIDScan_16(Probe) 0.00 0.00 cop[tikv] table:t3 keep order:false" ] @@ -3196,47 +3196,44 @@ { "SQL": "explain format = 'brief' select count(*) from fact_t join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN inner join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─Selection(Probe) 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t left join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", - " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", - " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", - " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN left outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 2.00 mpp[tiflash] ", + " │ └─ExchangeSender 2.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 2.00 mpp[tiflash] not(isnull(test.d1_t.d1_k))", + " │ └─TableFullScan 2.00 mpp[tiflash] table:d1_t keep order:false", + " └─TableFullScan(Probe) 8.00 mpp[tiflash] table:fact_t keep order:false" ] }, { "SQL": "explain format = 'brief' select count(*) from fact_t right join d1_t on fact_t.d1_k > d1_t.d1_k", "Plan": [ - "HashAgg 1.00 root funcs:count(Column#12)->Column#11", - "└─TableReader 1.00 root data:ExchangeSender", - " └─ExchangeSender 1.00 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg 1.00 mpp[tiflash] funcs:count(1)->Column#12", - " └─HashJoin 16.00 mpp[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", - " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", - " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", - " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", - " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" + "StreamAgg 1.00 root funcs:count(1)->Column#11", + "└─TableReader 16.00 root data:ExchangeSender", + " └─ExchangeSender 16.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin 16.00 mpp[tiflash] CARTESIAN right outer join, other cond:gt(test.fact_t.d1_k, test.d1_t.d1_k)", + " ├─ExchangeReceiver(Build) 8.00 mpp[tiflash] ", + " │ └─ExchangeSender 8.00 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection 8.00 mpp[tiflash] not(isnull(test.fact_t.d1_k))", + " │ └─TableFullScan 8.00 mpp[tiflash] table:fact_t keep order:false", + " └─TableFullScan(Probe) 2.00 mpp[tiflash] table:d1_t keep order:false" ] }, { @@ -7080,10 +7077,11 @@ " └─TableReader 8000.00 root data:ExchangeSender", " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#5, test.t.id", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#7)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#7", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -7093,10 +7091,11 @@ "└─TableReader 8000.00 root data:ExchangeSender", " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: PassThrough", " └─Projection 8000.00 mpp[tiflash] Column#5", - " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:count(1)->Column#5", - " └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", - " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:sum(Column#8)->Column#5", + " └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin]", + " └─HashAgg 8000.00 mpp[tiflash] group by:test.t.name, funcs:count(1)->Column#8", + " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] }, { @@ -7127,10 +7126,11 @@ " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: Broadcast", " │ └─Projection 7992.00 mpp[tiflash] test.t.id", " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 9990.00 mpp[tiflash] ", - " │ └─ExchangeSender 9990.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", - " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─ExchangeReceiver 7992.00 mpp[tiflash] ", + " │ └─ExchangeSender 7992.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.id, collate: binary]", + " │ └─HashAgg 7992.00 mpp[tiflash] group by:test.t.id, ", + " │ └─Selection 9990.00 mpp[tiflash] not(isnull(test.t.id))", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Selection(Probe) 9990.00 mpp[tiflash] not(isnull(test.t.id))", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ] @@ -7172,10 +7172,11 @@ " ├─Projection 8000.00 mpp[tiflash] cast(Column#12, bigint(21) BINARY)->Column#12", " │ └─Projection 8000.00 mpp[tiflash] Column#5", " │ └─Projection 8000.00 mpp[tiflash] Column#5, test.t.id", - " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#5, funcs:firstrow(test.t.id)->test.t.id", - " │ └─ExchangeReceiver 10000.00 mpp[tiflash] ", - " │ └─ExchangeSender 10000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", - " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:sum(Column#19)->Column#5, funcs:firstrow(test.t.id)->test.t.id", + " │ └─ExchangeReceiver 8000.00 mpp[tiflash] ", + " │ └─ExchangeSender 8000.00 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t.name, collate: utf8mb4_bin], [name: test.t.id, collate: binary]", + " │ └─HashAgg 8000.00 mpp[tiflash] group by:test.t.id, test.t.name, funcs:count(1)->Column#19", + " │ └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo", " └─Projection 10000.00 mpp[tiflash] cast(Column#11, bigint(21) BINARY)->Column#12", " └─Projection 10000.00 mpp[tiflash] plus(test.t.id, 1)->Column#11", " └─TableFullScan 10000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" @@ -7320,11 +7321,12 @@ " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c2, collate: binary], stream_count: 8", " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c2", - " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#4, funcs:firstrow(test.t1.c2)->test.t1.c2", - " └─ExchangeReceiver 3333.33 mpp[tiflash] ", - " └─ExchangeSender 3333.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary]", - " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", - " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -7338,11 +7340,12 @@ " └─ExchangeReceiver 2666.67 mpp[tiflash] stream_count: 8", " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary], stream_count: 8", " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", - " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", - " └─ExchangeReceiver 3333.33 mpp[tiflash] ", - " └─ExchangeSender 3333.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c2, collate: binary]", - " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, 10)", - " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:sum(Column#9)->Column#4, funcs:firstrow(Column#10)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c2, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c2, funcs:count(test.t1.c1)->Column#9, funcs:firstrow(test.t1.c1)->Column#10", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c2, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] }, { @@ -7408,11 +7411,12 @@ " └─Window 2666.67 mpp[tiflash] row_number()->Column#6 over(partition by test.t1.c1 order by test.t1.c2 rows between current row and current row)", " └─Sort 2666.67 mpp[tiflash] test.t1.c1, test.t1.c2", " └─Projection 2666.67 mpp[tiflash] Column#4, test.t1.c1, test.t1.c2", - " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(test.t1.c2)->test.t1.c2", - " └─ExchangeReceiver 3333.33 mpp[tiflash] ", - " └─ExchangeSender 3333.33 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary]", - " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", - " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:sum(Column#9)->Column#4, funcs:firstrow(test.t1.c1)->test.t1.c1, funcs:firstrow(Column#11)->test.t1.c2", + " └─ExchangeReceiver 2666.67 mpp[tiflash] ", + " └─ExchangeSender 2666.67 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t1.c1, collate: binary]", + " └─HashAgg 2666.67 mpp[tiflash] group by:test.t1.c1, funcs:count(test.t1.c2)->Column#9, funcs:firstrow(test.t1.c2)->Column#11", + " └─Selection 3333.33 mpp[tiflash] gt(test.t1.c1, 10)", + " └─TableFullScan 10000.00 mpp[tiflash] table:t1 keep order:false, stats:pseudo" ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 159862538cbe9..829a2683b4775 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -5320,11 +5320,11 @@ "│ └─TableFullScan 10000.00 cop[tikv] table:t3 keep order:false, stats:pseudo", "└─HashJoin(Probe) 12487.50 root right outer join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", - " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", - " │ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", + " │ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", " └─TableReader(Probe) 9990.00 root data:Selection", - " └─Selection 9990.00 cop[tikv] not(isnull(test.t2.b))", - " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" + " └─Selection 9990.00 cop[tikv] not(isnull(test.t1.a))", + " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" ], "Warning": [ "Warning 1815 leading hint is inapplicable, check if the leading hint table is valid", diff --git a/planner/core/testdata/ordered_result_mode_suite_out.json b/planner/core/testdata/ordered_result_mode_suite_out.json index f6acccadce758..7f6563beaa1a8 100644 --- a/planner/core/testdata/ordered_result_mode_suite_out.json +++ b/planner/core/testdata/ordered_result_mode_suite_out.json @@ -183,12 +183,13 @@ { "Plan": [ "Sort_11 9990.00 root test.t1.a, test.t1.b, test.t1.c, test.t1.d", - "└─HashJoin_23 9990.00 root inner join, equal:[eq(test.t1.a, test.t2.b)]", - " ├─HashAgg_36(Build) 7992.00 root group by:test.t2.b, funcs:firstrow(test.t2.b)->test.t2.b", - " │ └─IndexReader_43 9990.00 root index:IndexFullScan_42", - " │ └─IndexFullScan_42 9990.00 cop[tikv] table:t2, index:b(b) keep order:false, stats:pseudo", - " └─TableReader_47(Probe) 10000.00 root data:TableFullScan_46", - " └─TableFullScan_46 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "└─MergeJoin_13 9990.00 root inner join, left key:test.t1.a, right key:test.t2.b", + " ├─StreamAgg_30(Build) 7992.00 root group by:test.t2.b, funcs:firstrow(test.t2.b)->test.t2.b", + " │ └─IndexReader_31 7992.00 root index:StreamAgg_27", + " │ └─StreamAgg_27 7992.00 cop[tikv] group by:test.t2.b, ", + " │ └─IndexFullScan_29 9990.00 cop[tikv] table:t2, index:b(b) keep order:true, stats:pseudo", + " └─TableReader_26(Probe) 10000.00 root data:TableFullScan_25", + " └─TableFullScan_25 10000.00 cop[tikv] table:t1 keep order:true, stats:pseudo" ] }, { diff --git a/planner/core/testdata/plan_suite_out.json b/planner/core/testdata/plan_suite_out.json index 572f81f7b0b50..adbd39cf82e7e 100644 --- a/planner/core/testdata/plan_suite_out.json +++ b/planner/core/testdata/plan_suite_out.json @@ -397,7 +397,7 @@ }, { "SQL": "select * from t where (t.c > 0 and t.c < 2) or (t.c > 4 and t.c < 6) or (t.c > 8 and t.c < 10) or (t.c > 12 and t.c < 14) or (t.c > 16 and t.c < 18)", - "Best": "IndexLookUp(Index(t.c_d_e)[[1,1] [5,5] [9,9] [13,13] [17,17]], Table(t))" + "Best": "TableReader(Table(t)->Sel([or(or(and(gt(test.t.c, 0), lt(test.t.c, 2)), and(gt(test.t.c, 4), lt(test.t.c, 6))), or(and(gt(test.t.c, 8), lt(test.t.c, 10)), or(and(gt(test.t.c, 12), lt(test.t.c, 14)), and(gt(test.t.c, 16), lt(test.t.c, 18)))))]))" }, { "SQL": "select * from t where (t.c > 0 and t.c < 1) or (t.c > 2 and t.c < 3) or (t.c > 4 and t.c < 5) or (t.c > 6 and t.c < 7) or (t.c > 9 and t.c < 10)", @@ -477,7 +477,7 @@ }, { "SQL": "select c from t where t.c = 1 and t.d = 1 order by t.a limit 1", - "Best": "IndexReader(Index(t.c_d_e)[[1 1,1 1]])->TopN([test.t.a],0,1)->Projection" + "Best": "IndexReader(Index(t.c_d_e)[[1 1,1 1]]->TopN([test.t.a],0,1))->TopN([test.t.a],0,1)->Projection" }, { "SQL": "select * from t where t.c = 1 and t.a > 1 order by t.d limit 1", @@ -1528,13 +1528,13 @@ ], "Plan": [ "HashJoin 12487.50 root inner join, equal:[eq(test.t.a, test.t.b)]", - "├─UnionScan(Build) 9990.00 root not(isnull(test.t.b))", - "│ └─TableReader 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.b))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo", - "└─UnionScan(Probe) 10000.00 root ", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo" + "├─UnionScan(Build) 10000.00 root ", + "│ └─TableReader 10000.00 root data:TableFullScan", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "└─UnionScan(Probe) 9990.00 root not(isnull(test.t.b))", + " └─TableReader 9990.00 root data:Selection", + " └─Selection 9990.00 cop[tikv] not(isnull(test.t.b))", + " └─TableFullScan 10000.00 cop[tikv] table:t2 keep order:false, stats:pseudo" ] }, { @@ -1588,7 +1588,7 @@ "Cases": [ { "SQL": "select t1.a, (select count(t2.a) from t t2 where t2.g in (select t3.d from t t3 where t3.c = t1.a)) as agg_col from t t1;", - "Best": "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexHashJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]]->HashAgg)->HashAgg->IndexReader(Index(t.g)[[NULL,NULL]])}(test.t.d,test.t.g)}->HashAgg" + "Best": "Apply{IndexReader(Index(t.f)[[NULL,+inf]])->IndexJoin{IndexReader(Index(t.c_d_e)[[NULL,+inf]])->HashAgg->IndexReader(Index(t.g)[[NULL,NULL]])}(test.t.d,test.t.g)}->HashAgg" } ] }, @@ -1995,8 +1995,9 @@ "Plan": [ "HashJoin 80000000.00 root CARTESIAN inner join", "├─HashAgg(Build) 8000.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─TableReader 10000.00 root data:TableFullScan", - "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", + "│ └─TableReader 8000.00 root data:HashAgg", + "│ └─HashAgg 8000.00 cop[tikv] group by:test.t.a, ", + "│ └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo", "└─TableReader(Probe) 10000.00 root data:TableFullScan", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ], @@ -3362,11 +3363,11 @@ "SQL": "select date_format(d,'%Y') as df, sum(a), count(b), count(distinct c) from t group by date_format(d,'%Y')", "Plan": [ "Projection 8000.00 root date_format(test.t.d, %Y)->Column#9, Column#6, cast(Column#13, bigint(21) BINARY)->Column#7, Column#8", - "└─HashAgg 8000.00 root group by:Column#29, funcs:sum(Column#25)->Column#6, funcs:sum(Column#26)->Column#13, funcs:count(Column#27)->Column#8, funcs:firstrow(Column#28)->test.t.d", - " └─Projection 8000.00 root Column#11, cast(Column#12, decimal(20,0) BINARY)->Column#26, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#29", - " └─HashAgg 8000.00 root group by:Column#23, Column#24, funcs:sum(Column#19)->Column#11, funcs:count(Column#20)->Column#12, funcs:firstrow(Column#21)->test.t.c, funcs:firstrow(Column#22)->test.t.d", - " └─Projection 10000.00 root cast(test.t.a, decimal(10,0) BINARY)->Column#19, test.t.b, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#23, test.t.c", - " └─TableReader 10000.00 root data:TableFullScan", + "└─HashAgg 8000.00 root group by:Column#23, funcs:sum(Column#19)->Column#6, funcs:sum(Column#20)->Column#13, funcs:count(Column#21)->Column#8, funcs:firstrow(Column#22)->test.t.d", + " └─Projection 8000.00 root Column#11, cast(Column#12, decimal(20,0) BINARY)->Column#20, test.t.c, test.t.d, date_format(test.t.d, %Y)->Column#23", + " └─HashAgg 8000.00 root group by:Column#14, test.t.c, funcs:sum(Column#15)->Column#11, funcs:count(Column#16)->Column#12, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(Column#18)->test.t.d", + " └─TableReader 8000.00 root data:HashAgg", + " └─HashAgg 8000.00 cop[tikv] group by:date_format(test.t.d, \"%Y\"), test.t.c, funcs:sum(test.t.a)->Column#15, funcs:count(test.t.b)->Column#16, funcs:firstrow(test.t.d)->Column#18", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ] }, @@ -3376,9 +3377,10 @@ "Projection 8000.00 root test.t.d, test.t.a, cast(Column#10, bigint(21) BINARY)->Column#6, cast(Column#12, bigint(21) BINARY)->Column#7, Column#8", "└─HashAgg 8000.00 root group by:Column#23, Column#24, funcs:sum(Column#18)->Column#10, funcs:sum(Column#19)->Column#12, funcs:count(Column#20)->Column#8, funcs:firstrow(Column#21)->test.t.a, funcs:firstrow(Column#22)->test.t.d", " └─Projection 8000.00 root cast(Column#9, decimal(20,0) BINARY)->Column#18, cast(Column#11, decimal(20,0) BINARY)->Column#19, test.t.c, test.t.a, test.t.d, test.t.d, test.t.a", - " └─HashAgg 8000.00 root group by:test.t.a, test.t.c, test.t.d, funcs:count(1)->Column#9, funcs:count(test.t.b)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(test.t.d)->test.t.d", - " └─TableReader 10000.00 root data:TableFullScan", - " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" + " └─HashAgg 8000.00 root group by:test.t.a, test.t.c, test.t.d, funcs:count(Column#13)->Column#9, funcs:count(Column#14)->Column#11, funcs:firstrow(test.t.c)->test.t.c, funcs:firstrow(test.t.a)->test.t.a, funcs:firstrow(test.t.d)->test.t.d", + " └─TableReader 8000.00 root data:HashAgg", + " └─HashAgg 8000.00 cop[tikv] group by:test.t.a, test.t.c, test.t.d, funcs:count(1)->Column#13, funcs:count(test.t.b)->Column#14", + " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" ] }, { @@ -3874,9 +3876,10 @@ "Plan": [ "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", "├─HashAgg(Build) 7992.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─TableReader 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t.a, ", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" @@ -3888,9 +3891,10 @@ "Plan": [ "HashJoin 9990.00 root inner join, equal:[eq(test.t.a, test.t.a)]", "├─HashAgg(Build) 7992.00 root group by:test.t.a, funcs:firstrow(test.t.a)->test.t.a", - "│ └─TableReader 9990.00 root data:Selection", - "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", - "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", + "│ └─TableReader 7992.00 root data:HashAgg", + "│ └─HashAgg 7992.00 cop[tikv] group by:test.t.a, ", + "│ └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", + "│ └─TableFullScan 10000.00 cop[tikv] table:t1 keep order:false, stats:pseudo", "└─TableReader(Probe) 9990.00 root data:Selection", " └─Selection 9990.00 cop[tikv] not(isnull(test.t.a))", " └─TableFullScan 10000.00 cop[tikv] table:t keep order:false, stats:pseudo" From 96708adc93e32b494d70dc03a3a547ac155097c0 Mon Sep 17 00:00:00 2001 From: Jianjun Liao <36503113+Leavrth@users.noreply.github.com> Date: Tue, 8 Nov 2022 16:47:50 +0800 Subject: [PATCH 11/22] log-backup: truncate metadata streamingly (#38870) close pingcap/tidb#38869, ref pingcap/tidb#38939, ref pingcap/tidb#38940 --- br/pkg/restore/BUILD.bazel | 1 - br/pkg/restore/log_client_test.go | 11 +- br/pkg/restore/stream_metas.go | 320 +++-- br/pkg/restore/stream_metas_test.go | 2077 +++++++++++++++++++++++++-- br/pkg/task/stream.go | 100 +- 5 files changed, 2198 insertions(+), 311 deletions(-) diff --git a/br/pkg/restore/BUILD.bazel b/br/pkg/restore/BUILD.bazel index 2c9ce90552f7b..2c42717bafe36 100644 --- a/br/pkg/restore/BUILD.bazel +++ b/br/pkg/restore/BUILD.bazel @@ -149,7 +149,6 @@ go_test( "//testkit/testsetup", "//types", "//util/codec", - "//util/mathutil", "@com_github_fsouza_fake_gcs_server//fakestorage", "@com_github_golang_protobuf//proto", "@com_github_pingcap_errors//:errors", diff --git a/br/pkg/restore/log_client_test.go b/br/pkg/restore/log_client_test.go index c4c6d46d15b3f..71db52cf7678f 100644 --- a/br/pkg/restore/log_client_test.go +++ b/br/pkg/restore/log_client_test.go @@ -300,12 +300,19 @@ func testReadFromMetadataWithVersion(t *testing.T, m metaMaker) { meta := new(StreamMetadataSet) meta.Helper = stream.NewMetadataHelper() - meta.LoadUntil(ctx, loc, c.untilTS) + meta.LoadUntilAndCalculateShiftTS(ctx, loc, c.untilTS) var metas []*backuppb.Metadata - for _, m := range meta.metadata { + for path := range meta.metadataInfos { + data, err := loc.ReadFile(ctx, path) + require.NoError(t, err) + + m, err := meta.Helper.ParseToMetadataHard(data) + require.NoError(t, err) + metas = append(metas, m) } + actualStoreIDs := make([]int64, 0, len(metas)) for _, meta := range metas { actualStoreIDs = append(actualStoreIDs, meta.StoreId) diff --git a/br/pkg/restore/stream_metas.go b/br/pkg/restore/stream_metas.go index 24736c9298a70..2aa9c8f11a9db 100644 --- a/br/pkg/restore/stream_metas.go +++ b/br/pkg/restore/stream_metas.go @@ -12,67 +12,116 @@ import ( backuppb "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" berrors "github.com/pingcap/tidb/br/pkg/errors" + "github.com/pingcap/tidb/br/pkg/logutil" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" + "github.com/pingcap/tidb/br/pkg/utils" "github.com/pingcap/tidb/util/mathutil" "go.uber.org/zap" + "golang.org/x/sync/errgroup" ) +const notDeletedBecameFatalThreshold = 128 + type StreamMetadataSet struct { - metadata map[string]*backuppb.Metadata - // The metadata after changed that needs to be write back. - writeback map[string]*backuppb.Metadata + // if set true, the metadata and datafile won't be removed + DryRun bool + + // keeps the meta-information of metadata as little as possible + // to save the memory + metadataInfos map[string]*MetadataInfo + // a parser of metadata Helper *stream.MetadataHelper - BeforeDoWriteBack func(path string, last, current *backuppb.Metadata) (skip bool) + // for test + BeforeDoWriteBack func(path string, replaced *backuppb.Metadata) (skip bool) } -// MetaLen returns the length of read metadata. -func (ms *StreamMetadataSet) MetaLen() int { - return len(ms.metadata) +// keep these meta-information for statistics and filtering +type FileGroupInfo struct { + MaxTS uint64 + Length uint64 + KVCount int64 } -// LoadUntil loads the metadata until the specified timestamp. -// This would load all metadata files that *may* contain data from transaction committed before that TS. -// Note: maybe record the timestamp and reject reading data files after this TS? -func (ms *StreamMetadataSet) LoadUntil(ctx context.Context, s storage.ExternalStorage, until uint64) error { +// keep these meta-information for statistics and filtering +type MetadataInfo struct { + MinTS uint64 + FileGroupInfos []*FileGroupInfo +} + +// LoadUntilAndCalculateShiftTS loads the metadata until the specified timestamp and calculate the shift-until-ts by the way. +// This would record all metadata files that *may* contain data from transaction committed before that TS. +func (ms *StreamMetadataSet) LoadUntilAndCalculateShiftTS(ctx context.Context, s storage.ExternalStorage, until uint64) (uint64, error) { metadataMap := struct { sync.Mutex - metas map[string]*backuppb.Metadata + metas map[string]*MetadataInfo + shiftUntilTS uint64 }{} - ms.writeback = make(map[string]*backuppb.Metadata) - metadataMap.metas = make(map[string]*backuppb.Metadata) + metadataMap.metas = make(map[string]*MetadataInfo) + // `shiftUntilTS` must be less than `until` + metadataMap.shiftUntilTS = until err := stream.FastUnmarshalMetaData(ctx, s, func(path string, raw []byte) error { m, err := ms.Helper.ParseToMetadataHard(raw) if err != nil { return err } - metadataMap.Lock() // If the meta file contains only files with ts grater than `until`, when the file is from // `Default`: it should be kept, because its corresponding `write` must has commit ts grater than it, which should not be considered. // `Write`: it should trivially not be considered. if m.MinTs <= until { - metadataMap.metas[path] = m + // record these meta-information for statistics and filtering + fileGroupInfos := make([]*FileGroupInfo, 0, len(m.FileGroups)) + for _, group := range m.FileGroups { + var kvCount int64 = 0 + for _, file := range group.DataFilesInfo { + kvCount += file.NumberOfEntries + } + fileGroupInfos = append(fileGroupInfos, &FileGroupInfo{ + MaxTS: group.MaxTs, + Length: group.Length, + KVCount: kvCount, + }) + } + metadataMap.Lock() + metadataMap.metas[path] = &MetadataInfo{ + MinTS: m.MinTs, + FileGroupInfos: fileGroupInfos, + } + metadataMap.Unlock() + } + // filter out the metadatas whose ts-range is overlap with [until, +inf) + // and calculate their minimum begin-default-ts + ts, ok := UpdateShiftTS(m, until, mathutil.MaxUint) + if ok { + metadataMap.Lock() + if ts < metadataMap.shiftUntilTS { + metadataMap.shiftUntilTS = ts + } + metadataMap.Unlock() } - metadataMap.Unlock() return nil }) if err != nil { - return errors.Trace(err) + return 0, errors.Trace(err) } - ms.metadata = metadataMap.metas - return nil + ms.metadataInfos = metadataMap.metas + if metadataMap.shiftUntilTS != until { + log.Warn("calculate shift-ts", zap.Uint64("start-ts", until), zap.Uint64("shift-ts", metadataMap.shiftUntilTS)) + } + return metadataMap.shiftUntilTS, nil } -// LoadFrom loads data from an external storage into the stream metadata set. +// LoadFrom loads data from an external storage into the stream metadata set. (Now only for test) func (ms *StreamMetadataSet) LoadFrom(ctx context.Context, s storage.ExternalStorage) error { - return ms.LoadUntil(ctx, s, math.MaxUint64) + _, err := ms.LoadUntilAndCalculateShiftTS(ctx, s, math.MaxUint64) + return err } -func (ms *StreamMetadataSet) iterateDataFiles(f func(d *backuppb.DataFileGroup) (shouldBreak bool)) { - for _, m := range ms.metadata { - for _, d := range m.FileGroups { +func (ms *StreamMetadataSet) iterateDataFiles(f func(d *FileGroupInfo) (shouldBreak bool)) { + for _, m := range ms.metadataInfos { + for _, d := range m.FileGroupInfos { if f(d) { return } @@ -80,21 +129,6 @@ func (ms *StreamMetadataSet) iterateDataFiles(f func(d *backuppb.DataFileGroup) } } -// CalculateShiftTS calculates the shift-ts. -func (ms *StreamMetadataSet) CalculateShiftTS(startTS uint64) uint64 { - metadatas := make([]*backuppb.Metadata, 0, len(ms.metadata)) - for _, m := range ms.metadata { - metadatas = append(metadatas, m) - } - - minBeginTS, exist := CalculateShiftTS(metadatas, startTS, mathutil.MaxUint) - if !exist { - minBeginTS = startTS - } - log.Warn("calculate shift-ts", zap.Uint64("start-ts", startTS), zap.Uint64("shift-ts", minBeginTS)) - return minBeginTS -} - // IterateFilesFullyBefore runs the function over all files contain data before the timestamp only. // // 0 before @@ -103,83 +137,145 @@ func (ms *StreamMetadataSet) CalculateShiftTS(startTS uint64) uint64 { // |-file2--------------| <- File contains any record out of this won't be found. // // This function would call the `f` over file1 only. -func (ms *StreamMetadataSet) IterateFilesFullyBefore(before uint64, f func(d *backuppb.DataFileGroup) (shouldBreak bool)) { - ms.iterateDataFiles(func(d *backuppb.DataFileGroup) (shouldBreak bool) { - if d.MaxTs >= before { +func (ms *StreamMetadataSet) IterateFilesFullyBefore(before uint64, f func(d *FileGroupInfo) (shouldBreak bool)) { + ms.iterateDataFiles(func(d *FileGroupInfo) (shouldBreak bool) { + if d.MaxTS >= before { return false } return f(d) }) } -// RemoveDataBefore would find files contains only records before the timestamp, mark them as removed from meta, -// and returning their information. -func (ms *StreamMetadataSet) RemoveDataBefore(from uint64) []*backuppb.DataFileGroup { - removed := []*backuppb.DataFileGroup{} - for metaPath, m := range ms.metadata { - remainedDataFiles := make([]*backuppb.DataFileGroup, 0) - // can we assume those files are sorted to avoid traversing here? (by what?) - for _, ds := range m.FileGroups { - if ds.MaxTs < from { - removed = append(removed, ds) - } else { - remainedDataFiles = append(remainedDataFiles, ds) +// RemoveDataFilesAndUpdateMetadataInBatch concurrently remove datafilegroups and update metadata. +// Only one metadata is processed in each thread, including deleting its datafilegroup and updating it. +// Returns the not deleted datafilegroups. +func (ms *StreamMetadataSet) RemoveDataFilesAndUpdateMetadataInBatch(ctx context.Context, from uint64, storage storage.ExternalStorage, updateFn func(num int64)) ([]string, error) { + var notDeleted struct { + item []string + sync.Mutex + } + worker := utils.NewWorkerPool(128, "delete files") + eg, cx := errgroup.WithContext(ctx) + for path, metaInfo := range ms.metadataInfos { + path := path + minTS := metaInfo.MinTS + // It's safety to remove the item within a range loop + delete(ms.metadataInfos, path) + if minTS >= from { + // That means all the datafiles wouldn't be removed, + // so that the metadata is skipped. + continue + } + worker.ApplyOnErrorGroup(eg, func() error { + if cx.Err() != nil { + return cx.Err() + } + + data, err := storage.ReadFile(ctx, path) + if err != nil { + return err + } + + meta, err := ms.Helper.ParseToMetadataHard(data) + if err != nil { + return err + } + + num, notDeletedItems, err := ms.removeDataFilesAndUpdateMetadata(ctx, storage, from, meta, path) + if err != nil { + return err + } + + updateFn(num) + + notDeleted.Lock() + notDeleted.item = append(notDeleted.item, notDeletedItems...) + notDeleted.Unlock() + return nil + }) + } + + if err := eg.Wait(); err != nil { + return nil, errors.Trace(err) + } + + return notDeleted.item, nil +} + +// removeDataFilesAndUpdateMetadata removes some datafilegroups of the metadata, if their max-ts is less than `from` +func (ms *StreamMetadataSet) removeDataFilesAndUpdateMetadata(ctx context.Context, storage storage.ExternalStorage, from uint64, meta *backuppb.Metadata, metaPath string) (num int64, notDeleted []string, err error) { + removed := make([]*backuppb.DataFileGroup, 0) + remainedDataFiles := make([]*backuppb.DataFileGroup, 0) + notDeleted = make([]string, 0) + // can we assume those files are sorted to avoid traversing here? (by what?) + for _, ds := range meta.FileGroups { + if ds.MaxTs < from { + removed = append(removed, ds) + } else { + // That means some kvs in the datafilegroup shouldn't be removed, + // so it will be kept out being removed. + remainedDataFiles = append(remainedDataFiles, ds) + } + } + + num = int64(len(removed)) + + if ms.DryRun { + log.Debug("dry run, skip deletion ...") + return num, notDeleted, nil + } + + // remove data file groups + for _, f := range removed { + log.Debug("Deleting file", zap.String("path", f.Path)) + if err := storage.DeleteFile(ctx, f.Path); err != nil { + log.Warn("File not deleted.", zap.String("path", f.Path), logutil.ShortError(err)) + notDeleted = append(notDeleted, f.Path) + if len(notDeleted) > notDeletedBecameFatalThreshold { + return num, notDeleted, errors.Annotatef(berrors.ErrPiTRMalformedMetadata, "too many failure when truncating") } } - if len(remainedDataFiles) != len(m.FileGroups) { - mCopy := *m - mCopy.FileGroups = remainedDataFiles - ms.WriteBack(metaPath, &mCopy) + } + + // update metadata + if len(remainedDataFiles) != len(meta.FileGroups) { + // rewrite metadata + log.Info("Updating metadata.", zap.String("file", metaPath), + zap.Int("data-file-before", len(meta.FileGroups)), + zap.Int("data-file-after", len(remainedDataFiles))) + + // replace the filegroups and update the ts of the replaced metadata + ReplaceMetadata(meta, remainedDataFiles) + + if ms.BeforeDoWriteBack != nil && ms.BeforeDoWriteBack(metaPath, meta) { + return num, notDeleted, nil + } + + if err := ms.doWriteBackForFile(ctx, storage, metaPath, meta); err != nil { + // NOTE: Maybe we'd better roll back all writebacks? (What will happen if roll back fails too?) + return num, notDeleted, errors.Annotatef(err, "failed to write back file %s", metaPath) } } - return removed -} -func (ms *StreamMetadataSet) WriteBack(path string, file *backuppb.Metadata) { - ms.writeback[path] = file + return num, notDeleted, nil } -func (ms *StreamMetadataSet) doWriteBackForFile(ctx context.Context, s storage.ExternalStorage, path string) error { - data, ok := ms.writeback[path] - if !ok { - return errors.Annotatef(berrors.ErrInvalidArgument, "There is no write back for path %s", path) - } +func (ms *StreamMetadataSet) doWriteBackForFile(ctx context.Context, s storage.ExternalStorage, path string, meta *backuppb.Metadata) error { // If the metadata file contains no data file, remove it due to it is meanless. - if len(data.FileGroups) == 0 { + if len(meta.FileGroups) == 0 { if err := s.DeleteFile(ctx, path); err != nil { return errors.Annotatef(err, "failed to remove the empty meta %s", path) } return nil } - bs, err := ms.Helper.Marshal(data) + bs, err := ms.Helper.Marshal(meta) if err != nil { return errors.Annotatef(err, "failed to marshal the file %s", path) } return truncateAndWrite(ctx, s, path, bs) } -// PendingMeta returns the length of metadata waiting for be written back. -func (ms *StreamMetadataSet) PendingMeta() int { - return len(ms.writeback) -} - -func (ms *StreamMetadataSet) DoWriteBack(ctx context.Context, s storage.ExternalStorage) error { - for path := range ms.writeback { - if ms.BeforeDoWriteBack != nil && ms.BeforeDoWriteBack(path, ms.metadata[path], ms.writeback[path]) { - continue - } - err := ms.doWriteBackForFile(ctx, s, path) - // NOTE: Maybe we'd better roll back all writebacks? (What will happen if roll back fails too?) - if err != nil { - return errors.Annotatef(err, "failed to write back file %s", path) - } - - delete(ms.writeback, path) - } - return nil -} - func truncateAndWrite(ctx context.Context, s storage.ExternalStorage, path string, data []byte) error { // Performance hack: the `Write` implemention would truncate the file if it exists. if err := s.WriteFile(ctx, path, data); err != nil { @@ -258,26 +354,30 @@ func UpdateShiftTS(m *backuppb.Metadata, startTS uint64, restoreTS uint64) (uint return minBeginTS, isExist } -// CalculateShiftTS gets the minimal begin-ts about transaction according to the kv-event in write-cf. -func CalculateShiftTS( - metas []*backuppb.Metadata, - startTS uint64, - restoreTS uint64, -) (uint64, bool) { - var ( - minBeginTS uint64 - isExist bool - ) - for _, m := range metas { - if len(m.FileGroups) == 0 || m.MinTs > restoreTS || m.MaxTs < startTS { - continue +// replace the filegroups and update the ts of the replaced metadata +func ReplaceMetadata(meta *backuppb.Metadata, filegroups []*backuppb.DataFileGroup) { + // replace the origin metadata + meta.FileGroups = filegroups + + if len(meta.FileGroups) == 0 { + meta.MinTs = 0 + meta.MaxTs = 0 + meta.ResolvedTs = 0 + return + } + + meta.MinTs = meta.FileGroups[0].MinTs + meta.MaxTs = meta.FileGroups[0].MaxTs + meta.ResolvedTs = meta.FileGroups[0].MinResolvedTs + for _, group := range meta.FileGroups { + if group.MinTs < meta.MinTs { + meta.MinTs = group.MinTs + } + if group.MaxTs > meta.MaxTs { + meta.MaxTs = group.MaxTs } - ts, ok := UpdateShiftTS(m, startTS, restoreTS) - if ok && (!isExist || ts < minBeginTS) { - minBeginTS = ts - isExist = true + if group.MinResolvedTs < meta.ResolvedTs { + meta.ResolvedTs = group.MinResolvedTs } } - - return minBeginTS, isExist } diff --git a/br/pkg/restore/stream_metas_test.go b/br/pkg/restore/stream_metas_test.go index 8e75f7544885e..5b75e9de6b3d8 100644 --- a/br/pkg/restore/stream_metas_test.go +++ b/br/pkg/restore/stream_metas_test.go @@ -6,7 +6,10 @@ import ( "context" "fmt" "math/rand" + "os" + "path" "path/filepath" + "sync" "testing" "github.com/fsouza/fake-gcs-server/fakestorage" @@ -16,7 +19,6 @@ import ( "github.com/pingcap/tidb/br/pkg/restore" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/br/pkg/stream" - "github.com/pingcap/tidb/util/mathutil" "github.com/stretchr/testify/require" "go.uber.org/zap" ) @@ -36,13 +38,59 @@ func fakeDataFiles(s storage.ExternalStorage, base, item int) (result []*backupp return } +func fakeDataFilesV2(s storage.ExternalStorage, base, item int) (result []*backuppb.DataFileGroup) { + ctx := context.Background() + for i := base; i < base+item; i++ { + path := fmt.Sprintf("%04d_to_%04d.log", i, i+2) + s.WriteFile(ctx, path, []byte("test")) + data := &backuppb.DataFileGroup{ + Path: path, + MinTs: uint64(i), + MaxTs: uint64(i + 2), + } + result = append(result, data) + } + return +} + +func tsOfFile(dfs []*backuppb.DataFileInfo) (uint64, uint64) { + var minTS uint64 = 9876543210 + var maxTS uint64 = 0 + for _, df := range dfs { + if df.MaxTs > maxTS { + maxTS = df.MaxTs + } + if df.MinTs < minTS { + minTS = df.MinTs + } + } + return minTS, maxTS +} + +func tsOfFileGroup(dfs []*backuppb.DataFileGroup) (uint64, uint64) { + var minTS uint64 = 9876543210 + var maxTS uint64 = 0 + for _, df := range dfs { + if df.MaxTs > maxTS { + maxTS = df.MaxTs + } + if df.MinTs < minTS { + minTS = df.MinTs + } + } + return minTS, maxTS +} + func fakeStreamBackup(s storage.ExternalStorage) error { ctx := context.Background() base := 0 for i := 0; i < 6; i++ { dfs := fakeDataFiles(s, base, 4) base += 4 + minTS, maxTS := tsOfFile(dfs) meta := &backuppb.Metadata{ + MinTs: minTS, + MaxTs: maxTS, Files: dfs, StoreId: int64(i%3 + 1), } @@ -64,43 +112,13 @@ func fakeStreamBackupV2(s storage.ExternalStorage) error { ctx := context.Background() base := 0 for i := 0; i < 6; i++ { - dfs := fakeDataFiles(s, base, 4) - minTs1 := uint64(18446744073709551615) - maxTs1 := uint64(0) - for _, f := range dfs[0:2] { - f.Path = fmt.Sprintf("%d", i) - if minTs1 > f.MinTs { - minTs1 = f.MinTs - } - if maxTs1 < f.MaxTs { - maxTs1 = f.MaxTs - } - } - minTs2 := uint64(18446744073709551615) - maxTs2 := uint64(0) - for _, f := range dfs[2:] { - f.Path = fmt.Sprintf("%d", i) - if minTs2 > f.MinTs { - minTs2 = f.MinTs - } - if maxTs2 < f.MaxTs { - maxTs2 = f.MaxTs - } - } + dfs := fakeDataFilesV2(s, base, 4) base += 4 + minTS, maxTS := tsOfFileGroup(dfs) meta := &backuppb.Metadata{ - FileGroups: []*backuppb.DataFileGroup{ - { - DataFilesInfo: dfs[0:2], - MinTs: minTs1, - MaxTs: maxTs1, - }, - { - DataFilesInfo: dfs[2:], - MinTs: minTs2, - MaxTs: maxTs2, - }, - }, + MinTs: minTS, + MaxTs: maxTS, + FileGroups: dfs, StoreId: int64(i%3 + 1), MetaVersion: backuppb.MetaVersion_V2, } @@ -135,42 +153,59 @@ func TestTruncateLog(t *testing.T) { } require.NoError(t, s.LoadFrom(ctx, l)) - fs := []*backuppb.DataFileGroup{} - s.IterateFilesFullyBefore(17, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + fs := []*restore.FileGroupInfo{} + s.IterateFilesFullyBefore(17, func(d *restore.FileGroupInfo) (shouldBreak bool) { fs = append(fs, d) - require.Less(t, d.MaxTs, uint64(17)) + require.Less(t, d.MaxTS, uint64(17)) return false }) require.Len(t, fs, 15) - s.RemoveDataBefore(17) - deletedFiles := []string{} - modifiedFiles := []string{} - s.BeforeDoWriteBack = func(path string, last, current *backuppb.Metadata) bool { - require.NotNil(t, last) - if len(current.GetFileGroups()) == 0 { - deletedFiles = append(deletedFiles, path) - } else if len(current.GetFileGroups()) != len(last.GetFileGroups()) { - modifiedFiles = append(modifiedFiles, path) + var lock sync.Mutex + remainedFiles := []string{} + remainedDataFiles := []string{} + removedMetaFiles := []string{} + s.BeforeDoWriteBack = func(path string, replaced *backuppb.Metadata) bool { + lock.Lock() + require.NotNil(t, replaced) + if len(replaced.GetFileGroups()) > 0 { + remainedFiles = append(remainedFiles, path) + for _, ds := range replaced.FileGroups { + remainedDataFiles = append(remainedDataFiles, ds.Path) + } + } else { + removedMetaFiles = append(removedMetaFiles, path) } + lock.Unlock() return false } - require.NoError(t, s.DoWriteBack(ctx, l)) - require.ElementsMatch(t, deletedFiles, []string{"v1/backupmeta/0000.meta", "v1/backupmeta/0001.meta", "v1/backupmeta/0002.meta"}) - require.ElementsMatch(t, modifiedFiles, []string{"v1/backupmeta/0003.meta"}) + + var total int64 = 0 + notDeleted, err := s.RemoveDataFilesAndUpdateMetadataInBatch(ctx, 17, l, func(num int64) { + lock.Lock() + total += num + lock.Unlock() + }) + require.NoError(t, err) + require.Equal(t, len(notDeleted), 0) + require.ElementsMatch(t, remainedFiles, []string{"v1/backupmeta/0003.meta"}) + require.ElementsMatch(t, removedMetaFiles, []string{"v1/backupmeta/0000.meta", "v1/backupmeta/0001.meta", "v1/backupmeta/0002.meta"}) + require.ElementsMatch(t, remainedDataFiles, []string{"0015_to_0017.log"}) + require.Equal(t, total, int64(15)) require.NoError(t, s.LoadFrom(ctx, l)) - s.IterateFilesFullyBefore(17, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + s.IterateFilesFullyBefore(17, func(d *restore.FileGroupInfo) (shouldBreak bool) { t.Errorf("some of log files still not truncated, it is %#v", d) return true }) - l.WalkDir(ctx, &storage.WalkOption{ + err = l.WalkDir(ctx, &storage.WalkOption{ SubDir: stream.GetStreamBackupMetaPrefix(), }, func(s string, i int64) error { - require.NotContains(t, deletedFiles, s) + require.NotContains(t, removedMetaFiles, s) return nil }) + require.NoError(t, err) } func TestTruncateLogV2(t *testing.T) { @@ -190,42 +225,59 @@ func TestTruncateLogV2(t *testing.T) { } require.NoError(t, s.LoadFrom(ctx, l)) - fs := []*backuppb.DataFileGroup{} - s.IterateFilesFullyBefore(17, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + fs := []*restore.FileGroupInfo{} + s.IterateFilesFullyBefore(17, func(d *restore.FileGroupInfo) (shouldBreak bool) { fs = append(fs, d) - require.Less(t, d.MaxTs, uint64(17)) + require.Less(t, d.MaxTS, uint64(17)) return false }) - require.Len(t, fs, 7) - - s.RemoveDataBefore(17) - deletedFiles := []string{} - modifiedFiles := []string{} - s.BeforeDoWriteBack = func(path string, last, current *backuppb.Metadata) bool { - require.NotNil(t, last) - if len(current.GetFileGroups()) == 0 { - deletedFiles = append(deletedFiles, path) - } else if len(current.GetFileGroups()) != len(last.GetFileGroups()) { - modifiedFiles = append(modifiedFiles, path) + require.Len(t, fs, 15) + + var lock sync.Mutex + remainedFiles := []string{} + remainedDataFiles := []string{} + removedMetaFiles := []string{} + s.BeforeDoWriteBack = func(path string, replaced *backuppb.Metadata) bool { + lock.Lock() + require.NotNil(t, replaced) + if len(replaced.GetFileGroups()) > 0 { + remainedFiles = append(remainedFiles, path) + for _, ds := range replaced.FileGroups { + remainedDataFiles = append(remainedDataFiles, ds.Path) + } + } else { + removedMetaFiles = append(removedMetaFiles, path) } + lock.Unlock() return false } - require.NoError(t, s.DoWriteBack(ctx, l)) - require.ElementsMatch(t, deletedFiles, []string{"v1/backupmeta/0000.meta", "v1/backupmeta/0001.meta", "v1/backupmeta/0002.meta"}) - require.ElementsMatch(t, modifiedFiles, []string{"v1/backupmeta/0003.meta"}) + + var total int64 = 0 + notDeleted, err := s.RemoveDataFilesAndUpdateMetadataInBatch(ctx, 17, l, func(num int64) { + lock.Lock() + total += num + lock.Unlock() + }) + require.NoError(t, err) + require.Equal(t, len(notDeleted), 0) + require.ElementsMatch(t, remainedFiles, []string{"v1/backupmeta/0003.meta"}) + require.ElementsMatch(t, removedMetaFiles, []string{"v1/backupmeta/0000.meta", "v1/backupmeta/0001.meta", "v1/backupmeta/0002.meta"}) + require.ElementsMatch(t, remainedDataFiles, []string{"0015_to_0017.log"}) + require.Equal(t, total, int64(15)) require.NoError(t, s.LoadFrom(ctx, l)) - s.IterateFilesFullyBefore(17, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + s.IterateFilesFullyBefore(17, func(d *restore.FileGroupInfo) (shouldBreak bool) { t.Errorf("some of log files still not truncated, it is %#v", d) return true }) - l.WalkDir(ctx, &storage.WalkOption{ + err = l.WalkDir(ctx, &storage.WalkOption{ SubDir: stream.GetStreamBackupMetaPrefix(), }, func(s string, i int64) error { - require.NotContains(t, deletedFiles, s) + require.NotContains(t, removedMetaFiles, s) return nil }) + require.NoError(t, err) } func TestTruncateSafepoint(t *testing.T) { @@ -425,52 +477,1835 @@ func fakeMetaDataV2s(t *testing.T, helper *stream.MetadataHelper, cf string) []* return m2s } +func ff(minTS, maxTS uint64) *backuppb.DataFileGroup { + return f(0, minTS, maxTS, stream.DefaultCF, 0) +} + +func TestReplaceMetadataTs(t *testing.T) { + m := &backuppb.Metadata{} + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 3), + ff(4, 5), + }) + require.Equal(t, m.MinTs, uint64(1)) + require.Equal(t, m.MaxTs, uint64(5)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 4), + ff(3, 5), + }) + require.Equal(t, m.MinTs, uint64(1)) + require.Equal(t, m.MaxTs, uint64(5)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 6), + ff(0, 5), + }) + require.Equal(t, m.MinTs, uint64(0)) + require.Equal(t, m.MaxTs, uint64(6)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 3), + }) + require.Equal(t, m.MinTs, uint64(1)) + require.Equal(t, m.MaxTs, uint64(3)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{}) + require.Equal(t, m.MinTs, uint64(0)) + require.Equal(t, m.MaxTs, uint64(0)) + + restore.ReplaceMetadata(m, []*backuppb.DataFileGroup{ + ff(1, 3), + ff(2, 4), + ff(0, 2), + }) + require.Equal(t, m.MinTs, uint64(0)) + require.Equal(t, m.MaxTs, uint64(4)) +} + +func m(storeId int64, minTS, maxTS uint64) *backuppb.Metadata { + return &backuppb.Metadata{ + StoreId: storeId, + MinTs: minTS, + MaxTs: maxTS, + MetaVersion: backuppb.MetaVersion_V2, + } +} + +func f(storeId int64, minTS, maxTS uint64, cf string, defaultTS uint64) *backuppb.DataFileGroup { + return &backuppb.DataFileGroup{ + Path: logName(storeId, minTS, maxTS), + DataFilesInfo: []*backuppb.DataFileInfo{ + { + NumberOfEntries: 1, + MinTs: minTS, + MaxTs: maxTS, + Cf: cf, + MinBeginTsInDefaultCf: defaultTS, + }, + }, + MinTs: minTS, + MaxTs: maxTS, + } +} + +// get the metadata with only one datafilegroup +func m_1(storeId int64, minTS, maxTS uint64, cf string, defaultTS uint64) *backuppb.Metadata { + meta := m(storeId, minTS, maxTS) + meta.FileGroups = []*backuppb.DataFileGroup{ + f(storeId, minTS, maxTS, cf, defaultTS), + } + return meta +} + +// get the metadata with 2 datafilegroup +func m_2( + storeId int64, + minTSL, maxTSL uint64, cfL string, defaultTSL uint64, + minTSR, maxTSR uint64, cfR string, defaultTSR uint64, +) *backuppb.Metadata { + meta := m(storeId, minTSL, maxTSR) + meta.FileGroups = []*backuppb.DataFileGroup{ + f(storeId, minTSL, maxTSL, cfL, defaultTSL), + f(storeId, minTSR, maxTSR, cfR, defaultTSR), + } + return meta +} + +// clean the files in the external storage +func cleanFiles(ctx context.Context, s storage.ExternalStorage) error { + names := make([]string, 0) + err := s.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { + names = append(names, path) + return nil + }) + if err != nil { + return err + } + for _, path := range names { + err := s.DeleteFile(ctx, path) + if err != nil { + return err + } + } + return nil +} + +func metaName(storeId int64) string { + return fmt.Sprintf("%s/%04d.meta", stream.GetStreamBackupMetaPrefix(), storeId) +} + +func logName(storeId int64, minTS, maxTS uint64) string { + return fmt.Sprintf("%04d_%04d_%04d.log", storeId, minTS, maxTS) +} + +// generate the files to the external storage +func generateFiles(ctx context.Context, s storage.ExternalStorage, metas []*backuppb.Metadata, tmpDir string) error { + if err := cleanFiles(ctx, s); err != nil { + return err + } + fname := path.Join(tmpDir, stream.GetStreamBackupMetaPrefix()) + os.MkdirAll(fname, 0777) + for _, meta := range metas { + data, err := meta.Marshal() + if err != nil { + return err + } + + fname := metaName(meta.StoreId) + err = s.WriteFile(ctx, fname, data) + if err != nil { + return err + } + + for _, group := range meta.FileGroups { + fname := logName(meta.StoreId, group.MinTs, group.MaxTs) + err = s.WriteFile(ctx, fname, []byte("test")) + if err != nil { + return err + } + } + } + + return nil +} + +// check the files in the external storage +func checkFiles(ctx context.Context, s storage.ExternalStorage, metas []*backuppb.Metadata, t *testing.T) { + pathSet := make(map[string]struct{}) + for _, meta := range metas { + metaPath := metaName(meta.StoreId) + pathSet[metaPath] = struct{}{} + exists, err := s.FileExists(ctx, metaPath) + require.NoError(t, err) + require.True(t, exists) + + data, err := s.ReadFile(ctx, metaPath) + require.NoError(t, err) + metaRead := &backuppb.Metadata{} + err = metaRead.Unmarshal(data) + require.NoError(t, err) + require.Equal(t, meta.MinTs, metaRead.MinTs) + require.Equal(t, meta.MaxTs, metaRead.MaxTs) + for i, group := range meta.FileGroups { + require.Equal(t, metaRead.FileGroups[i].Path, group.Path) + logPath := logName(meta.StoreId, group.MinTs, group.MaxTs) + pathSet[logPath] = struct{}{} + exists, err := s.FileExists(ctx, logPath) + require.NoError(t, err) + require.True(t, exists) + } + } + + err := s.WalkDir(ctx, &storage.WalkOption{}, func(path string, size int64) error { + _, exists := pathSet[path] + require.True(t, exists, path) + return nil + }) + require.NoError(t, err) +} + +type testParam struct { + until []uint64 + shiftUntilTS uint64 + restMetadata []*backuppb.Metadata +} + +func TestTruncate1(t *testing.T) { + ctx := context.Background() + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + + cases := []struct { + metas []*backuppb.Metadata + testParams []*testParam + }{ + { + // metadata 10-----------20 + // ↑ ↑ + // +-----------+ + // ↓ ↓ + // filegroup 10-----d-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + testParams: []*testParam{ + { + until: []uint64{5}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + }, { + until: []uint64{10}, + shiftUntilTS: 10, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + }, { + until: []uint64{15}, + shiftUntilTS: 15, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + }, { + until: []uint64{20}, + shiftUntilTS: 20, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // ↑ ↑ + // +-----------+ + // ↓ ↓ + // filegroup 5-d--10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 7, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 5----8 10-----------20 + // ↑ ↑ ↑ ↑ + // +----+ +-----------+ + // ↓ ↓ ↓ ↓ + // filegroup 5--d-8 ↓ ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 8, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 8, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 9, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 8, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 5------10 ↑ + // ↑ ↑ ↑ + // +-------+-----------+ + // ↓ ↓ ↓ + // filegroup 5--d---10 ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 10, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 10, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 9, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 10, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 5-------↑-12 ↑ + // ↑ ↑ ↑ ↑ + // +-------+-+---------+ + // ↓ ↓ ↓ ↓ + // filegroup 5--d----↓-12 ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 12, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 12, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 9, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 12, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 5-------↑-----------20 + // ↑ ↑ ↑ + // +-------+-----------+ + // ↓ ↓ ↓ + // filegroup 5--d----↓-----------20 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 5-------↑-----------↑--22 + // ↑ ↑ ↑ ↑ + // +-------+-----------+--+ + // ↓ ↓ ↓ ↓ + // filegroup 5--d----↓-----------↓--22 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 15, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 5, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 10---14 ↑ + // ↑ ↑ ↑ + // +----+-------+ + // ↓ ↓ ↓ + // filegroup 10-d-14 ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 14, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 14, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 12, 14, 18, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 14, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 10-----------20 + // ↑ ↑ + // +------------+ + // ↓ ↓ + // filegroup 10----d------20 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata 10------------↑--22 + // ↑ ↑ ↑ + // +------------+---+ + // ↓ ↓ ↓ + // filegroup 10----d-------↓--22 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 10, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ 12-----18 ↑ + // ↑ ↑ ↑ ↑ + // +--+------+--+ + // ↓ ↓ ↓ ↓ + // filegroup ↓ 12--d--18 ↓ + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 12, 18, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 12, 18, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 11, 12, 15, 18, 19, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 12, 18, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ 14----20 + // ↑ ↑ ↑ + // +------+-----+ + // ↓ ↓ ↓ + // filegroup ↓ 14--d-20 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 14, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 20, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ 14-----↑--22 + // ↑ ↑ ↑ ↑ + // +------+-----+---+ + // ↓ ↓ ↓ ↓ + // filegroup ↓ 14-d--↓--22 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 14, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ 20--22 + // ↑ ↑ ↑ + // +------------+---+ + // ↓ ↓ ↓ + // filegroup ↓ 20--22 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 20, 22, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 10-----------20 + // metadata ↑ ↑ 21---24 + // ↑ ↑ ↑ ↑ + // +------------+--+----+ + // ↓ ↓ ↓ ↓ + // filegroup ↓ ↓ 21-d-24 + // filegroup 5--d---10-----w-----20 + metas: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + testParams: []*testParam{ + { + until: []uint64{3}, + shiftUntilTS: 3, restMetadata: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{5, 8, 10, 14, 20}, + shiftUntilTS: 5, restMetadata: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + m_1(2, 10, 20, stream.WriteCF, 5), + }, + }, { + until: []uint64{21}, + shiftUntilTS: 21, restMetadata: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + }, + }, { + until: []uint64{22}, + shiftUntilTS: 22, restMetadata: []*backuppb.Metadata{ + m_1(1, 21, 24, stream.DefaultCF, 0), + }, + }, { + until: []uint64{25}, + shiftUntilTS: 25, restMetadata: []*backuppb.Metadata{}, + }, + }, + }, + } + + for i, cs := range cases { + for j, ts := range cs.testParams { + for _, until := range ts.until { + t.Logf("case %d, param %d, until %d", i, j, until) + metas := restore.StreamMetadataSet{ + Helper: stream.NewMetadataHelper(), + } + err := generateFiles(ctx, s, cs.metas, tmpDir) + require.NoError(t, err) + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, s, until) + require.NoError(t, err) + require.Equal(t, shiftUntilTS, ts.shiftUntilTS) + n, err := metas.RemoveDataFilesAndUpdateMetadataInBatch(ctx, shiftUntilTS, s, func(num int64) {}) + require.Equal(t, len(n), 0) + require.NoError(t, err) + + // check the result + checkFiles(ctx, s, ts.restMetadata, t) + } + } + } +} + +type testParam2 struct { + until []uint64 + shiftUntilTS func(uint64) uint64 + restMetadata []*backuppb.Metadata +} + +func returnV(v uint64) func(uint64) uint64 { + return func(uint64) uint64 { + return v + } +} + +func returnSelf() func(uint64) uint64 { + return func(u uint64) uint64 { + return u + } +} + +func TestTruncate2(t *testing.T) { + ctx := context.Background() + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + + cases := []struct { + metas []*backuppb.Metadata + testParams []*testParam2 + }{ + { + // metadata 10-----------20 + // ↑ ↑ + // +-----------+ + // ↓ ↓ ↓ ↓ + // filegroup 10-d-13 ↓ ↓ + // filegroup 8----d--15-w-20 + metas: []*backuppb.Metadata{ + m_2(1, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{5}, + shiftUntilTS: returnV(5), restMetadata: []*backuppb.Metadata{ + m_2(1, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{8, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(8), restMetadata: []*backuppb.Metadata{ + m_2(1, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +-----------+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3 6 10-d-13 ↓ ↓ + // filegroup 1-----------d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 1, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{0}, + shiftUntilTS: returnV(0), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{1, 2, 3, 4, 6, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(1), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +-----------+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3 6 10-d-13 ↓ ↓ + // filegroup 3----------d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 3, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2}, + shiftUntilTS: returnV(2), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 3, + ), + }, + }, { + until: []uint64{3, 4, 6, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(3), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 3, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---7 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3 7 10-d-13 ↓ ↓ + // filegroup 5--------d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 5, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{5, 6, 7, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(5), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---7 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3 7 10-d-13 ↓ ↓ + // filegroup 7------d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 7, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6, 7}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 7, + ), + }, + }, { + until: []uint64{9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(7), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 7, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 7, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-13 ↓ ↓ + // filegroup 8----d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{7}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{8, 9, 10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(8), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-13 ↓ ↓ + // filegroup 10--d--15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 10, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 10, + ), + }, + }, { + until: []uint64{7, 8, 9}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 10, + ), + }, + }, { + until: []uint64{10, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(10), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 10, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 9-d-13 ↓ ↓ + // filegroup 11-d-15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 9, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 11, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 9, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 11, + ), + }, + }, { + until: []uint64{7, 8, 9, 10}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 9, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 11, + ), + }, + }, { + until: []uint64{11, 12, 13, 14, 15, 18, 20}, + shiftUntilTS: returnV(11), restMetadata: []*backuppb.Metadata{ + m_2(2, + 9, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 11, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+-+----+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-13 ↓ ↓ + // filegroup 13d15-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 13, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 13, + ), + }, + }, { + until: []uint64{7, 8, 9, 10, 12}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 13, + ), + }, + }, { + until: []uint64{13, 14, 15, 18, 20}, + shiftUntilTS: returnV(13), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 13, stream.DefaultCF, 0, + 15, 20, stream.WriteCF, 13, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-12 ↓ ↓ + // filegroup 14d16-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{7, 8, 9, 10, 11, 12}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{13}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(2, 16, 20, stream.WriteCF, 14), + }, + }, { + until: []uint64{14, 15, 18, 20}, + shiftUntilTS: returnV(14), restMetadata: []*backuppb.Metadata{ + m_1(2, 16, 20, stream.WriteCF, 14), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 3---6 10----------20 + // ↑ ↑ ↑ ↑ + // +---+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3-d-6 10-d-12 ↓ ↓ + // filegroup 14d16-w-20 + metas: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2, 3, 4, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, 3, 6, stream.DefaultCF, 0), + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{7, 8, 9, 10, 11, 12}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(2, + 10, 12, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{13}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(2, 16, 20, stream.WriteCF, 14), + }, + }, { + until: []uint64{14, 15, 18, 20}, + shiftUntilTS: returnV(14), restMetadata: []*backuppb.Metadata{ + m_1(2, 16, 20, stream.WriteCF, 14), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, + } + + for i, cs := range cases { + for j, ts := range cs.testParams { + for _, until := range ts.until { + t.Logf("case %d, param %d, until %d", i, j, until) + metas := restore.StreamMetadataSet{ + Helper: stream.NewMetadataHelper(), + } + err := generateFiles(ctx, s, cs.metas, tmpDir) + require.NoError(t, err) + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, s, until) + require.NoError(t, err) + require.Equal(t, shiftUntilTS, ts.shiftUntilTS(until)) + n, err := metas.RemoveDataFilesAndUpdateMetadataInBatch(ctx, shiftUntilTS, s, func(num int64) {}) + require.Equal(t, len(n), 0) + require.NoError(t, err) + + // check the result + checkFiles(ctx, s, ts.restMetadata, t) + } + } + } +} + +func TestTruncate3(t *testing.T) { + ctx := context.Background() + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + + cases := []struct { + metas []*backuppb.Metadata + testParams []*testParam2 + }{ + { + // metadata 3------10 12----------20 + // ↑ ↑ ↑ ↑ + // +-+--+--+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 3--d-7 ↓ ↓ ↓ ↓ ↓ + // filegroup 5--d-10 ↓ ↓ ↓ ↓ + // filegroup 3----d-----12---w--18 ↓ + // filegroup 5----d--------15--w--20 + metas: []*backuppb.Metadata{ + m_2(1, + 3, 7, stream.DefaultCF, 0, + 5, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 3, + 15, 20, stream.WriteCF, 5, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{2}, + shiftUntilTS: returnV(2), restMetadata: []*backuppb.Metadata{ + m_2(1, + 3, 7, stream.DefaultCF, 0, + 5, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 3, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{3, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14, 15, 16, 18}, + shiftUntilTS: returnV(3), restMetadata: []*backuppb.Metadata{ + m_2(1, + 3, 7, stream.DefaultCF, 0, + 5, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 3, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{19, 20}, + shiftUntilTS: returnV(5), restMetadata: []*backuppb.Metadata{ + m_2(1, + 3, 7, stream.DefaultCF, 0, + 5, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 3, + 15, 20, stream.WriteCF, 5, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 2------10 12----------20 + // ↑ ↑ ↑ ↑ + // +-+--+--+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 2--d-6 ↓ ↓ ↓ ↓ ↓ + // filegroup 4--d-10 ↓ ↓ ↓ ↓ + // filegroup 2----d-----12---w--18 ↓ + // filegroup 8---d----15--w--20 + metas: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 2, + 15, 20, stream.WriteCF, 8, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{1}, + shiftUntilTS: returnV(1), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 2, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{2, 3, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14, 15, 16, 18}, + shiftUntilTS: returnV(2), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 2, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{19, 20}, + shiftUntilTS: returnV(8), restMetadata: []*backuppb.Metadata{ + m_1(1, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 12, 18, stream.WriteCF, 2, + 15, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 2------10 14----------20 + // ↑ ↑ ↑ ↑ + // +-+--+--+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 2--d-6 ↓ ↓ ↓ ↓ ↓ + // filegroup 4--d-10 ↓ ↓ ↓ ↓ + // filegroup 2----d-------14---w--18 ↓ + // filegroup 12---d--16--w--20 + metas: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 2, + 16, 20, stream.WriteCF, 12, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{1}, + shiftUntilTS: returnV(1), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 2, + 16, 20, stream.WriteCF, 12, + ), + }, + }, { + until: []uint64{2, 3, 4, 5, 6, 7, 8, 10, 11, 12, 13, 14, 15, 16, 18}, + shiftUntilTS: returnV(2), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 4, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 2, + 16, 20, stream.WriteCF, 12, + ), + }, + }, { + until: []uint64{19, 20}, + shiftUntilTS: returnV(12), restMetadata: []*backuppb.Metadata{ + m_2(2, + 14, 18, stream.WriteCF, 2, + 16, 20, stream.WriteCF, 8, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 2-------10 14----------20 + // ↑ ↑ ↑ ↑ + // +-+--+---+ +----+--+---+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 2--d-6 ↓ ↓ ↓ ↓ ↓ + // filegroup 4-d-8w10 ↓ ↓ ↓ ↓ + // filegroup 14--d---18 ↓ + // filegroup 14-d--16-w--20 + metas: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.WriteCF, 4, + ), + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{1}, + shiftUntilTS: returnV(1), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.WriteCF, 4, + ), + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{2, 3}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.WriteCF, 4, + ), + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{4, 5, 6, 7, 8, 9, 10}, + shiftUntilTS: returnV(4), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.WriteCF, 4, + ), + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{12}, + shiftUntilTS: returnV(12), restMetadata: []*backuppb.Metadata{ + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{14, 15, 16, 17, 18, 19, 20}, + shiftUntilTS: returnV(14), restMetadata: []*backuppb.Metadata{ + m_2(2, + 14, 18, stream.DefaultCF, 0, + 16, 20, stream.WriteCF, 14, + ), + }, + }, { + until: []uint64{25}, + shiftUntilTS: returnV(25), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, { + // metadata 2-------10 14----------22 24-w-26 + // ↑ ↑ ↑ ↑ ↑ ↑ + // +-+--+---+ +----+--+---+ +----+ + // ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 2--d-6 ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 8d10 ↓ ↓ ↓ ↓ ↓ ↓ + // filegroup 9--d--14--w---18 ↓ ↓ ↓ + // filegroup 16-d--22 ↓ ↓ + // filegroup 20---d-24-w-26 + metas: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 9, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + testParams: []*testParam2{ + { + until: []uint64{1, 2, 3, 6}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_2(1, + 2, 6, stream.DefaultCF, 0, + 8, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 9, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{7, 8}, + shiftUntilTS: returnSelf(), restMetadata: []*backuppb.Metadata{ + m_1(1, + 8, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 9, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{9, 10, 11, 14, 15, 16, 17, 18}, + shiftUntilTS: returnV(9), restMetadata: []*backuppb.Metadata{ + m_1(1, + 8, 10, stream.DefaultCF, 0, + ), + m_2(2, + 14, 18, stream.WriteCF, 9, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{19}, + shiftUntilTS: returnV(19), restMetadata: []*backuppb.Metadata{ + m_1(2, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{20, 21, 22, 23, 24, 25, 26}, + shiftUntilTS: returnV(20), restMetadata: []*backuppb.Metadata{ + m_1(2, + 16, 22, stream.DefaultCF, 0, + ), + m_1(3, + 24, 26, stream.WriteCF, 20, + ), + }, + }, { + until: []uint64{28}, + shiftUntilTS: returnV(28), restMetadata: []*backuppb.Metadata{}, + }, + }, + }, + } + + for i, cs := range cases { + for j, ts := range cs.testParams { + for _, until := range ts.until { + t.Logf("case %d, param %d, until %d", i, j, until) + metas := restore.StreamMetadataSet{ + Helper: stream.NewMetadataHelper(), + } + err := generateFiles(ctx, s, cs.metas, tmpDir) + require.NoError(t, err) + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, s, until) + require.NoError(t, err) + require.Equal(t, shiftUntilTS, ts.shiftUntilTS(until)) + n, err := metas.RemoveDataFilesAndUpdateMetadataInBatch(ctx, shiftUntilTS, s, func(num int64) {}) + require.Equal(t, len(n), 0) + require.NoError(t, err) + + // check the result + checkFiles(ctx, s, ts.restMetadata, t) + } + } + } +} + +type testParam3 struct { + until []uint64 + shiftUntilTS func(uint64) uint64 +} + +func fi(minTS, maxTS uint64, cf string, defaultTS uint64) *backuppb.DataFileInfo { + return &backuppb.DataFileInfo{ + NumberOfEntries: 1, + MinTs: minTS, + MaxTs: maxTS, + Cf: cf, + MinBeginTsInDefaultCf: defaultTS, + } +} + +func getTsFromFiles(files []*backuppb.DataFileInfo) (uint64, uint64, uint64) { + if len(files) == 0 { + return 0, 0, 0 + } + f := files[0] + minTs, maxTs, resolvedTs := f.MinTs, f.MaxTs, f.ResolvedTs + for _, file := range files { + if file.MinTs < minTs { + minTs = file.MinTs + } + if file.MaxTs > maxTs { + maxTs = file.MaxTs + } + if file.ResolvedTs < resolvedTs { + resolvedTs = file.ResolvedTs + } + } + return minTs, maxTs, resolvedTs +} + +func mf(id int64, filess [][]*backuppb.DataFileInfo) *backuppb.Metadata { + filegroups := make([]*backuppb.DataFileGroup, 0) + for _, files := range filess { + minTs, maxTs, resolvedTs := getTsFromFiles(files) + filegroups = append(filegroups, &backuppb.DataFileGroup{ + DataFilesInfo: files, + MinTs: minTs, + MaxTs: maxTs, + MinResolvedTs: resolvedTs, + }) + } + + m := &backuppb.Metadata{ + StoreId: id, + MetaVersion: backuppb.MetaVersion_V2, + } + restore.ReplaceMetadata(m, filegroups) + return m +} + func TestCalculateShiftTS(t *testing.T) { - var ( - startTs uint64 = 2900 - restoreTS uint64 = 4500 - ) - - helper := stream.NewMetadataHelper() - ms := fakeMetaDatas(t, helper, stream.WriteCF) - shiftTS, exist := restore.CalculateShiftTS(ms, startTs, restoreTS) - require.Equal(t, shiftTS, uint64(2000)) - require.Equal(t, exist, true) - - shiftTS, exist = restore.CalculateShiftTS(ms, startTs, mathutil.MaxUint) - require.Equal(t, shiftTS, uint64(1800)) - require.Equal(t, exist, true) - - shiftTS, exist = restore.CalculateShiftTS(ms, 1999, 3001) - require.Equal(t, shiftTS, uint64(800)) - require.Equal(t, exist, true) - - ms = fakeMetaDatas(t, helper, stream.DefaultCF) - _, exist = restore.CalculateShiftTS(ms, startTs, restoreTS) - require.Equal(t, exist, false) -} - -func TestCalculateShiftTSV2(t *testing.T) { - var ( - startTs uint64 = 2900 - restoreTS uint64 = 5100 - ) - - helper := stream.NewMetadataHelper() - ms := fakeMetaDataV2s(t, helper, stream.WriteCF) - shiftTS, exist := restore.CalculateShiftTS(ms, startTs, restoreTS) - require.Equal(t, shiftTS, uint64(1800)) - require.Equal(t, exist, true) - - shiftTS, exist = restore.CalculateShiftTS(ms, startTs, mathutil.MaxUint) - require.Equal(t, shiftTS, uint64(1700)) - require.Equal(t, exist, true) - - shiftTS, exist = restore.CalculateShiftTS(ms, 1999, 3001) - require.Equal(t, shiftTS, uint64(800)) - require.Equal(t, exist, true) - - ms = fakeMetaDataV2s(t, helper, stream.DefaultCF) - _, exist = restore.CalculateShiftTS(ms, startTs, restoreTS) - require.Equal(t, exist, false) + ctx := context.Background() + tmpDir := t.TempDir() + s, err := storage.NewLocalStorage(tmpDir) + require.NoError(t, err) + + cases := []struct { + metas []*backuppb.Metadata + testParams []*testParam3 + }{ + { + // filegroup 10 35 + // ↑ ↑ + // +----+-++---+ + // ↓ ↓ ↓↓ ↓ + // fileinfo 10-d-20 + // fileinfo 8--d-15--w-30 + // fileinfo 11-d-25-w-35 + metas: []*backuppb.Metadata{ + mf(1, [][]*backuppb.DataFileInfo{ + { + fi(10, 20, stream.DefaultCF, 0), + fi(15, 30, stream.WriteCF, 8), + fi(25, 35, stream.WriteCF, 11), + }, + }), + }, + testParams: []*testParam3{ + { + until: []uint64{3}, + shiftUntilTS: returnV(3), + }, { + until: []uint64{8, 9, 10, 11, 12, 15, 16, 20, 21, 25, 26, 30}, + shiftUntilTS: returnV(8), + }, { + until: []uint64{31, 35}, + shiftUntilTS: returnV(11), + }, { + until: []uint64{36}, + shiftUntilTS: returnV(36), + }, + }, + }, { + // filegroup 50 85 + // ↑ ↑ + // +-+-+--+--+------+ + // ↓ ↓ ↓ ↓ ↓ ↓ + // fileinfo 55-d-65-70 + // fileinfo 50-d60 + // fileinfo 72d80w85 + metas: []*backuppb.Metadata{ + mf(1, [][]*backuppb.DataFileInfo{ + { + fi(65, 70, stream.WriteCF, 55), + fi(50, 60, stream.DefaultCF, 0), + fi(80, 85, stream.WriteCF, 72), + }, + }), + }, + testParams: []*testParam3{ + { + until: []uint64{45, 50, 52}, + shiftUntilTS: returnSelf(), + }, { + until: []uint64{55, 56, 60, 61, 65, 66, 70}, + shiftUntilTS: returnV(55), + }, { + until: []uint64{71}, + shiftUntilTS: returnV(71), + }, { + until: []uint64{72, 73, 80, 81, 85}, + shiftUntilTS: returnV(72), + }, { + until: []uint64{86}, + shiftUntilTS: returnV(86), + }, + }, + }, { + // filegroup 10 35 50 85 + // ↑ ↑ ↑ ↑ + // +----+-++---+ +-+-+--+--+------+ + // ↓ ↓ ↓↓ ↓ ↓ ↓ ↓ ↓ ↓ ↓ + // fileinfo 10-d-20 55-d-65-70 + // fileinfo 8--d-15--w-30 50-d60 + // fileinfo 11-d-25-w-35 72d80w85 + metas: []*backuppb.Metadata{ + mf(1, [][]*backuppb.DataFileInfo{ + { + fi(10, 20, stream.DefaultCF, 0), + fi(15, 30, stream.WriteCF, 8), + fi(25, 35, stream.WriteCF, 11), + }, + }), + mf(2, [][]*backuppb.DataFileInfo{ + { + fi(65, 70, stream.WriteCF, 55), + fi(50, 60, stream.DefaultCF, 0), + fi(80, 85, stream.WriteCF, 72), + }, + }), + }, + testParams: []*testParam3{ + { + until: []uint64{3}, + shiftUntilTS: returnV(3), + }, { + until: []uint64{8, 9, 10, 11, 12, 15, 16, 20, 21, 25, 26, 30}, + shiftUntilTS: returnV(8), + }, { + until: []uint64{31, 35}, + shiftUntilTS: returnV(11), + }, { + until: []uint64{36}, + shiftUntilTS: returnV(36), + }, { + until: []uint64{45, 50, 52}, + shiftUntilTS: returnSelf(), + }, { + until: []uint64{55, 56, 60, 61, 65, 66, 70}, + shiftUntilTS: returnV(55), + }, { + until: []uint64{71}, + shiftUntilTS: returnV(71), + }, { + until: []uint64{72, 73, 80, 81, 85}, + shiftUntilTS: returnV(72), + }, { + until: []uint64{86}, + shiftUntilTS: returnV(86), + }, + }, + }, + } + + for i, cs := range cases { + for j, ts := range cs.testParams { + for _, until := range ts.until { + t.Logf("case %d, param %d, until %d", i, j, until) + metas := restore.StreamMetadataSet{ + Helper: stream.NewMetadataHelper(), + } + err := generateFiles(ctx, s, cs.metas, tmpDir) + require.NoError(t, err) + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, s, until) + require.NoError(t, err) + require.Equal(t, shiftUntilTS, ts.shiftUntilTS(until), cs.metas) + } + } + } } diff --git a/br/pkg/task/stream.go b/br/pkg/task/stream.go index ff4a1c449e181..a259452b14b2d 100644 --- a/br/pkg/task/stream.go +++ b/br/pkg/task/stream.go @@ -57,7 +57,6 @@ import ( "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/exp/slices" - "golang.org/x/sync/errgroup" ) const ( @@ -68,8 +67,6 @@ const ( flagStreamStartTS = "start-ts" flagStreamEndTS = "end-ts" flagGCSafePointTTS = "gc-ttl" - - notDeletedBecameFatalThreshold = 128 ) var ( @@ -937,31 +934,24 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre readMetaDone := console.ShowTask("Reading Metadata... ", glue.WithTimeCost()) metas := restore.StreamMetadataSet{ Helper: stream.NewMetadataHelper(), - BeforeDoWriteBack: func(path string, last, current *backuppb.Metadata) (skip bool) { - log.Info("Updating metadata.", zap.String("file", path), - zap.Int("data-file-before", len(last.GetFileGroups())), - zap.Int("data-file-after", len(current.GetFileGroups()))) - return cfg.DryRun - }, + DryRun: cfg.DryRun, } - if err := metas.LoadUntil(ctx, storage, cfg.Until); err != nil { + shiftUntilTS, err := metas.LoadUntilAndCalculateShiftTS(ctx, storage, cfg.Until) + if err != nil { return err } readMetaDone() var ( - fileCount uint64 = 0 - kvCount int64 = 0 - totalSize uint64 = 0 - shiftUntilTS = metas.CalculateShiftTS(cfg.Until) + fileCount int = 0 + kvCount int64 = 0 + totalSize uint64 = 0 ) - metas.IterateFilesFullyBefore(shiftUntilTS, func(d *backuppb.DataFileGroup) (shouldBreak bool) { + metas.IterateFilesFullyBefore(shiftUntilTS, func(d *restore.FileGroupInfo) (shouldBreak bool) { fileCount++ totalSize += d.Length - for _, f := range d.DataFilesInfo { - kvCount += f.NumberOfEntries - } + kvCount += d.KVCount return }) console.Printf("We are going to remove %s files, until %s.\n", @@ -979,83 +969,39 @@ func RunStreamTruncate(c context.Context, g glue.Glue, cmdName string, cfg *Stre } } - removed := metas.RemoveDataBefore(shiftUntilTS) - - // remove log + // begin to remove p := console.StartProgressBar( - "Clearing Data Files", len(removed), + "Clearing Data Files and Metadata", fileCount, glue.WithTimeCost(), glue.WithConstExtraField("kv-count", kvCount), glue.WithConstExtraField("kv-size", fmt.Sprintf("%d(%s)", totalSize, units.HumanSize(float64(totalSize)))), ) defer p.Close() - worker := utils.NewWorkerPool(128, "delete files") - eg, cx := errgroup.WithContext(ctx) - const keepFirstNFailure = 16 - var notDeleted struct { - item []string - sync.Mutex - } - for _, f := range removed { - f := f - worker.ApplyOnErrorGroup(eg, func() error { - if cx.Err() != nil { - p.Close() - return cx.Err() - } - defer p.Inc() - log.Debug("Deleting file", zap.String("path", f.Path)) - if cfg.DryRun { - return nil - } - if err := storage.DeleteFile(ctx, f.Path); err != nil { - log.Warn("File not deleted.", zap.String("path", f.Path), logutil.ShortError(err)) - notDeleted.Lock() - defer notDeleted.Unlock() - notDeleted.item = append(notDeleted.item, f.Path) - if len(notDeleted.item) > notDeletedBecameFatalThreshold { - return errors.Annotatef(berrors.ErrPiTRMalformedMetadata, "too many failure when truncating") - } - } - return nil - }) - } - if err := p.Wait(ctx); err != nil { + + notDeleted, err := metas.RemoveDataFilesAndUpdateMetadataInBatch(ctx, shiftUntilTS, storage, p.IncBy) + if err != nil { return err } - if err := eg.Wait(); err != nil { + + if err := p.Wait(ctx); err != nil { return err } - notDeleted.Lock() - if len(notDeleted.item) > 0 { + if len(notDeleted) > 0 { + const keepFirstNFailure = 16 console.Println("Files below are not deleted due to error, you may clear it manually, check log for detail error:") - console.Println("- Total", em(len(notDeleted.item)), "items.") - if len(notDeleted.item) > keepFirstNFailure { - console.Println("-", em(len(notDeleted.item)-keepFirstNFailure), "items omitted.") + console.Println("- Total", em(len(notDeleted)), "items.") + if len(notDeleted) > keepFirstNFailure { + console.Println("-", em(len(notDeleted)-keepFirstNFailure), "items omitted.") // TODO: maybe don't add them at the very first. - notDeleted.item = notDeleted.item[:keepFirstNFailure] + notDeleted = notDeleted[:keepFirstNFailure] } - for _, f := range notDeleted.item { + for _, f := range notDeleted { console.Println(f) } } - notDeleted.Unlock() - // remove metadata - pw := console.StartProgressBar("Removing Metadata", metas.PendingMeta(), glue.WithTimeCost(), glue.WithConstExtraField("metas", metas.PendingMeta())) - defer pw.Close() - metas.BeforeDoWriteBack = func(path string, last, current *backuppb.Metadata) (skip bool) { - log.Info("Updating metadata.", zap.String("file", path), - zap.Int("data-file-before", len(last.GetFiles())), - zap.Int("data-file-after", len(current.GetFiles()))) - pw.Inc() - return cfg.DryRun - } - if err := metas.DoWriteBack(ctx, storage); err != nil { - return err - } - return pw.Wait(ctx) + return nil } // RunStreamRestore restores stream log. From 16911610d272fc03365b429741d049d9caf9f786 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E8=B6=85?= Date: Tue, 8 Nov 2022 17:55:50 +0800 Subject: [PATCH 12/22] extension: provide more complete info in StmtEventInfo when error (#38963) close pingcap/tidb#38962 --- extension/event_listener_test.go | 52 +++++++++++++++++++++++++++++++- planner/core/planbuilder.go | 4 +-- planner/optimize.go | 6 +++- server/conn.go | 14 ++++++--- server/conn_stmt.go | 7 ++++- server/extension.go | 12 ++++---- 6 files changed, 80 insertions(+), 15 deletions(-) diff --git a/extension/event_listener_test.go b/extension/event_listener_test.go index ece046f820b64..6e108262dbf8c 100644 --- a/extension/event_listener_test.go +++ b/extension/event_listener_test.go @@ -137,6 +137,7 @@ func TestExtensionStmtEvents(t *testing.T) { require.NoError(t, conn.HandleQuery(context.Background(), "SET tidb_multi_statement_mode='ON'")) require.NoError(t, conn.HandleQuery(context.Background(), "use test")) require.NoError(t, conn.HandleQuery(context.Background(), "create table t1(a int, b int)")) + require.NoError(t, conn.HandleQuery(context.Background(), "create table t2(id int primary key)")) require.NoError(t, conn.HandleQuery(context.Background(), "create database test2")) require.NoError(t, conn.HandleQuery(context.Background(), "create table test2.t1(c int, d int)")) require.NoError(t, conn.HandleQuery(context.Background(), "set @a=1")) @@ -158,6 +159,10 @@ func TestExtensionStmtEvents(t *testing.T) { stmtID3 := getPreparedID(t, conn.Context()) require.NoError(t, conn.HandleQuery(context.Background(), "drop table tnoexist")) + cmd = append([]byte{mysql.ComStmtPrepare}, []byte("insert into t2 values(?)")...) + require.NoError(t, conn.Dispatch(context.Background(), cmd)) + stmtID4 := getPreparedID(t, conn.Context()) + connID := conn.Context().Session.GetSessionVars().ConnectionID require.NotEqual(t, uint64(0), connID) @@ -186,6 +191,14 @@ func TestExtensionStmtEvents(t *testing.T) { {DB: "test", Table: "t1"}, }, }, + { + sql: "insert into t2 values(1)", + redactText: "insert into `t2` values ( ? )", + affectedRows: 1, + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t2"}, + }, + }, { binaryExecute: stmtID2, executeParams: []paramInfo{ @@ -206,7 +219,22 @@ func TestExtensionStmtEvents(t *testing.T) { }, originalText: "select * from tnoexist where n=?", redactText: "select * from `tnoexist` where `n` = ?", - err: "select * from tnoexist where n=? [arguments: 5]: [planner:8113]Schema change caused error: [schema:1146]Table 'test.tnoexist' doesn't exist", + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "tnoexist"}, + }, + err: "select * from tnoexist where n=? [arguments: 5]: [planner:8113]Schema change caused error: [schema:1146]Table 'test.tnoexist' doesn't exist", + }, + { + binaryExecute: stmtID4, + executeParams: []paramInfo{ + {value: 3}, + }, + originalText: "insert into t2 values(?)", + redactText: "insert into `t2` values ( ? )", + affectedRows: 1, + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t2"}, + }, }, { sql: "prepare s from 'select * from t1 where a=1 and b>? and b 1", multiQueryCases: []stmtEventCase{ @@ -258,6 +295,19 @@ func TestExtensionStmtEvents(t *testing.T) { }, }, }, + { + binaryExecute: stmtID4, + executeParams: []paramInfo{ + {value: 3}, + }, + err: "insert into t2 values(?) [arguments: 3]: [kv:1062]Duplicate entry '3' for key 't2.PRIMARY'", + originalText: "insert into t2 values(?)", + redactText: "insert into `t2` values ( ? )", + affectedRows: 0, + tables: []stmtctx.TableEntry{ + {DB: "test", Table: "t2"}, + }, + }, } for i, c := range cases { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index a07f3c13d6717..6815088d3f0af 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -633,7 +633,7 @@ func (b *PlanBuilder) GetIsForUpdateRead() bool { } // GetDBTableInfo gets the accessed dbs and tables info. -func (b *PlanBuilder) GetDBTableInfo() []stmtctx.TableEntry { +func GetDBTableInfo(visitInfo []visitInfo) []stmtctx.TableEntry { var tables []stmtctx.TableEntry existsFunc := func(tbls []stmtctx.TableEntry, tbl *stmtctx.TableEntry) bool { for _, t := range tbls { @@ -643,7 +643,7 @@ func (b *PlanBuilder) GetDBTableInfo() []stmtctx.TableEntry { } return false } - for _, v := range b.visitInfo { + for _, v := range visitInfo { tbl := &stmtctx.TableEntry{DB: v.db, Table: v.table} if !existsFunc(tables, tbl) { tables = append(tables, *tbl) diff --git a/planner/optimize.go b/planner/optimize.go index 85a35533ffbf8..287ea7978fd9e 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -441,7 +441,11 @@ func buildLogicalPlan(ctx context.Context, sctx sessionctx.Context, node ast.Nod return nil, err } sctx.GetSessionVars().RewritePhaseInfo.DurationRewrite = time.Since(beginRewrite) - sctx.GetSessionVars().StmtCtx.Tables = builder.GetDBTableInfo() + if exec, ok := p.(*core.Execute); ok && exec.PrepStmt != nil { + sctx.GetSessionVars().StmtCtx.Tables = core.GetDBTableInfo(exec.PrepStmt.VisitInfos) + } else { + sctx.GetSessionVars().StmtCtx.Tables = core.GetDBTableInfo(builder.GetVisitInfo()) + } return p, nil } diff --git a/server/conn.go b/server/conn.go index 6a466fbffae5f..b87eb5579219f 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1908,7 +1908,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { pointPlans, err = cc.prefetchPointPlanKeys(ctx, stmts) if err != nil { for _, stmt := range stmts { - cc.onExtensionStmtEnd(stmt, err) + cc.onExtensionStmtEnd(stmt, false, err) } return err } @@ -1919,12 +1919,18 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { } var retryable bool var lastStmt ast.StmtNode + var expiredStmtTaskID uint64 for i, stmt := range stmts { if lastStmt != nil { - cc.onExtensionStmtEnd(lastStmt, nil) + cc.onExtensionStmtEnd(lastStmt, true, nil) } - lastStmt = stmt + + // expiredTaskID is the task ID of the previous statement. When executing a stmt, + // the StmtCtx will be reinit and the TaskID will change. We can compare the StmtCtx.TaskID + // with the previous one to determine whether StmtCtx has been inited for the current stmt. + expiredStmtTaskID = sessVars.StmtCtx.TaskID + if len(pointPlans) > 0 { // Save the point plan in Session, so we don't need to build the point plan again. cc.ctx.SetValue(plannercore.PointPlanKey, plannercore.PointPlanVal{Plan: pointPlans[i]}) @@ -1966,7 +1972,7 @@ func (cc *clientConn) handleQuery(ctx context.Context, sql string) (err error) { } if lastStmt != nil { - cc.onExtensionStmtEnd(lastStmt, err) + cc.onExtensionStmtEnd(lastStmt, sessVars.StmtCtx.TaskID != expiredStmtTaskID, err) } return err diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 26dfd9f38d439..47d0dc241a3ed 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -205,8 +205,13 @@ func (cc *clientConn) handleStmtExecute(ctx context.Context, data []byte) (err e } } + sessVars := cc.ctx.GetSessionVars() + // expiredTaskID is the task ID of the previous statement. When executing a stmt, + // the StmtCtx will be reinit and the TaskID will change. We can compare the StmtCtx.TaskID + // with the previous one to determine whether StmtCtx has been inited for the current stmt. + expiredTaskID := sessVars.StmtCtx.TaskID err = cc.executePlanCacheStmt(ctx, stmt, args, useCursor) - cc.onExtensionBinaryExecuteEnd(stmt, args, err) + cc.onExtensionBinaryExecuteEnd(stmt, args, sessVars.StmtCtx.TaskID != expiredTaskID, err) return err } diff --git a/server/extension.go b/server/extension.go index 9973226cff841..c7cf018eb85a8 100644 --- a/server/extension.go +++ b/server/extension.go @@ -54,7 +54,7 @@ func (cc *clientConn) onExtensionConnEvent(tp extension.ConnEventTp, err error) cc.extensions.OnConnectionEvent(tp, info) } -func (cc *clientConn) onExtensionStmtEnd(node interface{}, err error, args ...expression.Expression) { +func (cc *clientConn) onExtensionStmtEnd(node interface{}, stmtCtxValid bool, err error, args ...expression.Expression) { if !cc.extensions.HasStmtEventListeners() { return } @@ -91,8 +91,8 @@ func (cc *clientConn) onExtensionStmtEnd(node interface{}, err error, args ...ex info.stmtNode = stmt } - if sc := sessVars.StmtCtx; sc != nil && err == nil { - info.sc = sc + if stmtCtxValid { + info.sc = sessVars.StmtCtx } else { info.sc = &stmtctx.StatementContext{} } @@ -112,8 +112,8 @@ func (cc *clientConn) onExtensionSQLParseFailed(sql string, err error) { }) } -func (cc *clientConn) onExtensionBinaryExecuteEnd(prep PreparedStatement, args []expression.Expression, err error) { - cc.onExtensionStmtEnd(prep, err, args...) +func (cc *clientConn) onExtensionBinaryExecuteEnd(prep PreparedStatement, args []expression.Expression, stmtCtxValid bool, err error) { + cc.onExtensionStmtEnd(prep, stmtCtxValid, err, args...) } type stmtEventInfo struct { @@ -191,7 +191,7 @@ func (e *stmtEventInfo) CurrentDB() string { } func (e *stmtEventInfo) AffectedRows() uint64 { - if e.sc == nil { + if e.sc == nil || e.err != nil { return 0 } return e.sc.AffectedRows() From 952a5b977d6570b1bdd82a3ec4b00bc0369b1743 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Tue, 8 Nov 2022 19:09:52 +0800 Subject: [PATCH 13/22] *: fix variable problem during initialization (#38971) close pingcap/tidb#38973 --- sessionctx/variable/BUILD.bazel | 1 + sessionctx/variable/sysvar.go | 6 ++++-- sessionctx/variable/variable_test.go | 8 ++++++-- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/sessionctx/variable/BUILD.bazel b/sessionctx/variable/BUILD.bazel index fa4865079e8bf..7c6bcd5330e25 100644 --- a/sessionctx/variable/BUILD.bazel +++ b/sessionctx/variable/BUILD.bazel @@ -85,6 +85,7 @@ go_test( ], embed = [":variable"], flaky = True, + shard_count = 2, deps = [ "//config", "//kv", diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 580a8deca7b7e..dee8b4c3997a5 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -746,7 +746,8 @@ var defaultSysVars = []*SysVar{ if floatValue < 0 && floatValue > 0.9 { return "", ErrWrongValueForVar.GenWithStackByArgs(TiDBGOGCTunerThreshold, normalizedValue) } - if globalMemoryLimitTuner < floatValue+0.05 { + // globalMemoryLimitTuner must not be 0. it will be 0 when tidb_server_memory_limit_gc_trigger is not set during startup. + if globalMemoryLimitTuner != 0 && globalMemoryLimitTuner < floatValue+0.05 { return "", errors.New("tidb_gogc_tuner_threshold should be less than tidb_server_memory_limit_gc_trigger - 0.05") } return strconv.FormatFloat(floatValue, 'f', -1, 64), nil @@ -827,7 +828,8 @@ var defaultSysVars = []*SysVar{ if floatValue < 0.51 && floatValue > 1 { // 51% ~ 100% return "", ErrWrongValueForVar.GenWithStackByArgs(TiDBServerMemoryLimitGCTrigger, normalizedValue) } - if floatValue < gogcTunerThreshold+0.05 { + // gogcTunerThreshold must not be 0. it will be 0 when tidb_gogc_tuner_threshold is not set during startup. + if gogcTunerThreshold != 0 && floatValue < gogcTunerThreshold+0.05 { return "", errors.New("tidb_server_memory_limit_gc_trigger should be greater than tidb_gogc_tuner_threshold + 0.05") } diff --git a/sessionctx/variable/variable_test.go b/sessionctx/variable/variable_test.go index bcf917413de3d..023cc75e7cba7 100644 --- a/sessionctx/variable/variable_test.go +++ b/sessionctx/variable/variable_test.go @@ -436,18 +436,22 @@ func TestDefaultValuesAreSettable(t *testing.T) { for _, sv := range GetSysVars() { if sv.HasSessionScope() && !sv.ReadOnly { val, err := sv.Validate(vars, sv.Value, ScopeSession) - require.Equal(t, val, sv.Value) require.NoError(t, err) + require.Equal(t, val, sv.Value) } if sv.HasGlobalScope() && !sv.ReadOnly { val, err := sv.Validate(vars, sv.Value, ScopeGlobal) - require.Equal(t, val, sv.Value) require.NoError(t, err) + require.Equal(t, val, sv.Value) } } } +func TestLimitBetweenVariable(t *testing.T) { + require.Less(t, DefTiDBGOGCTunerThreshold+0.05, DefTiDBServerMemoryLimitGCTrigger) +} + // TestSysVarNameIsLowerCase tests that no new sysvars are added with uppercase characters. // In MySQL variables are always lowercase, and can be set in a case-insensitive way. func TestSysVarNameIsLowerCase(t *testing.T) { From ad4d43219ab66c67eb54ea2aad709bd5d8944b12 Mon Sep 17 00:00:00 2001 From: Song Gao Date: Tue, 8 Nov 2022 19:31:50 +0800 Subject: [PATCH 14/22] planner: support record table runtime stats during optimize (#38876) ref pingcap/tidb#38779 --- executor/executor.go | 1 + infoschema/tables_test.go | 17 ++--- planner/core/BUILD.bazel | 1 + planner/core/collect_column_stats_usage.go | 20 ++++++ planner/core/plan_replayer_capture_test.go | 83 ++++++++++++++++++++++ planner/core/plan_stats.go | 32 +++++++++ planner/optimize.go | 2 + sessionctx/stmtctx/stmtctx.go | 2 + sessionctx/variable/sysvar.go | 7 ++ sessionctx/variable/tidb_vars.go | 20 +++--- statistics/handle/dump.go | 28 +++++--- 11 files changed, 187 insertions(+), 26 deletions(-) create mode 100644 planner/core/plan_replayer_capture_test.go diff --git a/executor/executor.go b/executor/executor.go index 542734e4b1a75..1fc3a88f3a4fd 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -1954,6 +1954,7 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) { vars.MemTracker.ResetMaxConsumed() vars.DiskTracker.ResetMaxConsumed() vars.MemTracker.SessionID = vars.ConnectionID + vars.StmtCtx.TableStats = make(map[int64]interface{}) if _, ok := s.(*ast.AnalyzeTableStmt); ok { sc.InitMemTracker(memory.LabelForAnalyzeMemory, -1) diff --git a/infoschema/tables_test.go b/infoschema/tables_test.go index 64b38de1c9db3..129a98ea921f3 100644 --- a/infoschema/tables_test.go +++ b/infoschema/tables_test.go @@ -1543,14 +1543,15 @@ func TestVariablesInfo(t *testing.T) { // See session/bootstrap.go:doDMLWorks() for where the exceptions are defined. stmt := tk.MustQuery(`SELECT variable_name, default_value, current_value FROM information_schema.variables_info WHERE current_value != default_value and default_value != '' ORDER BY variable_name`) stmt.Check(testkit.Rows( - "last_sql_use_alloc OFF ON", // for test stability - "tidb_enable_auto_analyze ON OFF", // always changed for tests - "tidb_enable_collect_execution_info ON OFF", // for test stability - "tidb_enable_mutation_checker OFF ON", // for new installs - "tidb_mem_oom_action CANCEL LOG", // always changed for tests - "tidb_row_format_version 1 2", // for new installs - "tidb_txn_assertion_level OFF FAST", // for new installs - "timestamp 0 123456789", // always dynamic + "last_sql_use_alloc OFF ON", // for test stability + "tidb_enable_auto_analyze ON OFF", // always changed for tests + "tidb_enable_collect_execution_info ON OFF", // for test stability + "tidb_enable_mutation_checker OFF ON", // for new installs + "tidb_enable_plan_replayer_capture OFF false", // for enable plan replayer capture + "tidb_mem_oom_action CANCEL LOG", // always changed for tests + "tidb_row_format_version 1 2", // for new installs + "tidb_txn_assertion_level OFF FAST", // for new installs + "timestamp 0 123456789", // always dynamic )) } diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index 711c4af298925..9db73e6d5fcb8 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -188,6 +188,7 @@ go_test( "plan_cost_detail_test.go", "plan_cost_ver1_test.go", "plan_cost_ver2_test.go", + "plan_replayer_capture_test.go", "plan_stats_test.go", "plan_test.go", "plan_to_pb_test.go", diff --git a/planner/core/collect_column_stats_usage.go b/planner/core/collect_column_stats_usage.go index 49f5cfdfd7011..49fffb149b85d 100644 --- a/planner/core/collect_column_stats_usage.go +++ b/planner/core/collect_column_stats_usage.go @@ -17,6 +17,7 @@ package core import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/sessionctx/variable" ) const ( @@ -43,6 +44,11 @@ type columnStatsUsageCollector struct { histNeededCols map[model.TableItemID]struct{} // cols is used to store columns collected from expressions and saves some allocation. cols []*expression.Column + + // collectVisitedTable indicates whether to collect visited table + collectVisitedTable bool + // visitedtbls indicates the visited table + visitedtbls map[int64]struct{} } func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector { @@ -58,6 +64,10 @@ func newColumnStatsUsageCollector(collectMode uint64) *columnStatsUsageCollector if collectMode&collectHistNeededColumns != 0 { collector.histNeededCols = make(map[model.TableItemID]struct{}) } + if variable.EnablePlanReplayerCapture.Load() { + collector.collectVisitedTable = true + collector.visitedtbls = map[int64]struct{}{} + } return collector } @@ -103,6 +113,9 @@ func (c *columnStatsUsageCollector) collectPredicateColumnsForDataSource(ds *Dat // For partition tables, no matter whether it is static or dynamic pruning mode, we use table ID rather than partition ID to // set TableColumnID.TableID. In this way, we keep the set of predicate columns consistent between different partitions and global table. tblID := ds.TableInfo().ID + if c.collectVisitedTable { + c.visitedtbls[tblID] = struct{}{} + } for _, col := range ds.Schema().Columns { tblColID := model.TableItemID{TableID: tblID, ID: col.ID, IsIndex: false} c.colMap[col.UniqueID] = map[model.TableItemID]struct{}{tblColID: {}} @@ -147,6 +160,10 @@ func (c *columnStatsUsageCollector) collectPredicateColumnsForUnionAll(p *Logica } func (c *columnStatsUsageCollector) addHistNeededColumns(ds *DataSource) { + if c.collectVisitedTable { + tblID := ds.TableInfo().ID + c.visitedtbls[tblID] = struct{}{} + } columns := expression.ExtractColumnsFromExpressions(c.cols[:0], ds.pushedDownConds, nil) for _, col := range columns { tblColID := model.TableItemID{TableID: ds.physicalTableID, ID: col.ID, IsIndex: false} @@ -285,6 +302,9 @@ func CollectColumnStatsUsage(lp LogicalPlan, predicate, histNeeded bool) ([]mode } collector := newColumnStatsUsageCollector(mode) collector.collectFromPlan(lp) + if collector.collectVisitedTable { + recordTableRuntimeStats(lp.SCtx(), collector.visitedtbls) + } set2slice := func(set map[model.TableItemID]struct{}) []model.TableItemID { ret := make([]model.TableItemID, 0, len(set)) for tblColID := range set { diff --git a/planner/core/plan_replayer_capture_test.go b/planner/core/plan_replayer_capture_test.go new file mode 100644 index 0000000000000..2e88f090bd784 --- /dev/null +++ b/planner/core/plan_replayer_capture_test.go @@ -0,0 +1,83 @@ +// Copyright 2022 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, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package core_test + +import ( + "context" + "testing" + + "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/planner/core" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/statistics" + "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tidb/util/hint" + "github.com/stretchr/testify/require" +) + +func TestPlanReplayerCaptureRecordJsonStats(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t) + tk := testkit.NewTestKit(t, store) + ctx := tk.Session().(sessionctx.Context) + tk.MustExec("use test") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int)") + tk.MustExec("SET global tidb_enable_plan_replayer_capture = ON;") + tk.MustExec("analyze table t1") + tk.MustExec("analyze table t2") + testcases := []struct { + sql string + count int + }{ + { + sql: "select * from t1", + count: 1, + }, + { + sql: "select * from t2", + count: 1, + }, + { + sql: "select * from t1,t2", + count: 2, + }, + } + for _, tc := range testcases { + tableStats := getTableStats(tc.sql, t, ctx, dom) + require.Equal(t, tc.count, len(tableStats)) + } +} + +func getTableStats(sql string, t *testing.T, ctx sessionctx.Context, dom *domain.Domain) map[int64]*statistics.Table { + p := parser.New() + stmt, err := p.ParseOneStmt(sql, "", "") + require.NoError(t, err) + err = core.Preprocess(context.Background(), ctx, stmt, core.WithPreprocessorReturn(&core.PreprocessorReturn{InfoSchema: dom.InfoSchema()})) + require.NoError(t, err) + sctx := core.MockContext() + builder, _ := core.NewPlanBuilder().Init(sctx, dom.InfoSchema(), &hint.BlockHintProcessor{}) + domain.GetDomain(sctx).MockInfoCacheAndLoadInfoSchema(dom.InfoSchema()) + plan, err := builder.Build(context.TODO(), stmt) + require.NoError(t, err) + _, _, err = core.DoOptimize(context.TODO(), sctx, builder.GetOptFlag(), plan.(core.LogicalPlan)) + require.NoError(t, err) + tableStats := sctx.GetSessionVars().StmtCtx.TableStats + r := make(map[int64]*statistics.Table) + for key, v := range tableStats { + r[key] = v.(*statistics.Table) + } + return r +} diff --git a/planner/core/plan_stats.go b/planner/core/plan_stats.go index 9575c3e8d79b3..9bea8edb8b0ba 100644 --- a/planner/core/plan_stats.go +++ b/planner/core/plan_stats.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" "go.uber.org/zap" @@ -170,3 +171,34 @@ func collectHistNeededItems(histNeededColumns []model.TableItemID, histNeededInd histNeededItems = append(histNeededItems, histNeededColumns...) return } + +func recordTableRuntimeStats(sctx sessionctx.Context, tbls map[int64]struct{}) { + tblStats := sctx.GetSessionVars().StmtCtx.TableStats + if tblStats == nil { + tblStats = map[int64]interface{}{} + } + for tblID := range tbls { + tblJSONStats, err := recordSingleTableRuntimeStats(sctx, tblID) + if err != nil { + logutil.BgLogger().Warn("record table json stats failed", zap.Int64("tblID", tblID), zap.Error(err)) + } + if tblJSONStats == nil { + logutil.BgLogger().Warn("record table json stats failed due to empty", zap.Int64("tblID", tblID)) + } + tblStats[tblID] = tblJSONStats + } + sctx.GetSessionVars().StmtCtx.TableStats = tblStats +} + +func recordSingleTableRuntimeStats(sctx sessionctx.Context, tblID int64) (*statistics.Table, error) { + dom := domain.GetDomain(sctx) + is := dom.InfoSchema() + statsHandle := dom.StatsHandle() + tbl, ok := is.TableByID(tblID) + if !ok { + return nil, nil + } + tableInfo := tbl.Meta() + stats := statsHandle.GetTableStats(tableInfo) + return stats, nil +} diff --git a/planner/optimize.go b/planner/optimize.go index 287ea7978fd9e..747dd5541596a 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -395,6 +395,8 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in beginOpt := time.Now() finalPlan, cost, err := core.DoOptimize(ctx, sctx, builder.GetOptFlag(), logic) + // TODO: capture plan replayer here if it matches sql and plan digest + sctx.GetSessionVars().DurationOptimization = time.Since(beginOpt) return finalPlan, names, cost, err } diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index bd716e8509938..a02090e490c32 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -341,6 +341,8 @@ type StatementContext struct { HasFKCascades bool } + // TableStats stores the visited runtime table stats by table id during query + TableStats map[int64]interface{} // useChunkAlloc indicates whether statement use chunk alloc useChunkAlloc bool } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index dee8b4c3997a5..a592e7a7e8831 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1039,6 +1039,13 @@ var defaultSysVars = []*SysVar{ }, GetGlobal: func(_ context.Context, s *SessionVars) (string, error) { return fmt.Sprintf("%d", MemoryUsageAlarmKeepRecordNum.Load()), nil }}, + {Scope: ScopeGlobal, Name: TiDBEnablePlanReplayerCapture, Value: BoolToOnOff(false), Type: TypeBool, + SetGlobal: func(ctx context.Context, s *SessionVars, val string) error { + EnablePlanReplayerCapture.Store(TiDBOptOn(val)) + return nil + }, GetGlobal: func(ctx context.Context, vars *SessionVars) (string, error) { + return strconv.FormatBool(EnablePlanReplayerCapture.Load()), nil + }}, /* The system variables below have GLOBAL and SESSION scope */ {Scope: ScopeGlobal | ScopeSession, Name: TiDBRowFormatVersion, Value: strconv.Itoa(DefTiDBRowFormatV1), Type: TypeUnsigned, MinValue: 1, MaxValue: 2, SetGlobal: func(_ context.Context, s *SessionVars, val string) error { diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 4c261d8def2e4..80c9b41f4cc6e 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -768,6 +768,8 @@ const ( // TiDBEnableExternalTSRead indicates whether to enable read through an external ts TiDBEnableExternalTSRead = "tidb_enable_external_ts_read" + // TiDBEnablePlanReplayerCapture indicates whether to enable plan replayer capture + TiDBEnablePlanReplayerCapture = "tidb_enable_plan_replayer_capture" // TiDBEnableReusechunk indicates whether to enable chunk alloc TiDBEnableReusechunk = "tidb_enable_reuse_chunk" ) @@ -1083,12 +1085,13 @@ const ( DefTiDBServerMemoryLimitGCTrigger = 0.7 DefTiDBEnableGOGCTuner = true // DefTiDBGOGCTunerThreshold is to limit TiDBGOGCTunerThreshold. - DefTiDBGOGCTunerThreshold float64 = 0.6 - DefTiDBOptPrefixIndexSingleScan = true - DefTiDBExternalTS = 0 - DefTiDBEnableExternalTSRead = false - DefTiDBEnableReusechunk = true - DefTiDBUseAlloc = false + DefTiDBGOGCTunerThreshold float64 = 0.6 + DefTiDBOptPrefixIndexSingleScan = true + DefTiDBExternalTS = 0 + DefTiDBEnableExternalTSRead = false + DefTiDBEnableReusechunk = true + DefTiDBUseAlloc = false + DefTiDBEnablePlanReplayerCapture = false ) // Process global variables. @@ -1145,8 +1148,9 @@ var ( // DefTiDBServerMemoryLimit indicates the default value of TiDBServerMemoryLimit(TotalMem * 80%). // It should be a const and shouldn't be modified after tidb is started. - DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() - GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) + DefTiDBServerMemoryLimit = serverMemoryLimitDefaultValue() + GOGCTunerThreshold = atomic.NewFloat64(DefTiDBGOGCTunerThreshold) + EnablePlanReplayerCapture = atomic.NewBool(DefTiDBEnablePlanReplayerCapture) ) var ( diff --git a/statistics/handle/dump.go b/statistics/handle/dump.go index 123f02b912000..777e3b70058c1 100644 --- a/statistics/handle/dump.go +++ b/statistics/handle/dump.go @@ -168,15 +168,8 @@ func (h *Handle) DumpStatsToJSONBySnapshot(dbName string, tableInfo *model.Table return jsonTbl, nil } -func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) { - tbl, err := h.TableStatsFromStorage(tableInfo, physicalID, true, snapshot) - if err != nil || tbl == nil { - return nil, err - } - tbl.Version, tbl.ModifyCount, tbl.Count, err = h.statsMetaByTableIDFromStorage(physicalID, snapshot) - if err != nil { - return nil, err - } +// GenJSONTableFromStats generate jsonTable from tableInfo and stats +func GenJSONTableFromStats(dbName string, tableInfo *model.TableInfo, tbl *statistics.Table) (*JSONTable, error) { jsonTbl := &JSONTable{ DatabaseName: dbName, TableName: tableInfo.Name.L, @@ -185,7 +178,6 @@ func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, phy Count: tbl.Count, ModifyCount: tbl.ModifyCount, } - for _, col := range tbl.Columns { sc := &stmtctx.StatementContext{TimeZone: time.UTC} hist, err := col.ConvertTo(sc, types.NewFieldType(mysql.TypeBlob)) @@ -202,6 +194,22 @@ func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, phy return jsonTbl, nil } +func (h *Handle) tableStatsToJSON(dbName string, tableInfo *model.TableInfo, physicalID int64, snapshot uint64) (*JSONTable, error) { + tbl, err := h.TableStatsFromStorage(tableInfo, physicalID, true, snapshot) + if err != nil || tbl == nil { + return nil, err + } + tbl.Version, tbl.ModifyCount, tbl.Count, err = h.statsMetaByTableIDFromStorage(physicalID, snapshot) + if err != nil { + return nil, err + } + jsonTbl, err := GenJSONTableFromStats(dbName, tableInfo, tbl) + if err != nil { + return nil, err + } + return jsonTbl, nil +} + // LoadStatsFromJSON will load statistic from JSONTable, and save it to the storage. func (h *Handle) LoadStatsFromJSON(is infoschema.InfoSchema, jsonTbl *JSONTable) error { table, err := is.TableByName(model.NewCIStr(jsonTbl.DatabaseName), model.NewCIStr(jsonTbl.TableName)) From ea26284ea3dd02f026430e4c6e8ac145d1360ad8 Mon Sep 17 00:00:00 2001 From: Aidan <97376271+keeplearning20221@users.noreply.github.com> Date: Tue, 8 Nov 2022 20:13:50 +0800 Subject: [PATCH 15/22] *: avoid special cases DATA RACE (#38918) close pingcap/tidb#38914 --- distsql/select_result.go | 2 +- executor/builder.go | 4 +- executor/distsql.go | 2 +- executor/executor.go | 4 +- executor/index_lookup_hash_join.go | 2 +- executor/index_lookup_join.go | 6 +-- executor/index_lookup_merge_join.go | 2 +- executor/index_merge_reader.go | 4 +- executor/join.go | 2 +- executor/joiner.go | 2 +- executor/pipelined_window.go | 2 +- executor/window.go | 2 +- server/conn.go | 2 +- server/conn_stmt.go | 2 +- sessionctx/variable/session.go | 60 +++++++++++++---------------- sessionctx/variable/session_test.go | 31 ++++++--------- testkit/testkit.go | 7 ++-- util/chunk/alloc.go | 25 +++++++++++- util/chunk/alloc_test.go | 19 +++++++++ 19 files changed, 103 insertions(+), 77 deletions(-) diff --git a/distsql/select_result.go b/distsql/select_result.go index a2d6215987a32..0e807b360d0ad 100644 --- a/distsql/select_result.go +++ b/distsql/select_result.go @@ -311,7 +311,7 @@ func (r *selectResult) readFromDefault(ctx context.Context, chk *chunk.Chunk) er func (r *selectResult) readFromChunk(ctx context.Context, chk *chunk.Chunk) error { if r.respChunkDecoder == nil { r.respChunkDecoder = chunk.NewDecoder( - r.ctx.GetSessionVars().GetNewChunk(r.fieldTypes, 0), + chunk.NewChunkWithCapacity(r.fieldTypes, 0), r.fieldTypes, ) } diff --git a/executor/builder.go b/executor/builder.go index 1b88b6e4047c6..70d91e64137a4 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1540,7 +1540,7 @@ func (b *executorBuilder) buildHashAgg(v *plannercore.PhysicalHashAgg) Executor e.defaultVal = nil } else { if v.IsFinalAgg() { - e.defaultVal = e.ctx.GetSessionVars().GetNewChunk(retTypes(e), 1) + e.defaultVal = e.ctx.GetSessionVars().GetNewChunkWithCapacity(retTypes(e), 1, 1, e.AllocPool) } } for _, aggDesc := range v.AggFuncs { @@ -1603,7 +1603,7 @@ func (b *executorBuilder) buildStreamAgg(v *plannercore.PhysicalStreamAgg) Execu } else { // Only do this for final agg, see issue #35295, #30923 if v.IsFinalAgg() { - e.defaultVal = e.ctx.GetSessionVars().GetNewChunk(retTypes(e), 1) + e.defaultVal = e.ctx.GetSessionVars().GetNewChunkWithCapacity(retTypes(e), 1, 1, e.AllocPool) } } for i, aggDesc := range v.AggFuncs { diff --git a/executor/distsql.go b/executor/distsql.go index 182831bc90021..0cef7e66d441e 100644 --- a/executor/distsql.go +++ b/executor/distsql.go @@ -866,7 +866,7 @@ func (w *indexWorker) fetchHandles(ctx context.Context, result distsql.SelectRes } }() retTps := w.idxLookup.getRetTpsByHandle() - chk := w.idxLookup.ctx.GetSessionVars().GetNewChunk(retTps, w.idxLookup.maxChunkSize) + chk := w.idxLookup.ctx.GetSessionVars().GetNewChunkWithCapacity(retTps, w.idxLookup.maxChunkSize, w.idxLookup.maxChunkSize, w.idxLookup.AllocPool) idxID := w.idxLookup.getIndexPlanRootID() if w.idxLookup.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { if idxID != w.idxLookup.id && w.idxLookup.stats != nil { diff --git a/executor/executor.go b/executor/executor.go index 1fc3a88f3a4fd..3d10ff32a5168 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -132,6 +132,7 @@ type baseExecutor struct { children []Executor retFieldTypes []*types.FieldType runtimeStats *execdetails.BasicRuntimeStats + AllocPool chunk.Allocator } const ( @@ -234,7 +235,7 @@ func newFirstChunk(e Executor) *chunk.Chunk { func tryNewCacheChunk(e Executor) *chunk.Chunk { base := e.base() s := base.ctx.GetSessionVars() - return s.GetNewChunkWithCapacity(base.retFieldTypes, base.initCap, base.maxChunkSize) + return s.GetNewChunkWithCapacity(base.retFieldTypes, base.initCap, base.maxChunkSize, base.AllocPool) } // newList creates a new List to buffer current executor's result. @@ -267,6 +268,7 @@ func newBaseExecutor(ctx sessionctx.Context, schema *expression.Schema, id int, schema: schema, initCap: ctx.GetSessionVars().InitChunkSize, maxChunkSize: ctx.GetSessionVars().MaxChunkSize, + AllocPool: ctx.GetSessionVars().ChunkPool.Alloc, } if ctx.GetSessionVars().StmtCtx.RuntimeStatsColl != nil { if e.id > 0 { diff --git a/executor/index_lookup_hash_join.go b/executor/index_lookup_hash_join.go index 9601dffc77900..c54b60749601d 100644 --- a/executor/index_lookup_hash_join.go +++ b/executor/index_lookup_hash_join.go @@ -418,7 +418,7 @@ func (e *IndexNestedLoopHashJoin) newInnerWorker(taskCh chan *indexHashJoinTask, innerCtx: e.innerCtx, outerCtx: e.outerCtx, ctx: e.ctx, - executorChk: e.ctx.GetSessionVars().GetNewChunk(e.innerCtx.rowTypes, e.maxChunkSize), + executorChk: e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize, e.maxChunkSize, e.AllocPool), indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, stats: innerStats, diff --git a/executor/index_lookup_join.go b/executor/index_lookup_join.go index 92f195985a191..05cc337d3d7ee 100644 --- a/executor/index_lookup_join.go +++ b/executor/index_lookup_join.go @@ -226,7 +226,7 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork outerCtx: e.outerCtx, taskCh: taskCh, ctx: e.ctx, - executorChk: e.ctx.GetSessionVars().GetNewChunk(e.innerCtx.rowTypes, e.maxChunkSize), + executorChk: e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.innerCtx.rowTypes, e.maxChunkSize, e.maxChunkSize, e.AllocPool), indexRanges: copiedRanges, keyOff2IdxOff: e.keyOff2IdxOff, stats: innerStats, @@ -431,7 +431,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { } maxChunkSize := ow.ctx.GetSessionVars().MaxChunkSize for requiredRows > task.outerResult.Len() { - chk := ow.ctx.GetSessionVars().GetNewChunk(ow.outerCtx.rowTypes, maxChunkSize) + chk := ow.ctx.GetSessionVars().GetNewChunkWithCapacity(ow.outerCtx.rowTypes, maxChunkSize, maxChunkSize, ow.executor.base().AllocPool) chk = chk.SetRequiredRows(requiredRows, maxChunkSize) err := Next(ctx, ow.executor, chk) if err != nil { @@ -462,7 +462,7 @@ func (ow *outerWorker) buildTask(ctx context.Context) (*lookUpJoinTask, error) { } task.encodedLookUpKeys = make([]*chunk.Chunk, task.outerResult.NumChunks()) for i := range task.encodedLookUpKeys { - task.encodedLookUpKeys[i] = ow.ctx.GetSessionVars().GetNewChunk([]*types.FieldType{types.NewFieldType(mysql.TypeBlob)}, task.outerResult.GetChunk(i).NumRows()) + task.encodedLookUpKeys[i] = ow.ctx.GetSessionVars().GetNewChunkWithCapacity([]*types.FieldType{types.NewFieldType(mysql.TypeBlob)}, task.outerResult.GetChunk(i).NumRows(), task.outerResult.GetChunk(i).NumRows(), ow.executor.base().AllocPool) } return task, nil } diff --git a/executor/index_lookup_merge_join.go b/executor/index_lookup_merge_join.go index 1ba2c2940c3fd..8bd379944c825 100644 --- a/executor/index_lookup_merge_join.go +++ b/executor/index_lookup_merge_join.go @@ -706,7 +706,7 @@ func (imw *innerMergeWorker) dedupDatumLookUpKeys(lookUpContents []*indexJoinLoo // fetchNextInnerResult collects a chunk of inner results from inner child executor. func (imw *innerMergeWorker) fetchNextInnerResult(ctx context.Context, task *lookUpMergeJoinTask) (beginRow chunk.Row, err error) { - task.innerResult = imw.ctx.GetSessionVars().GetNewChunk(retTypes(imw.innerExec), imw.ctx.GetSessionVars().MaxChunkSize) + task.innerResult = imw.ctx.GetSessionVars().GetNewChunkWithCapacity(retTypes(imw.innerExec), imw.ctx.GetSessionVars().MaxChunkSize, imw.ctx.GetSessionVars().MaxChunkSize, imw.innerExec.base().AllocPool) err = Next(ctx, imw.innerExec, task.innerResult) task.innerIter = chunk.NewIterator4Chunk(task.innerResult) beginRow = task.innerIter.Begin() diff --git a/executor/index_merge_reader.go b/executor/index_merge_reader.go index 82c6ab2f50817..0e7eb394710fd 100644 --- a/executor/index_merge_reader.go +++ b/executor/index_merge_reader.go @@ -508,7 +508,7 @@ func (w *partialTableWorker) syncErr(resultCh chan<- *lookupTableTask, err error func (w *partialTableWorker) fetchHandles(ctx context.Context, exitCh <-chan struct{}, fetchCh chan<- *lookupTableTask, resultCh chan<- *lookupTableTask, finished <-chan struct{}, handleCols plannercore.HandleCols) (count int64, err error) { - chk := w.sc.GetSessionVars().GetNewChunk(retTypes(w.tableReader), w.maxChunkSize) + chk := w.sc.GetSessionVars().GetNewChunkWithCapacity(retTypes(w.tableReader), w.maxChunkSize, w.maxChunkSize, w.tableReader.base().AllocPool) var basic *execdetails.BasicRuntimeStats if be := w.tableReader.base(); be != nil && be.runtimeStats != nil { basic = be.runtimeStats @@ -817,7 +817,7 @@ func (w *partialIndexWorker) fetchHandles( resultCh chan<- *lookupTableTask, finished <-chan struct{}, handleCols plannercore.HandleCols) (count int64, err error) { - chk := w.sc.GetSessionVars().GetNewChunk(handleCols.GetFieldsTypes(), w.maxChunkSize) + chk := chunk.NewChunkWithCapacity(handleCols.GetFieldsTypes(), w.maxChunkSize) var basicStats *execdetails.BasicRuntimeStats if w.stats != nil { if w.idxID != 0 { diff --git a/executor/join.go b/executor/join.go index 48d3e5d5a56f8..87781a49d7c50 100644 --- a/executor/join.go +++ b/executor/join.go @@ -297,7 +297,7 @@ func (e *HashJoinExec) fetchBuildSideRows(ctx context.Context, chkCh chan<- *chu if e.finished.Load().(bool) { return } - chk := e.ctx.GetSessionVars().GetNewChunk(e.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize) + chk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.buildSideExec.base().retFieldTypes, e.ctx.GetSessionVars().MaxChunkSize, e.ctx.GetSessionVars().MaxChunkSize, e.AllocPool) err = Next(ctx, e.buildSideExec, chk) if err != nil { e.buildFinished <- errors.Trace(err) diff --git a/executor/joiner.go b/executor/joiner.go index 5fe4d92eba2a2..842135802444f 100644 --- a/executor/joiner.go +++ b/executor/joiner.go @@ -192,7 +192,7 @@ func newJoiner(ctx sessionctx.Context, joinType plannercore.JoinType, return &antiLeftOuterSemiJoiner{base} case plannercore.LeftOuterJoin, plannercore.RightOuterJoin, plannercore.InnerJoin: if len(base.conditions) > 0 { - base.chk = ctx.GetSessionVars().GetNewChunk(shallowRowType, ctx.GetSessionVars().MaxChunkSize) + base.chk = chunk.NewChunkWithCapacity(shallowRowType, ctx.GetSessionVars().MaxChunkSize) } switch joinType { case plannercore.LeftOuterJoin: diff --git a/executor/pipelined_window.go b/executor/pipelined_window.go index 505cf09f415d7..cda1d9c389fd0 100644 --- a/executor/pipelined_window.go +++ b/executor/pipelined_window.go @@ -217,7 +217,7 @@ func (e *PipelinedWindowExec) fetchChild(ctx context.Context) (EOF bool, err err } // TODO: reuse chunks - resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows) + resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows, e.AllocPool) err = e.copyChk(childResult, resultChk) if err != nil { return false, err diff --git a/executor/window.go b/executor/window.go index ef284344d0c8c..aaa1e51cacc85 100644 --- a/executor/window.go +++ b/executor/window.go @@ -162,7 +162,7 @@ func (e *WindowExec) fetchChild(ctx context.Context) (EOF bool, err error) { return true, nil } - resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows) + resultChk := e.ctx.GetSessionVars().GetNewChunkWithCapacity(e.retFieldTypes, 0, numRows, e.AllocPool) err = e.copyChk(childResult, resultChk) if err != nil { return false, err diff --git a/server/conn.go b/server/conn.go index b87eb5579219f..b319679fdbf14 100644 --- a/server/conn.go +++ b/server/conn.go @@ -1121,8 +1121,8 @@ func (cc *clientConn) Run(ctx context.Context) { startTime := time.Now() err = cc.dispatch(ctx, data) + cc.ctx.GetSessionVars().ClearAlloc(&cc.chunkAlloc, err != nil) cc.chunkAlloc.Reset() - cc.ctx.GetSessionVars().ClearAlloc() if err != nil { cc.audit(plugin.Error) // tell the plugin API there was a dispatch error if terror.ErrorEqual(err, io.EOF) { diff --git a/server/conn_stmt.go b/server/conn_stmt.go index 47d0dc241a3ed..436b2d1721ca0 100644 --- a/server/conn_stmt.go +++ b/server/conn_stmt.go @@ -307,7 +307,7 @@ const ( func (cc *clientConn) handleStmtFetch(ctx context.Context, data []byte) (err error) { cc.ctx.GetSessionVars().StartTime = time.Now() - cc.ctx.GetSessionVars().ClearAlloc() + cc.ctx.GetSessionVars().ClearAlloc(nil, false) stmtID, fetchSize, err := parseStmtFetchCmd(data) if err != nil { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index e046d28fdfc6f..b11b727079630 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -96,6 +96,12 @@ type RetryInfo struct { LastRcReadTS uint64 } +// ReuseChunkPool save Alloc object +type ReuseChunkPool struct { + mu sync.Mutex + Alloc chunk.Allocator +} + // Clean does some clean work. func (r *RetryInfo) Clean() { r.autoIncrementIDs.clean() @@ -1292,10 +1298,7 @@ type SessionVars struct { OptPrefixIndexSingleScan bool // ChunkPool Several chunks and columns are cached - ChunkPool struct { - Lock sync.Mutex - Alloc chunk.Allocator - } + ChunkPool ReuseChunkPool // EnableReuseCheck indicates request chunk whether use chunk alloc EnableReuseCheck bool @@ -1304,34 +1307,18 @@ type SessionVars struct { preUseChunkAlloc bool } -// GetNewChunk Attempt to request memory from the chunk pool -// thread safety -func (s *SessionVars) GetNewChunk(fields []*types.FieldType, capacity int) *chunk.Chunk { - //Chunk memory pool is not set - if s.ChunkPool.Alloc == nil { - return chunk.NewChunkWithCapacity(fields, capacity) - } - s.ChunkPool.Lock.Lock() - defer s.ChunkPool.Lock.Unlock() - if s.ChunkPool.Alloc.CheckReuseAllocSize() && (!s.GetUseChunkAlloc()) { - s.StmtCtx.SetUseChunkAlloc() - } - chk := s.ChunkPool.Alloc.Alloc(fields, capacity, capacity) - return chk -} - // GetNewChunkWithCapacity Attempt to request memory from the chunk pool // thread safety -func (s *SessionVars) GetNewChunkWithCapacity(fields []*types.FieldType, capacity int, maxCachesize int) *chunk.Chunk { - if s.ChunkPool.Alloc == nil { +func (s *SessionVars) GetNewChunkWithCapacity(fields []*types.FieldType, capacity int, maxCachesize int, pool chunk.Allocator) *chunk.Chunk { + if pool == nil { return chunk.New(fields, capacity, maxCachesize) } - s.ChunkPool.Lock.Lock() - defer s.ChunkPool.Lock.Unlock() - if s.ChunkPool.Alloc.CheckReuseAllocSize() && (!s.GetUseChunkAlloc()) { + s.ChunkPool.mu.Lock() + defer s.ChunkPool.mu.Unlock() + if pool.CheckReuseAllocSize() && (!s.GetUseChunkAlloc()) { s.StmtCtx.SetUseChunkAlloc() } - chk := s.ChunkPool.Alloc.Alloc(fields, capacity, maxCachesize) + chk := pool.Alloc(fields, capacity, maxCachesize) return chk } @@ -1354,8 +1341,19 @@ func (s *SessionVars) SetAlloc(alloc chunk.Allocator) { } // ClearAlloc indicates stop reuse chunk -func (s *SessionVars) ClearAlloc() { +func (s *SessionVars) ClearAlloc(alloc *chunk.Allocator, b bool) { + if !b { + s.ChunkPool.Alloc = nil + return + } + + // If an error is reported, re-apply for alloc + // Prevent the goroutine left before, affecting the execution of the next sql + // issuse 38918 + s.ChunkPool.mu.Lock() s.ChunkPool.Alloc = nil + s.ChunkPool.mu.Unlock() + *alloc = chunk.NewAllocator() } // GetPreparedStmtByName returns the prepared statement specified by stmtName. @@ -1654,12 +1652,8 @@ func NewSessionVars(hctx HookContext) *SessionVars { ForeignKeyChecks: DefTiDBForeignKeyChecks, HookContext: hctx, EnableReuseCheck: DefTiDBEnableReusechunk, - //useChunkAlloc: DefTiDBUseAlloc, - preUseChunkAlloc: DefTiDBUseAlloc, - ChunkPool: struct { - Lock sync.Mutex - Alloc chunk.Allocator - }{Alloc: nil}, + preUseChunkAlloc: DefTiDBUseAlloc, + ChunkPool: ReuseChunkPool{Alloc: nil}, } vars.KVVars = tikvstore.NewVariables(&vars.Killed) vars.Concurrency = Concurrency{ diff --git a/sessionctx/variable/session_test.go b/sessionctx/variable/session_test.go index 903ecf1bf4fa8..92049902618c2 100644 --- a/sessionctx/variable/session_test.go +++ b/sessionctx/variable/session_test.go @@ -444,10 +444,8 @@ func TestGetReuseChunk(t *testing.T) { require.Nil(t, sessVars.ChunkPool.Alloc) require.False(t, sessVars.GetUseChunkAlloc()) // alloc is nil ,Allocate memory from the system - chk1 := sessVars.GetNewChunk(fieldTypes, 10) + chk1 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10, sessVars.ChunkPool.Alloc) require.NotNil(t, chk1) - chk2 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10) - require.NotNil(t, chk2) chunkReuseMap := make(map[*chunk.Chunk]struct{}, 14) columnReuseMap := make(map[*chunk.Column]struct{}, 14) @@ -461,35 +459,28 @@ func TestGetReuseChunk(t *testing.T) { //tries to apply from the cache initCap := 10 - chk1 = sessVars.GetNewChunk(fieldTypes, initCap) + chk1 = sessVars.GetNewChunkWithCapacity(fieldTypes, initCap, initCap, sessVars.ChunkPool.Alloc) require.NotNil(t, chk1) chunkReuseMap[chk1] = struct{}{} for i := 0; i < chk1.NumCols(); i++ { columnReuseMap[chk1.Column(i)] = struct{}{} } - chk2 = sessVars.GetNewChunkWithCapacity(fieldTypes, initCap, initCap) - require.NotNil(t, chk2) - chunkReuseMap[chk2] = struct{}{} - for i := 0; i < chk2.NumCols(); i++ { - columnReuseMap[chk2.Column(i)] = struct{}{} - } alloc.Reset() - chkres1 := sessVars.GetNewChunk(fieldTypes, 10) + chkres1 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10, sessVars.ChunkPool.Alloc) + require.NotNil(t, chkres1) _, exist := chunkReuseMap[chkres1] require.True(t, exist) for i := 0; i < chkres1.NumCols(); i++ { _, exist := columnReuseMap[chkres1.Column(i)] require.True(t, exist) } - chkres2 := sessVars.GetNewChunkWithCapacity(fieldTypes, 10, 10) - require.NotNil(t, chkres2) - _, exist = chunkReuseMap[chkres2] - require.True(t, exist) - for i := 0; i < chkres2.NumCols(); i++ { - _, exist := columnReuseMap[chkres2.Column(i)] - require.True(t, exist) - } - sessVars.ClearAlloc() + allocpool := variable.ReuseChunkPool{Alloc: alloc} + + sessVars.ClearAlloc(&allocpool.Alloc, false) + require.Equal(t, alloc, allocpool.Alloc) + + sessVars.ClearAlloc(&allocpool.Alloc, true) + require.NotEqual(t, allocpool.Alloc, alloc) require.Nil(t, sessVars.ChunkPool.Alloc) } diff --git a/testkit/testkit.go b/testkit/testkit.go index 6952f4c36d484..25479bc862d96 100644 --- a/testkit/testkit.go +++ b/testkit/testkit.go @@ -115,7 +115,6 @@ func (tk *TestKit) Session() session.Session { // MustExec executes a sql statement and asserts nil error. func (tk *TestKit) MustExec(sql string, args ...interface{}) { defer func() { - tk.Session().GetSessionVars().ClearAlloc() if tk.alloc != nil { tk.alloc.Reset() } @@ -138,7 +137,6 @@ func (tk *TestKit) MustExecWithContext(ctx context.Context, sql string, args ... // If expected result is set it asserts the query result equals expected result. func (tk *TestKit) MustQuery(sql string, args ...interface{}) *Result { defer func() { - tk.Session().GetSessionVars().ClearAlloc() if tk.alloc != nil { tk.alloc.Reset() } @@ -271,7 +269,8 @@ func (tk *TestKit) Exec(sql string, args ...interface{}) (sqlexec.RecordSet, err } // ExecWithContext executes a sql statement using the prepared stmt API -func (tk *TestKit) ExecWithContext(ctx context.Context, sql string, args ...interface{}) (sqlexec.RecordSet, error) { +func (tk *TestKit) ExecWithContext(ctx context.Context, sql string, args ...interface{}) (rs sqlexec.RecordSet, err error) { + defer tk.Session().GetSessionVars().ClearAlloc(&tk.alloc, err != nil) if len(args) == 0 { sc := tk.session.GetSessionVars().StmtCtx prevWarns := sc.GetWarnings() @@ -315,7 +314,7 @@ func (tk *TestKit) ExecWithContext(ctx context.Context, sql string, args ...inte } params := expression.Args2Expressions4Test(args...) tk.Session().GetSessionVars().SetAlloc(tk.alloc) - rs, err := tk.session.ExecutePreparedStmt(ctx, stmtID, params) + rs, err = tk.session.ExecutePreparedStmt(ctx, stmtID, params) if err != nil { return rs, errors.Trace(err) } diff --git a/util/chunk/alloc.go b/util/chunk/alloc.go index 44fbb126a4989..af3385a644389 100644 --- a/util/chunk/alloc.go +++ b/util/chunk/alloc.go @@ -128,9 +128,9 @@ func (a *allocator) Reset() { a.allocated = a.allocated[:0] //column objects and put them to the column allocator for reuse. - for _, pool := range a.columnAlloc.pool { + for id, pool := range a.columnAlloc.pool { for _, col := range pool.allocColumns { - if (len(pool.freeColumns) < a.columnAlloc.freeColumnsPerType) && (!col.avoidReusing) && (cap(col.data) < MaxCachedLen) { + if (len(pool.freeColumns) < a.columnAlloc.freeColumnsPerType) && checkColumnType(id, col) { col.reset() pool.freeColumns = append(pool.freeColumns, col) } @@ -139,6 +139,27 @@ func (a *allocator) Reset() { } } +// checkColumnType check whether the conditions for entering the corresponding queue are met +// column Reset may change type +func checkColumnType(id int, col *Column) bool { + if col.avoidReusing { + return false + } + + if id == varElemLen { + //Take up too much memory, + if cap(col.data) > MaxCachedLen { + return false + } + return col.elemBuf == nil + } + + if col.elemBuf == nil { + return false + } + return id == cap(col.elemBuf) +} + var _ ColumnAllocator = &poolColumnAllocator{} type poolColumnAllocator struct { diff --git a/util/chunk/alloc_test.go b/util/chunk/alloc_test.go index edad5e3008e77..7c09d818d1222 100644 --- a/util/chunk/alloc_test.go +++ b/util/chunk/alloc_test.go @@ -270,3 +270,22 @@ func TestColumnAllocatorLimit(t *testing.T) { alloc = NewAllocator() require.False(t, alloc.CheckReuseAllocSize()) } + +func TestColumnAllocatorCheck(t *testing.T) { + fieldTypes := []*types.FieldType{ + types.NewFieldTypeBuilder().SetType(mysql.TypeFloat).BuildP(), + types.NewFieldTypeBuilder().SetType(mysql.TypeDatetime).BuildP(), + } + InitChunkAllocSize(10, 20) + alloc := NewAllocator() + for i := 0; i < 4; i++ { + alloc.Alloc(fieldTypes, 5, 10) + } + col := alloc.columnAlloc.NewColumn(types.NewFieldTypeBuilder().SetType(mysql.TypeFloat).BuildP(), 10) + col.Reset(types.ETDatetime) + alloc.Reset() + num := alloc.columnAlloc.pool[getFixedLen(types.NewFieldTypeBuilder().SetType(mysql.TypeFloat).BuildP())].Len() + require.Equal(t, num, 4) + num = alloc.columnAlloc.pool[getFixedLen(types.NewFieldTypeBuilder().SetType(mysql.TypeDatetime).BuildP())].Len() + require.Equal(t, num, 4) +} From 18cb721079e96e973bf1fc56b5583bc75455b4ba Mon Sep 17 00:00:00 2001 From: tiancaiamao Date: Tue, 8 Nov 2022 21:33:50 +0800 Subject: [PATCH 16/22] autoid_service,server: handle TLS config of the etcd client (#38975) ref pingcap/tidb#38977 --- autoid_service/BUILD.bazel | 2 ++ autoid_service/autoid.go | 21 ++++++++++++++++++--- server/http_status.go | 2 +- 3 files changed, 21 insertions(+), 4 deletions(-) diff --git a/autoid_service/BUILD.bazel b/autoid_service/BUILD.bazel index 7e29b826e7b29..df3d361d412ed 100644 --- a/autoid_service/BUILD.bazel +++ b/autoid_service/BUILD.bazel @@ -6,6 +6,7 @@ go_library( importpath = "github.com/pingcap/tidb/autoid_service", visibility = ["//visibility:public"], deps = [ + "//config", "//kv", "//meta", "//metrics", @@ -16,6 +17,7 @@ go_library( "@com_github_pingcap_kvproto//pkg/autoid", "@io_etcd_go_etcd_client_v3//:client", "@org_golang_google_grpc//:grpc", + "@org_golang_google_grpc//keepalive", "@org_uber_go_zap//:zap", ], ) diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index 7e0edb7ea3e63..d20c78fd06098 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -16,12 +16,14 @@ package autoid import ( "context" + "crypto/tls" "math" "sync" "time" "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/autoid" + "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" @@ -31,6 +33,7 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" ) var ( @@ -245,10 +248,22 @@ type Service struct { } // New return a Service instance. -func New(selfAddr string, etcdAddr []string, store kv.Storage) *Service { +func New(selfAddr string, etcdAddr []string, store kv.Storage, tlsConfig *tls.Config) *Service { + cfg := config.GetGlobalConfig() + etcdLogCfg := zap.NewProductionConfig() cli, err := clientv3.New(clientv3.Config{ - Endpoints: etcdAddr, - DialTimeout: time.Second, + LogConfig: &etcdLogCfg, + Endpoints: etcdAddr, + AutoSyncInterval: 30 * time.Second, + DialTimeout: 5 * time.Second, + DialOptions: []grpc.DialOption{ + grpc.WithBackoffMaxDelay(time.Second * 3), + grpc.WithKeepaliveParams(keepalive.ClientParameters{ + Time: time.Duration(cfg.TiKVClient.GrpcKeepAliveTime) * time.Second, + Timeout: time.Duration(cfg.TiKVClient.GrpcKeepAliveTimeout) * time.Second, + }), + }, + TLS: tlsConfig, }) if err != nil { panic(err) diff --git a/server/http_status.go b/server/http_status.go index 20c6a7ae550e0..5f02e1762d4ac 100644 --- a/server/http_status.go +++ b/server/http_status.go @@ -477,7 +477,7 @@ func (s *Server) startStatusServerAndRPCServer(serverMux *http.ServeMux) { logutil.BgLogger().Error("tikv store not etcd background", zap.Error(err)) break } - service := autoid.New(s.statusListener.Addr().String(), etcdAddr, store) + service := autoid.New(s.statusListener.Addr().String(), etcdAddr, store, ebd.TLSConfig()) pb.RegisterAutoIDAllocServer(grpcServer, service) s.autoIDService = service break From c498090ac5b3aba74c3cfb87636b7fac88c4d248 Mon Sep 17 00:00:00 2001 From: tangenta Date: Wed, 9 Nov 2022 09:57:50 +0800 Subject: [PATCH 17/22] server: set the keepalive MinTime for gRPC server (#38987) close pingcap/tidb#38983 --- server/rpc_server.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/server/rpc_server.go b/server/rpc_server.go index 893b923d41a77..4dcad639ad4a7 100644 --- a/server/rpc_server.go +++ b/server/rpc_server.go @@ -55,6 +55,12 @@ func NewRPCServer(config *config.Config, dom *domain.Domain, sm util.SessionMana Time: time.Duration(config.Status.GRPCKeepAliveTime) * time.Second, Timeout: time.Duration(config.Status.GRPCKeepAliveTimeout) * time.Second, }), + grpc.KeepaliveEnforcementPolicy(keepalive.EnforcementPolicy{ + // Allow clients send consecutive pings in every 5 seconds. + // The default value of MinTime is 5 minutes, + // which is too long compared with 10 seconds of TiDB's keepalive time. + MinTime: 5 * time.Second, + }), grpc.MaxConcurrentStreams(uint32(config.Status.GRPCConcurrentStreams)), grpc.InitialWindowSize(int32(config.Status.GRPCInitialWindowSize)), grpc.MaxSendMsgSize(config.Status.GRPCMaxSendMsgSize), From 10115e09b779495a8b6b756092d307c072817e47 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 8 Nov 2022 18:33:51 -0800 Subject: [PATCH 18/22] executor/analyzetest: Stabilise analyze table tests due to SyncWaitStatsLoad timeout (#38990) close pingcap/tidb#38882, close pingcap/tidb#38883 --- executor/analyzetest/analyze_test.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/executor/analyzetest/analyze_test.go b/executor/analyzetest/analyze_test.go index d789b6a276fdf..c5043935e1650 100644 --- a/executor/analyzetest/analyze_test.go +++ b/executor/analyzetest/analyze_test.go @@ -975,6 +975,7 @@ func TestSavedAnalyzeOptions(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("create table t(a int, b int, c int, primary key(a), key idx(b))") tk.MustExec("insert into t values (1,1,1),(2,1,2),(3,1,3),(4,1,4),(5,1,5),(6,1,6),(7,7,7),(8,8,8),(9,9,9)") @@ -1062,6 +1063,7 @@ func TestSavedPartitionAnalyzeOptions(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") createTable := `CREATE TABLE t (a int, b int, c varchar(10), primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( @@ -2580,6 +2582,7 @@ func TestAnalyzePartitionTableWithDynamicMode(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic'") createTable := `CREATE TABLE t (a int, b int, c varchar(10), d int, primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( @@ -2673,6 +2676,7 @@ func TestAnalyzePartitionTableStaticToDynamic(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("set @@session.tidb_partition_prune_mode = 'static'") createTable := `CREATE TABLE t (a int, b int, c varchar(10), d int, primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( @@ -2864,6 +2868,7 @@ func TestAnalyzePartitionStaticToDynamic(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 2") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test createTable := `CREATE TABLE t (a int, b int, c varchar(10), d int, primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( PARTITION p0 VALUES LESS THAN (10), @@ -2943,6 +2948,7 @@ func TestAnalyzePartitionUnderV1Dynamic(t *testing.T) { tk.MustExec("use test") tk.MustExec("set @@session.tidb_analyze_version = 1") + tk.MustExec("set @@session.tidb_stats_load_sync_wait = 20000") // to stabilise test tk.MustExec("set @@session.tidb_partition_prune_mode = 'dynamic'") createTable := `CREATE TABLE t (a int, b int, c varchar(10), d int, primary key(a), index idx(b)) PARTITION BY RANGE ( a ) ( From ce629152894f7cfa1a9cd46c710cb7936009f7b4 Mon Sep 17 00:00:00 2001 From: Yuanjia Zhang Date: Wed, 9 Nov 2022 10:55:50 +0800 Subject: [PATCH 19/22] planner: update the scan-row-size calculation formula in model2 (#38968) ref pingcap/tidb#35240 --- planner/core/plan_cost_ver1.go | 4 +- planner/core/plan_cost_ver2.go | 32 ++-- planner/core/plan_cost_ver2_test.go | 44 ++++- planner/core/task.go | 5 +- .../core/testdata/enforce_mpp_suite_out.json | 96 +++++----- .../core/testdata/integration_suite_out.json | 180 +++++++++--------- .../core/testdata/join_reorder_suite_out.json | 28 +-- 7 files changed, 217 insertions(+), 172 deletions(-) diff --git a/planner/core/plan_cost_ver1.go b/planner/core/plan_cost_ver1.go index 0cd8c5eabd473..d1a7e6e95a9c5 100644 --- a/planner/core/plan_cost_ver1.go +++ b/planner/core/plan_cost_ver1.go @@ -852,7 +852,7 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, isMPP bool, costFlag uint sessVars := p.ctx.GetSessionVars() oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() memQuota := sessVars.MemTracker.GetBytesLimit() // sessVars.MemQuotaQuery && hint - rowSize := getAvgRowSize(build.statsInfo(), build.Schema()) + rowSize := getAvgRowSize(build.statsInfo(), build.Schema().Columns) spill := oomUseTmpStorage && memQuota > 0 && rowSize*buildCnt > float64(memQuota) && p.storeTp != kv.TiFlash // Cost of building hash table. cpuFactor := sessVars.GetCPUFactor() @@ -1049,7 +1049,7 @@ func (p *PhysicalSort) GetCost(count float64, schema *expression.Schema) float64 oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() memQuota := sessVars.MemTracker.GetBytesLimit() // sessVars.MemQuotaQuery && hint - rowSize := getAvgRowSize(p.statsInfo(), schema) + rowSize := getAvgRowSize(p.statsInfo(), schema.Columns) spill := oomUseTmpStorage && memQuota > 0 && rowSize*count > float64(memQuota) diskCost := count * sessVars.GetDiskFactor() * rowSize if !spill { diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 1afa6237386ae..b717b8580d2b5 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -115,7 +115,7 @@ func (p *PhysicalIndexScan) getPlanCostVer2(taskType property.TaskType, option * } rows := getCardinality(p, option.CostFlag) - rowSize := math.Max(p.getScanRowSize(), 2.0) + rowSize := math.Max(getAvgRowSize(p.stats, p.schema.Columns), 2.0) // consider all index columns scanFactor := getTaskScanFactorVer2(p, taskType) p.planCostVer2 = scanCostVer2(option, rows, rowSize, scanFactor) @@ -132,7 +132,13 @@ func (p *PhysicalTableScan) getPlanCostVer2(taskType property.TaskType, option * } rows := getCardinality(p, option.CostFlag) - rowSize := math.Max(p.getScanRowSize(), 2.0) + var rowSize float64 + if p.StoreType == kv.TiKV { + rowSize = getAvgRowSize(p.stats, p.tblCols) // consider all columns if TiKV + } else { // TiFlash + rowSize = getAvgRowSize(p.stats, p.schema.Columns) + } + rowSize = math.Max(rowSize, 2.0) scanFactor := getTaskScanFactorVer2(p, taskType) p.planCostVer2 = scanCostVer2(option, rows, rowSize, scanFactor) @@ -155,7 +161,7 @@ func (p *PhysicalIndexReader) getPlanCostVer2(taskType property.TaskType, option } rows := getCardinality(p.indexPlan, option.CostFlag) - rowSize := getAvgRowSize(p.indexPlan.Stats(), p.indexPlan.Schema()) + rowSize := getAvgRowSize(p.indexPlan.Stats(), p.indexPlan.Schema().Columns) netFactor := getTaskNetFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) @@ -180,7 +186,7 @@ func (p *PhysicalTableReader) getPlanCostVer2(taskType property.TaskType, option } rows := getCardinality(p.tablePlan, option.CostFlag) - rowSize := getAvgRowSize(p.tablePlan.Stats(), p.tablePlan.Schema()) + rowSize := getAvgRowSize(p.tablePlan.Stats(), p.tablePlan.Schema().Columns) netFactor := getTaskNetFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) childType := property.CopSingleReadTaskType @@ -284,7 +290,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer2(taskType property.TaskType, o var tableSideCost costVer2 if tablePath := p.tablePlan; tablePath != nil { rows := getCardinality(tablePath, option.CostFlag) - rowSize := getAvgRowSize(tablePath.Stats(), tablePath.Schema()) + rowSize := getAvgRowSize(tablePath.Stats(), tablePath.Schema().Columns) tableNetCost := netCostVer2(option, rows, rowSize, netFactor) tableChildCost, err := tablePath.getPlanCostVer2(taskType, option) @@ -297,7 +303,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer2(taskType property.TaskType, o indexSideCost := make([]costVer2, 0, len(p.partialPlans)) for _, indexPath := range p.partialPlans { rows := getCardinality(indexPath, option.CostFlag) - rowSize := getAvgRowSize(indexPath.Stats(), indexPath.Schema()) + rowSize := getAvgRowSize(indexPath.Stats(), indexPath.Schema().Columns) indexNetCost := netCostVer2(option, rows, rowSize, netFactor) indexChildCost, err := indexPath.getPlanCostVer2(taskType, option) @@ -329,7 +335,7 @@ func (p *PhysicalSort) getPlanCostVer2(taskType property.TaskType, option *PlanC } rows := math.Max(getCardinality(p.children[0], option.CostFlag), 1) - rowSize := getAvgRowSize(p.statsInfo(), p.Schema()) + rowSize := getAvgRowSize(p.statsInfo(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) diskFactor := defaultVer2Factors.TiDBDisk @@ -378,7 +384,7 @@ func (p *PhysicalTopN) getPlanCostVer2(taskType property.TaskType, option *PlanC rows := getCardinality(p.children[0], option.CostFlag) N := math.Max(1, float64(p.Count+p.Offset)) - rowSize := getAvgRowSize(p.statsInfo(), p.Schema()) + rowSize := getAvgRowSize(p.statsInfo(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) @@ -429,7 +435,7 @@ func (p *PhysicalHashAgg) getPlanCostVer2(taskType property.TaskType, option *Pl inputRows := getCardinality(p.children[0], option.CostFlag) outputRows := getCardinality(p, option.CostFlag) - outputRowSize := getAvgRowSize(p.Stats(), p.Schema()) + outputRowSize := getAvgRowSize(p.Stats(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) concurrency := float64(p.ctx.GetSessionVars().HashAggFinalConcurrency()) @@ -501,7 +507,7 @@ func (p *PhysicalHashJoin) getPlanCostVer2(taskType property.TaskType, option *P } buildRows := getCardinality(build, option.CostFlag) probeRows := getCardinality(probe, option.CostFlag) - buildRowSize := getAvgRowSize(build.Stats(), build.Schema()) + buildRowSize := getAvgRowSize(build.Stats(), build.Schema().Columns) tidbConcurrency := float64(p.Concurrency) mppConcurrency := float64(3) // TODO: remove this empirical value cpuFactor := getTaskCPUFactorVer2(p, taskType) @@ -645,7 +651,7 @@ func (p *PhysicalExchangeReceiver) getPlanCostVer2(taskType property.TaskType, o } rows := getCardinality(p, option.CostFlag) - rowSize := getAvgRowSize(p.stats, p.Schema()) + rowSize := getAvgRowSize(p.stats, p.Schema().Columns) netFactor := getTaskNetFactorVer2(p, taskType) isBCast := false if sender, ok := p.children[0].(*PhysicalExchangeSender); ok { @@ -678,7 +684,7 @@ func (p *PointGetPlan) getPlanCostVer2(taskType property.TaskType, option *PlanC p.planCostInit = true return zeroCostVer2, nil } - rowSize := getAvgRowSize(p.stats, p.schema) + rowSize := getAvgRowSize(p.stats, p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) p.planCostVer2 = netCostVer2(option, 1, rowSize, netFactor) @@ -698,7 +704,7 @@ func (p *BatchPointGetPlan) getPlanCostVer2(taskType property.TaskType, option * return zeroCostVer2, nil } rows := getCardinality(p, option.CostFlag) - rowSize := getAvgRowSize(p.stats, p.schema) + rowSize := getAvgRowSize(p.stats, p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) p.planCostVer2 = netCostVer2(option, rows, rowSize, netFactor) diff --git a/planner/core/plan_cost_ver2_test.go b/planner/core/plan_cost_ver2_test.go index 0043aa0be898e..27b5b913980e4 100644 --- a/planner/core/plan_cost_ver2_test.go +++ b/planner/core/plan_cost_ver2_test.go @@ -142,15 +142,55 @@ func TestCostModelShowFormula(t *testing.T) { actual := make([][]interface{}, 0, len(plan)) for _, row := range plan { actual = append(actual, []interface{}{row[0], row[3]}) // id,costFormula - fmt.Println(actual) } require.Equal(t, actual, [][]interface{}{ {"TableReader_7", "((Selection_6) + (net(2*rowsize(16)*tidb_kv_net_factor(3.96))))/15"}, {"└─Selection_6", "(cpu(3*filters(1)*tikv_cpu_factor(49.9))) + (TableFullScan_5)"}, - {" └─TableFullScan_5", "scan(3*logrowsize(29)*tikv_scan_factor(40.7))"}, + {" └─TableFullScan_5", "scan(3*logrowsize(32)*tikv_scan_factor(40.7))"}, }) } +func TestCostModelVer2ScanRowSize(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec(`create table t (pk int, a int, b int, c int, d int, primary key(pk), index ab(a, b), index abc(a, b, c))`) + tk.MustExec("insert into t values (1, 1, 1, 1, 1)") + tk.MustExec(`set @@tidb_cost_model_version=2`) + + cases := []struct { + query string + scanFormula string + }{ + // index scan row-size on idx_ab is always equal to row-size(index_ab) + {"select a from t use index(ab) where a=1", "scan(1*logrowsize(32)*tikv_scan_factor(40.7))"}, + {"select a, b from t use index(ab) where a=1", "scan(1*logrowsize(32)*tikv_scan_factor(40.7))"}, + {"select b from t use index(ab) where a=1 and b=1", "scan(1*logrowsize(32)*tikv_scan_factor(40.7))"}, + // index scan row-size on idx_abc is always equal to row-size(index_abc) + {"select a from t use index(abc) where a=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, + {"select a from t use index(abc) where a=1 and b=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, + {"select a, b from t use index(abc) where a=1 and b=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, + {"select a, b, c from t use index(abc) where a=1 and b=1 and c=1", "scan(1*logrowsize(48)*tikv_scan_factor(40.7))"}, + // table scan row-size is always equal to row-size(*) + {"select a from t use index(primary) where a=1", "scan(1*logrowsize(80)*tikv_scan_factor(40.7))"}, + {"select a, d from t use index(primary) where a=1", "scan(1*logrowsize(80)*tikv_scan_factor(40.7))"}, + {"select * from t use index(primary) where a=1", "scan(1*logrowsize(80)*tikv_scan_factor(40.7))"}, + } + for _, c := range cases { + rs := tk.MustQuery("explain analyze format=true_card_cost " + c.query).Rows() + scan := rs[len(rs)-1] + formula := scan[3] + require.Equal(t, formula, c.scanFormula) + } + + tk.MustQuery("explain select a from t where a=1").Check(testkit.Rows( + `IndexReader_6 10.00 root index:IndexRangeScan_5`, // use idx_ab automatically since it has the smallest row-size in all access paths. + `└─IndexRangeScan_5 10.00 cop[tikv] table:t, index:ab(a, b) range:[1,1], keep order:false, stats:pseudo`)) + tk.MustQuery("explain select a, b, c from t where a=1").Check(testkit.Rows( + `IndexReader_6 10.00 root index:IndexRangeScan_5`, // use idx_abc automatically + `└─IndexRangeScan_5 10.00 cop[tikv] table:t, index:abc(a, b, c) range:[1,1], keep order:false, stats:pseudo`)) +} + func TestCostModelTraceVer2(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/planner/core/task.go b/planner/core/task.go index 44295ca6b4def..11d502568d3ae 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -297,12 +297,11 @@ func (p *PhysicalIndexJoin) attach2Task(tasks ...task) task { return t } -func getAvgRowSize(stats *property.StatsInfo, schema *expression.Schema) (size float64) { +func getAvgRowSize(stats *property.StatsInfo, cols []*expression.Column) (size float64) { if stats.HistColl != nil { - size = stats.HistColl.GetAvgRowSizeListInDisk(schema.Columns) + size = stats.HistColl.GetAvgRowSizeListInDisk(cols) } else { // Estimate using just the type info. - cols := schema.Columns for _, col := range cols { size += float64(chunk.EstimateTypeWidth(col.GetType())) } diff --git a/planner/core/testdata/enforce_mpp_suite_out.json b/planner/core/testdata/enforce_mpp_suite_out.json index 20f349233aed4..3f8a2667a4dc2 100644 --- a/planner/core/testdata/enforce_mpp_suite_out.json +++ b/planner/core/testdata/enforce_mpp_suite_out.json @@ -31,30 +31,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "StreamAgg_24 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_25 1.00 167.19 root index:StreamAgg_9", - " └─StreamAgg_9 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_23 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_24 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_25 1.00 143.91 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_23 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_17 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_18 1.00 167.19 root index:StreamAgg_9", - " └─StreamAgg_9 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_16 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_17 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_18 1.00 143.91 root index:StreamAgg_9", + " └─StreamAgg_9 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_16 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_10 1.00 65360.67 root funcs:count(1)->Column#4", - "└─TableReader_24 10.00 64861.67 root data:Selection_23", - " └─Selection_23 10.00 972291.38 cop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_22 10000.00 948291.38 cop[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_10 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_24 10.00 63508.91 root data:Selection_23", + " └─Selection_23 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_22 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -71,30 +71,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "StreamAgg_31 1.00 217.09 root funcs:count(Column#7)->Column#4", - "└─IndexReader_32 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_30 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_19 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_20 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_18 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_12 1.00 65360.67 root funcs:count(1)->Column#4", - "└─TableReader_31 10.00 64861.67 root data:Selection_30", - " └─Selection_30 10.00 972291.38 cop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_29 10000.00 948291.38 cop[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_12 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_31 10.00 63508.91 root data:Selection_30", + " └─Selection_30 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_29 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -106,30 +106,30 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "StreamAgg_31 1.00 217.09 root funcs:count(Column#7)->Column#4", - "└─IndexReader_32 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_30 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_19 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_20 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_18 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tiflash[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_12 1.00 65360.67 root funcs:count(1)->Column#4", - "└─TableReader_31 10.00 64861.67 root data:Selection_30", - " └─Selection_30 10.00 972291.38 cop[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_29 10000.00 948291.38 cop[tiflash] table:t keep order:false, stats:pseudo" + "StreamAgg_12 1.00 64007.91 root funcs:count(1)->Column#4", + "└─TableReader_31 10.00 63508.91 root data:Selection_30", + " └─Selection_30 10.00 952000.00 cop[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_29 10000.00 928000.00 cop[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null }, @@ -141,20 +141,20 @@ { "SQL": "explain format='verbose' select count(*) from t where a=1", "Plan": [ - "StreamAgg_31 1.00 217.09 root funcs:count(Column#7)->Column#4", - "└─IndexReader_32 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#7", - " └─IndexRangeScan_30 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_31 1.00 193.81 root funcs:count(Column#7)->Column#4", + "└─IndexReader_32 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#7", + " └─IndexRangeScan_30 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": null }, { "SQL": "explain format='verbose' select /*+ read_from_storage(tikv[t]) */ count(*) from t where a=1", "Plan": [ - "StreamAgg_19 1.00 217.09 root funcs:count(Column#6)->Column#4", - "└─IndexReader_20 1.00 167.19 root index:StreamAgg_11", - " └─StreamAgg_11 1.00 2476.20 cop[tikv] funcs:count(1)->Column#6", - " └─IndexRangeScan_18 10.00 1977.20 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" + "StreamAgg_19 1.00 193.81 root funcs:count(Column#6)->Column#4", + "└─IndexReader_20 1.00 143.91 root index:StreamAgg_11", + " └─StreamAgg_11 1.00 2127.00 cop[tikv] funcs:count(1)->Column#6", + " └─IndexRangeScan_18 10.00 1628.00 cop[tikv] table:t, index:idx(a) range:[1,1], keep order:false, stats:pseudo" ], "Warn": [ "MPP mode may be blocked because you have set a hint to read table `t` from TiKV." @@ -165,10 +165,10 @@ "Plan": [ "HashAgg_22 1.00 39930.30 root funcs:count(Column#6)->Column#4", "└─TableReader_24 1.00 0.00 root data:ExchangeSender_23", - " └─ExchangeSender_23 1.00 974211.94 mpp[tiflash] ExchangeType: PassThrough", - " └─HashAgg_10 1.00 974211.94 mpp[tiflash] funcs:count(1)->Column#6", - " └─Selection_21 10.00 972291.38 mpp[tiflash] eq(test.t.a, 1)", - " └─TableFullScan_20 10000.00 948291.38 mpp[tiflash] table:t keep order:false, stats:pseudo" + " └─ExchangeSender_23 1.00 953920.56 mpp[tiflash] ExchangeType: PassThrough", + " └─HashAgg_10 1.00 953920.56 mpp[tiflash] funcs:count(1)->Column#6", + " └─Selection_21 10.00 952000.00 mpp[tiflash] eq(test.t.a, 1)", + " └─TableFullScan_20 10000.00 928000.00 mpp[tiflash] table:t keep order:false, stats:pseudo" ], "Warn": null } diff --git a/planner/core/testdata/integration_suite_out.json b/planner/core/testdata/integration_suite_out.json index 0a04a8f55f24d..e223d6604e3f1 100644 --- a/planner/core/testdata/integration_suite_out.json +++ b/planner/core/testdata/integration_suite_out.json @@ -1861,8 +1861,8 @@ { "SQL": "select f, g from t1 where f = 2 and g > 3", "Plan": [ - "IndexReader_6 33.33 752.77 root index:IndexRangeScan_5", - "└─IndexRangeScan_5 33.33 7067.49 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" + "IndexReader_6 33.33 733.82 root index:IndexRangeScan_5", + "└─IndexRangeScan_5 33.33 6783.33 cop[tikv] table:t1, index:f_g(f, g) range:(2 3,2 +inf], keep order:false, stats:pseudo" ], "Warnings": [ "Note 1105 unique index f_g of t1 is selected since the path only fetches limited number of rows with single scan" @@ -2486,65 +2486,65 @@ { "SQL": "explain format = 'verbose' select count(*) from t3", "Plan": [ - "StreamAgg_20 1.00 99.31 root funcs:count(Column#9)->Column#4", - "└─TableReader_21 1.00 49.41 root data:StreamAgg_8", - " └─StreamAgg_8 1.00 709.52 cop[tikv] funcs:count(1)->Column#9", - " └─TableFullScan_18 3.00 559.82 cop[tikv] table:t3 keep order:false" + "StreamAgg_20 1.00 102.69 root funcs:count(Column#9)->Column#4", + "└─IndexReader_21 1.00 52.79 root index:StreamAgg_8", + " └─StreamAgg_8 1.00 760.20 cop[tikv] funcs:count(1)->Column#9", + " └─IndexFullScan_19 3.00 610.50 cop[tikv] table:t3, index:c(b) keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t2", "Plan": [ - "StreamAgg_26 1.00 99.31 root funcs:count(Column#7)->Column#4", - "└─TableReader_27 1.00 49.41 root data:StreamAgg_10", - " └─StreamAgg_10 1.00 709.52 cop[tikv] funcs:count(1)->Column#7", - " └─TableFullScan_24 3.00 559.82 cop[tikv] table:t2 keep order:false" + "StreamAgg_26 1.00 107.45 root funcs:count(Column#7)->Column#4", + "└─TableReader_27 1.00 57.55 root data:StreamAgg_10", + " └─StreamAgg_10 1.00 831.62 cop[tikv] funcs:count(1)->Column#7", + " └─TableFullScan_24 3.00 681.92 cop[tikv] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by a", "Plan": [ - "Sort_4 3.00 310.13 root test.t3.a", - "└─TableReader_8 3.00 62.67 root data:TableFullScan_7", - " └─TableFullScan_7 3.00 559.82 cop[tikv] table:t3 keep order:false" + "Sort_4 3.00 318.27 root test.t3.a", + "└─TableReader_8 3.00 70.81 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 681.92 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by b", "Plan": [ - "Sort_4 3.00 310.13 root test.t3.b", - "└─TableReader_8 3.00 62.67 root data:TableFullScan_7", - " └─TableFullScan_7 3.00 559.82 cop[tikv] table:t3 keep order:false" + "Sort_4 3.00 318.27 root test.t3.b", + "└─TableReader_8 3.00 70.81 root data:TableFullScan_7", + " └─TableFullScan_7 3.00 681.92 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by a limit 1", "Plan": [ - "TopN_7 1.00 44.96 root test.t3.a, offset:0, count:1", - "└─TableReader_16 1.00 41.76 root data:TopN_15", - " └─TopN_15 1.00 563.02 cop[tikv] test.t3.a, offset:0, count:1", - " └─TableFullScan_14 3.00 559.82 cop[tikv] table:t3 keep order:false" + "TopN_7 1.00 53.10 root test.t3.a, offset:0, count:1", + "└─TableReader_16 1.00 49.90 root data:TopN_15", + " └─TopN_15 1.00 685.12 cop[tikv] test.t3.a, offset:0, count:1", + " └─TableFullScan_14 3.00 681.92 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select * from t3 order by b limit 1", "Plan": [ - "TopN_7 1.00 44.96 root test.t3.b, offset:0, count:1", - "└─TableReader_16 1.00 41.76 root data:TopN_15", - " └─TopN_15 1.00 563.02 cop[tikv] test.t3.b, offset:0, count:1", - " └─TableFullScan_14 3.00 559.82 cop[tikv] table:t3 keep order:false" + "TopN_7 1.00 53.10 root test.t3.b, offset:0, count:1", + "└─TableReader_16 1.00 49.90 root data:TopN_15", + " └─TopN_15 1.00 685.12 cop[tikv] test.t3.b, offset:0, count:1", + " └─TableFullScan_14 3.00 681.92 cop[tikv] table:t3 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t2 group by a", "Plan": [ - "TableReader_44 3.00 25896.85 root data:ExchangeSender_43", - "└─ExchangeSender_43 3.00 388400.01 mpp[tiflash] ExchangeType: PassThrough", - " └─Projection_38 3.00 388400.00 mpp[tiflash] Column#4", - " └─HashAgg_36 3.00 388399.98 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", - " └─ExchangeReceiver_22 3.00 385507.26 mpp[tiflash] ", - " └─ExchangeSender_21 3.00 385459.26 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", - " └─TableFullScan_20 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" + "TableReader_44 3.00 31142.18 root data:ExchangeSender_43", + "└─ExchangeSender_43 3.00 467079.95 mpp[tiflash] ExchangeType: PassThrough", + " └─Projection_38 3.00 467079.93 mpp[tiflash] Column#4", + " └─HashAgg_36 3.00 467079.92 mpp[tiflash] group by:test.t2.a, funcs:count(1)->Column#4", + " └─ExchangeReceiver_22 3.00 464187.20 mpp[tiflash] ", + " └─ExchangeSender_21 3.00 464139.20 mpp[tiflash] ExchangeType: HashPartition, Hash Cols: [name: test.t2.a, collate: binary]", + " └─TableFullScan_20 3.00 464139.20 mpp[tiflash] table:t2 keep order:false" ] }, { @@ -2567,90 +2567,90 @@ { "SQL": "explain format = 'verbose' select count(*) from t2 where a = 0", "Plan": [ - "StreamAgg_12 1.00 47.30 root funcs:count(1)->Column#4", - "└─TableReader_21 0.00 47.30 root data:Selection_20", - " └─Selection_20 0.00 709.52 cop[tikv] eq(test.t2.a, 0)", - " └─TableFullScan_19 3.00 559.82 cop[tikv] table:t2 keep order:false" + "StreamAgg_12 1.00 55.44 root funcs:count(1)->Column#4", + "└─TableReader_21 0.00 55.44 root data:Selection_20", + " └─Selection_20 0.00 831.62 cop[tikv] eq(test.t2.a, 0)", + " └─TableFullScan_19 3.00 681.92 cop[tikv] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select count(*) from t3 t join t3 on t.a = t3.b", "Plan": [ - "StreamAgg_10 1.00 630.77 root funcs:count(1)->Column#7", - "└─HashJoin_40 3.00 481.07 root inner join, equal:[eq(test.t3.a, test.t3.b)]", - " ├─IndexReader_28(Build) 3.00 52.22 root index:IndexFullScan_27", - " │ └─IndexFullScan_27 3.00 593.16 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_26(Probe) 3.00 59.97 root data:Selection_25", - " └─Selection_25 3.00 709.52 cop[tikv] not(isnull(test.t3.a))", - " └─TableFullScan_24 3.00 559.82 cop[tikv] table:t keep order:false" + "StreamAgg_10 1.00 631.93 root funcs:count(1)->Column#7", + "└─HashJoin_40 3.00 482.23 root inner join, equal:[eq(test.t3.a, test.t3.b)]", + " ├─IndexReader_28(Build) 3.00 45.23 root index:IndexFullScan_27", + " │ └─IndexFullScan_27 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_26(Probe) 3.00 68.11 root data:Selection_25", + " └─Selection_25 3.00 831.62 cop[tikv] not(isnull(test.t3.a))", + " └─TableFullScan_24 3.00 681.92 cop[tikv] table:t keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "StreamAgg_15 1.00 51562.56 root funcs:count(1)->Column#7", - "└─TableReader_41 3.00 51412.86 root data:ExchangeSender_40", - " └─ExchangeSender_40 3.00 771087.32 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin_37 3.00 771087.32 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_22(Build) 3.00 385610.46 mpp[tiflash] ", - " │ └─ExchangeSender_21 3.00 385466.46 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection_20 3.00 385466.46 mpp[tiflash] not(isnull(test.t1.a))", - " │ └─TableFullScan_19 3.00 385459.26 mpp[tiflash] table:t1 keep order:false", - " └─Selection_24(Probe) 3.00 385466.46 mpp[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_23 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" + "StreamAgg_15 1.00 62053.22 root funcs:count(1)->Column#7", + "└─TableReader_41 3.00 61903.52 root data:ExchangeSender_40", + " └─ExchangeSender_40 3.00 928447.20 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_37 3.00 928447.20 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_22(Build) 3.00 464290.40 mpp[tiflash] ", + " │ └─ExchangeSender_21 3.00 464146.40 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_20 3.00 464146.40 mpp[tiflash] not(isnull(test.t1.a))", + " │ └─TableFullScan_19 3.00 464139.20 mpp[tiflash] table:t1 keep order:false", + " └─Selection_24(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_23 3.00 464139.20 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a join t3 on t1.b = t3.b", "Plan": [ - "StreamAgg_15 1.00 54032.05 root funcs:count(1)->Column#10", - "└─HashJoin_59 3.00 53882.35 root inner join, equal:[eq(test.t1.b, test.t3.b)]", - " ├─IndexReader_47(Build) 3.00 52.22 root index:IndexFullScan_46", - " │ └─IndexFullScan_46 3.00 593.16 cop[tikv] table:t3, index:c(b) keep order:false", - " └─TableReader_39(Probe) 3.00 53461.26 root data:ExchangeSender_38", - " └─ExchangeSender_38 3.00 801760.47 mpp[tiflash] ExchangeType: PassThrough", - " └─HashJoin_29 3.00 801760.47 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", - " ├─ExchangeReceiver_35(Build) 3.00 416282.81 mpp[tiflash] ", - " │ └─ExchangeSender_34 3.00 415994.81 mpp[tiflash] ExchangeType: Broadcast", - " │ └─Selection_33 3.00 415994.81 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", - " │ └─TableFullScan_32 3.00 415980.41 mpp[tiflash] table:t1 keep order:false", - " └─Selection_37(Probe) 3.00 385466.46 mpp[tiflash] not(isnull(test.t2.a))", - " └─TableFullScan_36 3.00 385459.26 mpp[tiflash] table:t2 keep order:false" + "StreamAgg_15 1.00 70216.64 root funcs:count(1)->Column#10", + "└─HashJoin_59 3.00 70066.94 root inner join, equal:[eq(test.t1.b, test.t3.b)]", + " ├─IndexReader_47(Build) 3.00 45.23 root index:IndexFullScan_46", + " │ └─IndexFullScan_46 3.00 488.40 cop[tikv] table:t3, index:c(b) keep order:false", + " └─TableReader_39(Probe) 3.00 69652.83 root data:ExchangeSender_38", + " └─ExchangeSender_38 3.00 1044634.00 mpp[tiflash] ExchangeType: PassThrough", + " └─HashJoin_29 3.00 1044634.00 mpp[tiflash] inner join, equal:[eq(test.t1.a, test.t2.a)]", + " ├─ExchangeReceiver_35(Build) 3.00 580476.40 mpp[tiflash] ", + " │ └─ExchangeSender_34 3.00 580188.40 mpp[tiflash] ExchangeType: Broadcast", + " │ └─Selection_33 3.00 580188.40 mpp[tiflash] not(isnull(test.t1.a)), not(isnull(test.t1.b))", + " │ └─TableFullScan_32 3.00 580174.00 mpp[tiflash] table:t1 keep order:false", + " └─Selection_37(Probe) 3.00 464146.40 mpp[tiflash] not(isnull(test.t2.a))", + " └─TableFullScan_36 3.00 464139.20 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select (2) in (select /*+ read_from_storage(tiflash[t1]) */ count(*) from t1) from (select t.b < (select /*+ read_from_storage(tiflash[t2]) */ t.b from t2 limit 1 ) from t3 t) t", "Plan": [ - "HashJoin_19 3.00 110724.79 root CARTESIAN left outer semi join", - "├─Selection_38(Build) 0.80 24733.71 root eq(2, Column#18)", - "│ └─StreamAgg_45 1.00 24683.81 root funcs:count(1)->Column#18", - "│ └─TableReader_59 3.00 24534.11 root data:TableFullScan_58", - "│ └─TableFullScan_58 3.00 367821.61 cop[tiflash] table:t1 keep order:false", - "└─Projection_20(Probe) 3.00 85919.94 root 1->Column#28", - " └─Apply_22 3.00 85919.64 root CARTESIAN left outer join", - " ├─TableReader_24(Build) 3.00 49.99 root data:TableFullScan_23", - " │ └─TableFullScan_23 3.00 559.82 cop[tikv] table:t keep order:false", - " └─Projection_27(Probe) 3.00 28623.22 root 1->Column#26", - " └─Limit_30 3.00 28623.12 root offset:0, count:1", - " └─TableReader_37 3.00 28623.12 root data:ExchangeSender_36", - " └─ExchangeSender_36 3.00 429293.93 mpp[tiflash] ExchangeType: PassThrough", - " └─Limit_35 3.00 429293.93 mpp[tiflash] offset:0, count:1", - " └─TableFullScan_34 3.00 429293.93 mpp[tiflash] table:t2 keep order:false" + "HashJoin_19 3.00 160879.40 root CARTESIAN left outer semi join", + "├─Selection_38(Build) 0.80 31154.89 root eq(2, Column#18)", + "│ └─StreamAgg_45 1.00 31104.99 root funcs:count(1)->Column#18", + "│ └─TableReader_59 3.00 30955.29 root data:TableFullScan_58", + "│ └─TableFullScan_58 3.00 464139.20 cop[tiflash] table:t1 keep order:false", + "└─Projection_20(Probe) 3.00 129653.38 root 1->Column#28", + " └─Apply_22 3.00 129653.08 root CARTESIAN left outer join", + " ├─TableReader_24(Build) 3.00 58.13 root data:TableFullScan_23", + " │ └─TableFullScan_23 3.00 681.92 cop[tikv] table:t keep order:false", + " └─Projection_27(Probe) 3.00 43198.32 root 1->Column#26", + " └─Limit_30 3.00 43198.22 root offset:0, count:1", + " └─TableReader_37 3.00 43198.22 root data:ExchangeSender_36", + " └─ExchangeSender_36 3.00 647920.44 mpp[tiflash] ExchangeType: PassThrough", + " └─Limit_35 3.00 647920.44 mpp[tiflash] offset:0, count:1", + " └─TableFullScan_34 3.00 647920.44 mpp[tiflash] table:t2 keep order:false" ] }, { "SQL": "explain format = 'verbose' select /*+ merge_join(t1), read_from_storage(tiflash[t1, t2]) */ count(*) from t1 join t2 on t1.a = t2.a", "Plan": [ - "StreamAgg_14 1.00 52067.30 root funcs:count(1)->Column#7", - "└─MergeJoin_26 3.00 51917.60 root inner join, left key:test.t1.a, right key:test.t2.a", - " ├─Sort_24(Build) 3.00 25957.31 root test.t2.a", - " │ └─TableReader_23 3.00 25710.44 root data:Selection_22", - " │ └─Selection_22 3.00 385466.46 cop[tiflash] not(isnull(test.t2.a))", - " │ └─TableFullScan_21 3.00 385459.26 cop[tiflash] table:t2 keep order:false", - " └─Sort_20(Probe) 3.00 25957.31 root test.t1.a", - " └─TableReader_19 3.00 25710.44 root data:Selection_18", - " └─Selection_18 3.00 385466.46 cop[tiflash] not(isnull(test.t1.a))", - " └─TableFullScan_17 3.00 385459.26 cop[tiflash] table:t1 keep order:false" + "StreamAgg_14 1.00 62557.96 root funcs:count(1)->Column#7", + "└─MergeJoin_26 3.00 62408.26 root inner join, left key:test.t1.a, right key:test.t2.a", + " ├─Sort_24(Build) 3.00 31202.63 root test.t2.a", + " │ └─TableReader_23 3.00 30955.77 root data:Selection_22", + " │ └─Selection_22 3.00 464146.40 cop[tiflash] not(isnull(test.t2.a))", + " │ └─TableFullScan_21 3.00 464139.20 cop[tiflash] table:t2 keep order:false", + " └─Sort_20(Probe) 3.00 31202.63 root test.t1.a", + " └─TableReader_19 3.00 30955.77 root data:Selection_18", + " └─Selection_18 3.00 464146.40 cop[tiflash] not(isnull(test.t1.a))", + " └─TableFullScan_17 3.00 464139.20 cop[tiflash] table:t1 keep order:false" ] } ] diff --git a/planner/core/testdata/join_reorder_suite_out.json b/planner/core/testdata/join_reorder_suite_out.json index 829a2683b4775..e17cbae1f3b3d 100644 --- a/planner/core/testdata/join_reorder_suite_out.json +++ b/planner/core/testdata/join_reorder_suite_out.json @@ -6333,8 +6333,8 @@ "SQL": "select /*+ straight_join() */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7394,8 +7394,8 @@ "SQL": "select /*+ leading(t4) */ * from t1 join t2 on t1.a=t2.a join t4 on t1.b = t4.b where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 12475.01 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─Projection(Probe) 15593.77 root test.t1.a, test.t1.b, test.t2.a, test.t2.b, test.t4.a, test.t4.b", " └─HashJoin 15593.77 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", @@ -7415,8 +7415,8 @@ "SQL": "select /*+ leading(t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7433,8 +7433,8 @@ "SQL": "select /*+ leading(t2, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7452,8 +7452,8 @@ "SQL": "select /*+ leading(t1, t3@sel_2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7471,8 +7471,8 @@ "SQL": "select /*+ leading(t3@sel_2, t2) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", @@ -7490,8 +7490,8 @@ "SQL": "select /*+ leading(t3@sel_2, t1) */ * from t1 join t2 on t1.a=t2.a where t1.a not in (select t3.a from t3)", "Plan": [ "HashJoin 9990.00 root CARTESIAN anti semi join, other cond:eq(test.t1.a, test.t3.a)", - "├─TableReader(Build) 3.00 root data:TableFullScan", - "│ └─TableFullScan 3.00 cop[tikv] table:t3 keep order:false", + "├─IndexReader(Build) 3.00 root index:IndexFullScan", + "│ └─IndexFullScan 3.00 cop[tikv] table:t3, index:a(a) keep order:false", "└─HashJoin(Probe) 12487.50 root inner join, equal:[eq(test.t1.a, test.t2.a)]", " ├─TableReader(Build) 9990.00 root data:Selection", " │ └─Selection 9990.00 cop[tikv] not(isnull(test.t2.a))", From 9efcb88f112ca98865b9c8c35101a5d795e6a0f6 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Tue, 8 Nov 2022 20:07:50 -0800 Subject: [PATCH 20/22] ddl: fix rename partition column panic (#38932) (#38989) close pingcap/tidb#38932 --- ddl/db_partition_test.go | 23 +++++++++++++++++++++++ ddl/ddl_api.go | 1 + ddl/partition.go | 15 ++++++++++++++- 3 files changed, 38 insertions(+), 1 deletion(-) diff --git a/ddl/db_partition_test.go b/ddl/db_partition_test.go index b10f01c272ef9..e5ad2aa2bbfec 100644 --- a/ddl/db_partition_test.go +++ b/ddl/db_partition_test.go @@ -4535,6 +4535,7 @@ func TestPartitionTableWithAnsiQuotes(t *testing.T) { ` PARTITION "p4" VALUES LESS THAN ('\\''\t\n','\\''\t\n'),` + "\n" + ` PARTITION "pMax" VALUES LESS THAN (MAXVALUE,MAXVALUE))`)) } + func TestAlterModifyColumnOnPartitionedTable(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -4665,6 +4666,28 @@ func TestAlterModifyColumnOnPartitionedTable(t *testing.T) { tk.MustGetErrCode(`alter table t modify a varchar(20)`, errno.ErrUnsupportedDDLOperation) } +func TestAlterModifyColumnOnPartitionedTableRename(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + schemaName := "modColPartRename" + tk.MustExec("create database " + schemaName) + tk.MustExec("use " + schemaName) + tk.MustExec(`create table t (a int, b char) partition by range (a) (partition p0 values less than (10))`) + tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a char, b char) partition by range columns (a) (partition p0 values less than ('z'))`) + tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]New column does not match partition definitions: [ddl:1567]partition column name cannot be found") + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a int, b char) partition by list (a) (partition p0 values in (10))`) + tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a char, b char) partition by list columns (a) (partition p0 values in ('z'))`) + tk.MustContainErrMsg(`alter table t change a c char`, "[ddl:8200]New column does not match partition definitions: [ddl:1567]partition column name cannot be found") + tk.MustExec(`drop table t`) + tk.MustExec(`create table t (a int, b char) partition by hash (a) partitions 3`) + tk.MustContainErrMsg(`alter table t change a c int`, "[planner:1054]Unknown column 'a' in 'expression'") +} + func TestAlterModifyColumnOnPartitionedTableFail(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index e67c0eb001128..93cc9352fd9ed 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4670,6 +4670,7 @@ func GetModifiableColumnJob( newTblInfo.Columns = newCols var buf bytes.Buffer + // TODO: update the partitioning columns with new names if column is renamed AppendPartitionInfo(tblInfo.GetPartitionInfo(), &buf, mysql.ModeNone) // The parser supports ALTER TABLE ... PARTITION BY ... even if the ddl code does not yet :) // Ignoring warnings diff --git a/ddl/partition.go b/ddl/partition.go index 5a6a5b561b146..7bba0b1006332 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -782,6 +782,9 @@ func generatePartitionDefinitionsFromInterval(ctx sessionctx.Context, partOption } if len(tbInfo.Partition.Columns) > 0 { colTypes := collectColumnsType(tbInfo) + if len(colTypes) != len(tbInfo.Partition.Columns) { + return dbterror.ErrWrongPartitionName.GenWithStack("partition column name cannot be found") + } if _, err := checkAndGetColumnsTypeAndValuesMatch(ctx, colTypes, first.Exprs); err != nil { return err } @@ -1081,6 +1084,9 @@ func buildListPartitionDefinitions(ctx sessionctx.Context, defs []*ast.Partition definitions := make([]model.PartitionDefinition, 0, len(defs)) exprChecker := newPartitionExprChecker(ctx, nil, checkPartitionExprAllowed) colTypes := collectColumnsType(tbInfo) + if len(colTypes) != len(tbInfo.Partition.Columns) { + return nil, dbterror.ErrWrongPartitionName.GenWithStack("partition column name cannot be found") + } for _, def := range defs { if err := def.Clause.Validate(model.PartitionTypeList, len(tbInfo.Partition.Columns)); err != nil { return nil, err @@ -1139,7 +1145,11 @@ func collectColumnsType(tbInfo *model.TableInfo) []types.FieldType { if len(tbInfo.Partition.Columns) > 0 { colTypes := make([]types.FieldType, 0, len(tbInfo.Partition.Columns)) for _, col := range tbInfo.Partition.Columns { - colTypes = append(colTypes, findColumnByName(col.L, tbInfo).FieldType) + c := findColumnByName(col.L, tbInfo) + if c == nil { + return nil + } + colTypes = append(colTypes, c.FieldType) } return colTypes @@ -1152,6 +1162,9 @@ func buildRangePartitionDefinitions(ctx sessionctx.Context, defs []*ast.Partitio definitions := make([]model.PartitionDefinition, 0, len(defs)) exprChecker := newPartitionExprChecker(ctx, nil, checkPartitionExprAllowed) colTypes := collectColumnsType(tbInfo) + if len(colTypes) != len(tbInfo.Partition.Columns) { + return nil, dbterror.ErrWrongPartitionName.GenWithStack("partition column name cannot be found") + } for _, def := range defs { if err := def.Clause.Validate(model.PartitionTypeRange, len(tbInfo.Partition.Columns)); err != nil { return nil, err From 9303bfebe26425c679db09f0bbb7a948bdb83464 Mon Sep 17 00:00:00 2001 From: Hangjie Mo Date: Wed, 9 Nov 2022 13:17:50 +0800 Subject: [PATCH 21/22] ddl: set tidb_super_read_only=ON when do flashback cluster (#38986) close pingcap/tidb#38984 --- ddl/cluster.go | 55 +++++++++++++++++++++-------- ddl/cluster_test.go | 18 ++++++++-- ddl/ddl_api.go | 7 ++-- tests/readonlytest/readonly_test.go | 5 +++ 4 files changed, 65 insertions(+), 20 deletions(-) diff --git a/ddl/cluster.go b/ddl/cluster.go index 7d9070578af50..ebb833156cec2 100644 --- a/ddl/cluster.go +++ b/ddl/cluster.go @@ -60,12 +60,15 @@ var pdScheduleKey = []string{ const ( flashbackMaxBackoff = 1800000 // 1800s flashbackTimeout = 3 * time.Minute // 3min +) - pdScheduleArgsOffset = 1 - gcEnabledArgsOffset = 2 - autoAnalyzeOffset = 3 - totalLockedRegionsOffset = 4 - commitTSOffset = 5 +const ( + pdScheduleArgsOffset = 1 + iota + gcEnabledOffset + autoAnalyzeOffset + readOnlyOffset + totalLockedRegionsOffset + commitTSOffset ) func closePDSchedule() error { @@ -122,8 +125,8 @@ func ValidateFlashbackTS(ctx context.Context, sctx sessionctx.Context, flashBack return gcutil.ValidateSnapshotWithGCSafePoint(flashBackTS, gcSafePoint) } -func setTiDBEnableAutoAnalyze(sess sessionctx.Context, value string) error { - return sess.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(context.Background(), variable.TiDBEnableAutoAnalyze, value) +func setTiDBEnableAutoAnalyze(ctx context.Context, sess sessionctx.Context, value string) error { + return sess.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(ctx, variable.TiDBEnableAutoAnalyze, value) } func getTiDBEnableAutoAnalyze(sess sessionctx.Context) (string, error) { @@ -134,6 +137,18 @@ func getTiDBEnableAutoAnalyze(sess sessionctx.Context) (string, error) { return val, nil } +func setTiDBSuperReadOnly(ctx context.Context, sess sessionctx.Context, value string) error { + return sess.GetSessionVars().GlobalVarsAccessor.SetGlobalSysVar(ctx, variable.TiDBSuperReadOnly, value) +} + +func getTiDBSuperReadOnly(sess sessionctx.Context) (string, error) { + val, err := sess.GetSessionVars().GlobalVarsAccessor.GetGlobalSysVar(variable.TiDBSuperReadOnly) + if err != nil { + return "", errors.Trace(err) + } + return val, nil +} + func checkAndSetFlashbackClusterInfo(sess sessionctx.Context, d *ddlCtx, t *meta.Meta, job *model.Job, flashbackTS uint64) (err error) { if err = ValidateFlashbackTS(d.ctx, sess, flashbackTS); err != nil { return err @@ -145,7 +160,10 @@ func checkAndSetFlashbackClusterInfo(sess sessionctx.Context, d *ddlCtx, t *meta if err = closePDSchedule(); err != nil { return err } - if err = setTiDBEnableAutoAnalyze(sess, variable.Off); err != nil { + if err = setTiDBEnableAutoAnalyze(d.ctx, sess, variable.Off); err != nil { + return err + } + if err = setTiDBSuperReadOnly(d.ctx, sess, variable.On); err != nil { return err } @@ -465,9 +483,9 @@ func (w *worker) onFlashbackCluster(d *ddlCtx, t *meta.Meta, job *model.Job) (ve var flashbackTS, lockedRegions, commitTS uint64 var pdScheduleValue map[string]interface{} - var autoAnalyzeValue string + var autoAnalyzeValue, readOnlyValue string var gcEnabledValue bool - if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabledValue, &autoAnalyzeValue, &lockedRegions, &commitTS); err != nil { + if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabledValue, &autoAnalyzeValue, &readOnlyValue, &lockedRegions, &commitTS); err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } @@ -494,13 +512,19 @@ func (w *worker) onFlashbackCluster(d *ddlCtx, t *meta.Meta, job *model.Job) (ve job.State = model.JobStateCancelled return ver, errors.Trace(err) } - job.Args[gcEnabledArgsOffset] = &gcEnableValue + job.Args[gcEnabledOffset] = &gcEnableValue autoAnalyzeValue, err = getTiDBEnableAutoAnalyze(sess) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } job.Args[autoAnalyzeOffset] = &autoAnalyzeValue + readOnlyValue, err = getTiDBSuperReadOnly(sess) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + job.Args[readOnlyOffset] = &readOnlyValue job.SchemaState = model.StateDeleteOnly return ver, nil // Stage 2, check flashbackTS, close GC and PD schedule. @@ -593,10 +617,10 @@ func finishFlashbackCluster(w *worker, job *model.Job) error { var flashbackTS, lockedRegions, commitTS uint64 var pdScheduleValue map[string]interface{} - var autoAnalyzeValue string + var autoAnalyzeValue, readOnlyValue string var gcEnabled bool - if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabled, &autoAnalyzeValue, &lockedRegions, &commitTS); err != nil { + if err := job.DecodeArgs(&flashbackTS, &pdScheduleValue, &gcEnabled, &autoAnalyzeValue, &readOnlyValue, &lockedRegions, &commitTS); err != nil { return errors.Trace(err) } sess, err := w.sessPool.get() @@ -614,7 +638,10 @@ func finishFlashbackCluster(w *worker, job *model.Job) error { return err } } - return setTiDBEnableAutoAnalyze(sess, autoAnalyzeValue) + if err = setTiDBSuperReadOnly(w.ctx, sess, readOnlyValue); err != nil { + return err + } + return setTiDBEnableAutoAnalyze(w.ctx, sess, autoAnalyzeValue) }) if err != nil { return err diff --git a/ddl/cluster_test.go b/ddl/cluster_test.go index b32fcd709e924..db406230050fb 100644 --- a/ddl/cluster_test.go +++ b/ddl/cluster_test.go @@ -202,23 +202,35 @@ func TestGlobalVariablesOnFlashback(t *testing.T) { rs, err = tk.Exec("show variables like 'tidb_enable_auto_analyze'") assert.NoError(t, err) assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + rs, err = tk.Exec("show variables like 'tidb_super_read_only'") + assert.NoError(t, err) + assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) } } dom.DDL().SetHook(hook) - // first try with `tidb_gc_enable` = on + // first try with `tidb_gc_enable` = on and `tidb_super_read_only` = off tk.MustExec("set global tidb_gc_enable = on") + tk.MustExec("set global tidb_super_read_only = off") tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts))) - rs, err := tk.Exec("show variables like 'tidb_gc_enable'") + + rs, err := tk.Exec("show variables like 'tidb_super_read_only'") + require.NoError(t, err) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) + rs, err = tk.Exec("show variables like 'tidb_gc_enable'") require.NoError(t, err) require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) - // second try with `tidb_gc_enable` = off + // second try with `tidb_gc_enable` = off and `tidb_super_read_only` = on tk.MustExec("set global tidb_gc_enable = off") + tk.MustExec("set global tidb_super_read_only = on") ts, err = tk.Session().GetStore().GetOracle().GetTimestamp(context.Background(), &oracle.Option{}) require.NoError(t, err) tk.MustExec(fmt.Sprintf("flashback cluster to timestamp '%s'", oracle.GetTimeFromTS(ts))) + rs, err = tk.Exec("show variables like 'tidb_super_read_only'") + require.NoError(t, err) + require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.On) rs, err = tk.Exec("show variables like 'tidb_gc_enable'") require.NoError(t, err) require.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][1], variable.Off) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 93cc9352fd9ed..f60a80ece0d06 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2716,9 +2716,10 @@ func (d *ddl) FlashbackCluster(ctx sessionctx.Context, flashbackTS uint64) error Args: []interface{}{ flashbackTS, map[string]interface{}{}, - true, /* tidb_gc_enable */ - variable.On, /* tidb_enable_auto_analyze */ - 0, /* totalRegions */ + true, /* tidb_gc_enable */ + variable.On, /* tidb_enable_auto_analyze */ + variable.Off, /* tidb_super_read_only */ + 0, /* totalRegions */ 0 /* newCommitTS */}, } err := d.DoDDLJob(ctx, job) diff --git a/tests/readonlytest/readonly_test.go b/tests/readonlytest/readonly_test.go index 836debae6ac11..654e2542738e5 100644 --- a/tests/readonlytest/readonly_test.go +++ b/tests/readonlytest/readonly_test.go @@ -158,6 +158,11 @@ func TestRestriction(t *testing.T) { require.Error(t, err) require.Equal(t, err.Error(), PriviledgedErrMsg) + // can't do flashback cluster + _, err = s.udb.Exec("flashback cluster to timestamp ''") + require.Error(t, err) + require.Equal(t, err.Error(), ReadOnlyErrMsg) + // can do some Admin stmts _, err = s.udb.Exec("admin show ddl jobs") require.NoError(t, err) From 8653b1e736f1967361e953a5fb4e5605eac1c4b5 Mon Sep 17 00:00:00 2001 From: Weizhen Wang Date: Wed, 9 Nov 2022 13:55:51 +0800 Subject: [PATCH 22/22] statisticstest: fix goleak on the statisticstest (#39005) close pingcap/tidb#39004 --- tests/realtikvtest/statisticstest/main_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/realtikvtest/statisticstest/main_test.go b/tests/realtikvtest/statisticstest/main_test.go index 8b9cf1e62690c..5fef5b584be04 100644 --- a/tests/realtikvtest/statisticstest/main_test.go +++ b/tests/realtikvtest/statisticstest/main_test.go @@ -26,6 +26,7 @@ func TestMain(m *testing.M) { opts := []goleak.Option{ goleak.IgnoreTopFunction("github.com/golang/glog.(*loggingT).flushDaemon"), goleak.IgnoreTopFunction("github.com/lestrrat-go/httprc.runFetchWorker"), + goleak.IgnoreTopFunction("google.golang.org/grpc.(*ccBalancerWrapper).watcher"), goleak.IgnoreTopFunction("internal/poll.runtime_pollWait"), goleak.IgnoreTopFunction("net/http.(*persistConn).writeLoop"), }