Skip to content

Commit

Permalink
merge upstream master
Browse files Browse the repository at this point in the history
  • Loading branch information
mmyj committed May 12, 2021
2 parents 7f27c38 + b1d134d commit 55a8461
Show file tree
Hide file tree
Showing 58 changed files with 951 additions and 187 deletions.
5 changes: 5 additions & 0 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/metrics"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv/mockstore/cluster"
"github.com/pingcap/tidb/util"
Expand Down Expand Up @@ -70,6 +71,10 @@ type mockSessionManager struct {
PS []*util.ProcessInfo
}

func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
for _, item := range msm.PS {
Expand Down
5 changes: 5 additions & 0 deletions domain/domain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv"
Expand Down Expand Up @@ -241,6 +242,10 @@ type mockSessionManager struct {
PS []*util.ProcessInfo
}

func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
for _, item := range msm.PS {
Expand Down
2 changes: 1 addition & 1 deletion errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1020,7 +1020,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrBuildExecutor: mysql.Message("Failed to build executor", nil),
ErrBatchInsertFail: mysql.Message("Batch insert failed, please clean the table and try again.", nil),
ErrGetStartTS: mysql.Message("Can not get start ts", nil),
ErrPrivilegeCheckFail: mysql.Message("privilege check fail", nil), // this error message should begin lowercased to be compatible with the test
ErrPrivilegeCheckFail: mysql.Message("privilege check for '%s' fail", nil), // this error message should begin lowercased to be compatible with the test
ErrInvalidWildCard: mysql.Message("Wildcard fields without any table name appears in wrong place", nil),
ErrMixOfGroupFuncAndFieldsIncompatible: mysql.Message("In aggregated query without GROUP BY, expression #%d of SELECT list contains nonaggregated column '%s'; this is incompatible with sql_mode=only_full_group_by", nil),
ErrUnsupportedSecondArgumentType: mysql.Message("JSON_OBJECTAGG: unsupported second argument type %v", nil),
Expand Down
2 changes: 1 addition & 1 deletion errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1133,7 +1133,7 @@ Schema has changed

["planner:8121"]
error = '''
privilege check fail
privilege check for '%s' fail
'''

["planner:8122"]
Expand Down
3 changes: 3 additions & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ import (
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/stmtsummary"
"github.com/pingcap/tidb/util/stringutil"

"go.uber.org/zap"
"go.uber.org/zap/zapcore"
)
Expand Down Expand Up @@ -377,6 +378,7 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) {
if txn.Valid() {
txnStartTS = txn.StartTS()
}

return &recordSet{
executor: e,
stmt: a,
Expand Down Expand Up @@ -590,6 +592,7 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error {
}
e, err = a.handlePessimisticLockError(ctx, err)
if err != nil {
// todo: Report deadlock
if ErrDeadlock.Equal(err) {
metrics.StatementDeadlockDetectDuration.Observe(time.Since(startLocking).Seconds())
}
Expand Down
4 changes: 3 additions & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1531,7 +1531,9 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo
strings.ToLower(infoschema.TablePlacementPolicy),
strings.ToLower(infoschema.TableClientErrorsSummaryGlobal),
strings.ToLower(infoschema.TableClientErrorsSummaryByUser),
strings.ToLower(infoschema.TableClientErrorsSummaryByHost):
strings.ToLower(infoschema.TableClientErrorsSummaryByHost),
strings.ToLower(infoschema.TableTiDBTrx),
strings.ToLower(infoschema.ClusterTableTiDBTrx):
return &MemTableReaderExec{
baseExecutor: newBaseExecutor(b.ctx, v.Schema(), v.ID()),
table: v.Table,
Expand Down
5 changes: 5 additions & 0 deletions executor/executor_pkg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/executor/aggfuncs"
"github.com/pingcap/tidb/expression"
plannerutil "github.com/pingcap/tidb/planner/util"
txninfo "github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
Expand Down Expand Up @@ -60,6 +61,10 @@ type mockSessionManager struct {
serverID uint64
}

func (msm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

// ShowProcessList implements the SessionManager.ShowProcessList interface.
func (msm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
Expand Down
5 changes: 5 additions & 0 deletions executor/explainfor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/parser/auth"
"github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
txninfo "github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/israce"
Expand All @@ -38,6 +39,10 @@ type mockSessionManager1 struct {
PS []*util.ProcessInfo
}

func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo {
return nil
}

// ShowProcessList implements the SessionManager.ShowProcessList interface.
func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
Expand Down
38 changes: 38 additions & 0 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,10 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
infoschema.TableClientErrorsSummaryByUser,
infoschema.TableClientErrorsSummaryByHost:
err = e.setDataForClientErrorsSummary(sctx, e.table.Name.O)
case infoschema.TableTiDBTrx:
e.setDataForTiDBTrx(sctx)
case infoschema.ClusterTableTiDBTrx:
err = e.setDataForClusterTiDBTrx(sctx)
}
if err != nil {
return nil, err
Expand Down Expand Up @@ -2011,6 +2015,40 @@ func (e *memtableRetriever) setDataForClientErrorsSummary(ctx sessionctx.Context
return nil
}

func (e *memtableRetriever) setDataForTiDBTrx(ctx sessionctx.Context) {
sm := ctx.GetSessionManager()
if sm == nil {
return
}

loginUser := ctx.GetSessionVars().User
var hasProcessPriv bool
if pm := privilege.GetPrivilegeManager(ctx); pm != nil {
if pm.RequestVerification(ctx.GetSessionVars().ActiveRoles, "", "", "", mysql.ProcessPriv) {
hasProcessPriv = true
}
}
infoList := sm.ShowTxnList()
for _, info := range infoList {
// If you have the PROCESS privilege, you can see all running transactions.
// Otherwise, you can see only your own transactions.
if !hasProcessPriv && loginUser != nil && info.Username != loginUser.Username {
continue
}
e.rows = append(e.rows, info.ToDatum())
}
}

func (e *memtableRetriever) setDataForClusterTiDBTrx(ctx sessionctx.Context) error {
e.setDataForTiDBTrx(ctx)
rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows)
if err != nil {
return err
}
e.rows = rows
return nil
}

type hugeMemTableRetriever struct {
dummyCloser
table *model.TableInfo
Expand Down
5 changes: 5 additions & 0 deletions executor/infoschema_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/server"
"github.com/pingcap/tidb/session"
txninfo "github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle"
Expand Down Expand Up @@ -728,6 +729,10 @@ type mockSessionManager struct {
serverID uint64
}

func (sm *mockSessionManager) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

func (sm *mockSessionManager) ShowProcessList() map[uint64]*util.ProcessInfo {
return sm.processInfoMap
}
Expand Down
2 changes: 1 addition & 1 deletion executor/mpp_gather.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import (
)

func useMPPExecution(ctx sessionctx.Context, tr *plannercore.PhysicalTableReader) bool {
if !ctx.GetSessionVars().AllowMPPExecution {
if !ctx.GetSessionVars().IsMPPAllowed() {
return false
}
_, ok := tr.GetTablePlan().(*plannercore.PhysicalExchangeSender)
Expand Down
66 changes: 66 additions & 0 deletions executor/partition_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -434,6 +434,72 @@ func (s *partitionTableSuite) TestGlobalStatsAndSQLBinding(c *C) {
tk.MustIndexLookup("select * from tlist where a<1")
}

func createTable4DynamicPruneModeTestWithExpression(tk *testkit.TestKit) {
tk.MustExec("create table trange(a int) partition by range(a) (partition p0 values less than(3), partition p1 values less than (5), partition p2 values less than(11));")
tk.MustExec("create table thash(a int) partition by hash(a) partitions 4;")
tk.MustExec("create table t(a int)")
tk.MustExec("insert into trange values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);")
tk.MustExec("insert into thash values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);")
tk.MustExec("insert into t values(1), (1), (1), (2), (3), (4), (5), (6), (7), (7), (10), (NULL), (NULL);")
tk.MustExec("set session tidb_partition_prune_mode='dynamic'")
tk.MustExec("analyze table trange")
tk.MustExec("analyze table thash")
tk.MustExec("analyze table t")
}

type testData4Expression struct {
sql string
partitions []string
}

func (s *partitionTableSuite) TestDynamicPruneModeWithEqualExpression(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop database if exists db_equal_expression")
tk.MustExec("create database db_equal_expression")
tk.MustExec("use db_equal_expression")
createTable4DynamicPruneModeTestWithExpression(tk)

tables := []string{"trange", "thash"}
tests := []testData4Expression{
{
sql: "select * from %s where a = 2",
partitions: []string{
"p0",
"p2",
},
},
{
sql: "select * from %s where a = 4 or a = 1",
partitions: []string{
"p0,p1",
"p0,p1",
},
},
{
sql: "select * from %s where a = -1",
partitions: []string{
"p0",
"p1",
},
},
{
sql: "select * from %s where a is NULL",
partitions: []string{
"p0",
"p0",
},
},
}

for _, t := range tests {
for i := range t.partitions {
sql := fmt.Sprintf(t.sql, tables[i])
c.Assert(tk.MustPartition(sql, t.partitions[i]), IsTrue)
tk.MustQuery(sql).Sort().Check(tk.MustQuery(fmt.Sprintf(t.sql, "t")).Sort().Rows())
}
}
}

func (s *partitionTableSuite) TestDirectReadingWithAgg(c *C) {
if israce.RaceEnabled {
c.Skip("exhaustive types test, skip race test")
Expand Down
5 changes: 5 additions & 0 deletions executor/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/domain"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
txninfo "github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/israce"
Expand Down Expand Up @@ -135,6 +136,10 @@ type mockSessionManager2 struct {
killed bool
}

func (sm *mockSessionManager2) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

func (sm *mockSessionManager2) ShowProcessList() map[uint64]*util.ProcessInfo {
pl := make(map[uint64]*util.ProcessInfo)
if pi, ok := sm.GetProcessInfo(0); ok {
Expand Down
6 changes: 6 additions & 0 deletions executor/revoke.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,8 +88,14 @@ func (e *RevokeExec) Next(ctx context.Context, req *chunk.Chunk) error {
return err
}

sessVars := e.ctx.GetSessionVars()
// Revoke for each user.
for _, user := range e.Users {
if user.User.CurrentUser {
user.User.Username = sessVars.User.AuthUsername
user.User.Hostname = sessVars.User.AuthHostname
}

// Check if user exists.
exists, err := userExists(e.ctx, user.User.Username, user.User.Hostname)
if err != nil {
Expand Down
5 changes: 5 additions & 0 deletions executor/seqtest/prepared_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/metrics"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/session/txninfo"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/kvcache"
Expand Down Expand Up @@ -796,6 +797,10 @@ type mockSessionManager1 struct {
Se session.Session
}

func (msm *mockSessionManager1) ShowTxnList() []*txninfo.TxnInfo {
panic("unimplemented!")
}

// ShowProcessList implements the SessionManager.ShowProcessList interface.
func (msm *mockSessionManager1) ShowProcessList() map[uint64]*util.ProcessInfo {
ret := make(map[uint64]*util.ProcessInfo)
Expand Down
Loading

0 comments on commit 55a8461

Please sign in to comment.