Skip to content

Commit

Permalink
Merge branch 'master' into issue_27422
Browse files Browse the repository at this point in the history
  • Loading branch information
sylzd authored Aug 25, 2021
2 parents 25922f4 + f10b625 commit 5369b4e
Show file tree
Hide file tree
Showing 27 changed files with 884 additions and 202 deletions.
2 changes: 1 addition & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,7 @@

include Makefile.common

.PHONY: all clean test gotest server dev benchkv benchraw check checklist parser tidy ddltest
.PHONY: all clean test gotest server dev benchkv benchraw check checklist parser tidy ddltest build_br build_lightning build_lightning-ctl

default: server buildsucc

Expand Down
7 changes: 5 additions & 2 deletions br/pkg/backup/push.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,13 +168,16 @@ func (push *pushDown) pushBackup(
logutil.CL(ctx).Error("", zap.String("error", berrors.ErrKVStorage.Error()+": "+errMsg),
zap.String("work around", "please ensure br and tikv node share a same disk and the user of br and tikv has same uid."))
}

if utils.MessageIsPermissionDeniedStorageError(errPb.GetMsg()) {
errMsg := fmt.Sprintf("I/O permission denied error occurs on TiKV Node(store id: %v; Address: %s)", store.GetId(), redact.String(store.GetAddress()))
logutil.CL(ctx).Error("", zap.String("error", berrors.ErrKVStorage.Error()+": "+errMsg),
zap.String("work around", "please ensure tikv has permission to read from & write to the storage."))
}
return res, berrors.ErrKVStorage
return res, errors.Annotatef(berrors.ErrKVStorage, "error happen in store %v at %s: %s",
store.GetId(),
redact.String(store.GetAddress()),
errPb.Msg,
)
}
}
case err := <-push.errCh:
Expand Down
2 changes: 1 addition & 1 deletion cmd/explaintest/r/explain_easy_stats.result
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ Projection 1999.00 root eq(test.t1.c2, test.t2.c2)->Column#11
└─IndexLookUp 1.00 root
├─Limit(Build) 1.00 cop[tikv] offset:0, count:1
│ └─IndexRangeScan 1.25 cop[tikv] table:t2, index:c1(c1) range: decided by [eq(test.t1.c1, test.t2.c1)], keep order:true
└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false, stats:pseudo
└─TableRowIDScan(Probe) 1.00 cop[tikv] table:t2 keep order:false
explain format = 'brief' select * from t1 order by c1 desc limit 1;
id estRows task access object operator info
Limit 1.00 root offset:0, count:1
Expand Down
6 changes: 5 additions & 1 deletion ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -5725,10 +5725,14 @@ func (s *testSerialDBSuite) TestSetTiFlashReplicaForTemporaryTable(c *C) {

tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("set tidb_enable_global_temporary_table=true")
tk.MustExec("set tidb_enable_noop_functions = 1")
tk.MustExec("drop table if exists temp, temp2")
tk.MustExec("drop table if exists temp")
tk.MustExec("create global temporary table temp(id int) on commit delete rows")
tk.MustExec("create temporary table temp2(id int)")
tk.MustGetErrCode("alter table temp set tiflash replica 1", errno.ErrOptOnTemporaryTable)
tk.MustExec("drop table temp")
tk.MustGetErrCode("alter table temp2 set tiflash replica 1", errno.ErrUnsupportedDDLOperation)
tk.MustExec("drop table temp, temp2")

tk.MustExec("drop table if exists normal")
tk.MustExec("create table normal(id int)")
Expand Down
9 changes: 0 additions & 9 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/diagnosticspb"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/auth"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/distsql"
Expand Down Expand Up @@ -740,14 +739,6 @@ func (b *executorBuilder) buildShow(v *plannercore.PhysicalShow) Executor {
GlobalScope: v.GlobalScope,
Extended: v.Extended,
}
if e.Tp == ast.ShowGrants && e.User == nil {
// The input is a "show grants" statement, fulfill the user and roles field.
// Note: "show grants" result are different from "show grants for current_user",
// The former determine privileges with roles, while the later doesn't.
vars := e.ctx.GetSessionVars()
e.User = &auth.UserIdentity{Username: vars.User.AuthUsername, Hostname: vars.User.AuthHostname}
e.Roles = vars.ActiveRoles
}
if e.Tp == ast.ShowMasterStatus {
// show master status need start ts.
if _, err := e.ctx.Txn(true); err != nil {
Expand Down
32 changes: 18 additions & 14 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -1414,30 +1414,34 @@ func (e *ShowExec) fetchShowCreateUser(ctx context.Context) error {
}

func (e *ShowExec) fetchShowGrants() error {
// Get checker
vars := e.ctx.GetSessionVars()
checker := privilege.GetPrivilegeManager(e.ctx)
if checker == nil {
return errors.New("miss privilege checker")
}
sessVars := e.ctx.GetSessionVars()
if !e.User.CurrentUser {
userName := sessVars.User.AuthUsername
hostName := sessVars.User.AuthHostname
if e.User == nil || e.User.CurrentUser {
// The input is a "SHOW GRANTS" statement with no users *or* SHOW GRANTS FOR CURRENT_USER()
// In these cases we include the active roles for showing privileges.
e.User = &auth.UserIdentity{Username: vars.User.AuthUsername, Hostname: vars.User.AuthHostname}
e.Roles = vars.ActiveRoles
} else {
userName := vars.User.AuthUsername
hostName := vars.User.AuthHostname
// Show grant user requires the SELECT privilege on mysql schema.
// Ref https://dev.mysql.com/doc/refman/8.0/en/show-grants.html
if userName != e.User.Username || hostName != e.User.Hostname {
activeRoles := sessVars.ActiveRoles
if !checker.RequestVerification(activeRoles, mysql.SystemDB, "", "", mysql.SelectPriv) {
if !checker.RequestVerification(vars.ActiveRoles, mysql.SystemDB, "", "", mysql.SelectPriv) {
return ErrDBaccessDenied.GenWithStackByArgs(userName, hostName, mysql.SystemDB)
}
}
}
for _, r := range e.Roles {
if r.Hostname == "" {
r.Hostname = "%"
}
if !checker.FindEdge(e.ctx, r, e.User) {
return ErrRoleNotGranted.GenWithStackByArgs(r.String(), e.User.String())
// This is for the syntax SHOW GRANTS FOR x USING role
for _, r := range e.Roles {
if r.Hostname == "" {
r.Hostname = "%"
}
if !checker.FindEdge(e.ctx, r, e.User) {
return ErrRoleNotGranted.GenWithStackByArgs(r.String(), e.User.String())
}
}
}
gs, err := checker.ShowGrants(e.ctx, e.User, e.Roles)
Expand Down
3 changes: 2 additions & 1 deletion executor/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -267,7 +267,8 @@ func (s *testSuite5) TestIssue10549(c *C) {
c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "dev", Hostname: "%", AuthUsername: "dev", AuthHostname: "%"}, nil, nil), IsTrue)
tk.MustQuery("SHOW DATABASES;").Check(testkit.Rows("INFORMATION_SCHEMA", "newdb"))
tk.MustQuery("SHOW GRANTS;").Check(testkit.Rows("GRANT USAGE ON *.* TO 'dev'@'%'", "GRANT ALL PRIVILEGES ON newdb.* TO 'dev'@'%'", "GRANT 'app_developer'@'%' TO 'dev'@'%'"))
tk.MustQuery("SHOW GRANTS FOR CURRENT_USER").Check(testkit.Rows("GRANT USAGE ON *.* TO 'dev'@'%'", "GRANT 'app_developer'@'%' TO 'dev'@'%'"))
tk.MustQuery("SHOW GRANTS FOR CURRENT_USER").Check(testkit.Rows("GRANT USAGE ON *.* TO 'dev'@'%'", "GRANT ALL PRIVILEGES ON newdb.* TO 'dev'@'%'", "GRANT 'app_developer'@'%' TO 'dev'@'%'"))
tk.MustQuery("SHOW GRANTS FOR dev").Check(testkit.Rows("GRANT USAGE ON *.* TO 'dev'@'%'", "GRANT 'app_developer'@'%' TO 'dev'@'%'"))
}

func (s *testSuite5) TestIssue11165(c *C) {
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ require (
github.com/spf13/pflag v1.0.5
github.com/stretchr/testify v1.7.0
github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2
github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210820051608-ff94d0c9acb8
github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210824090536-16d902a3c7e5
github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d
github.com/twmb/murmur3 v1.1.3
github.com/uber-go/atomic v1.4.0
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -694,8 +694,8 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfK
github.com/tidwall/gjson v1.3.5/go.mod h1:P256ACg0Mn+j1RXIDXoss50DeIABTYK1PULOJHhxOls=
github.com/tidwall/match v1.0.1/go.mod h1:LujAq0jyVjBy028G1WhWfIzbpQfMO8bBZ6Tyb0+pL9E=
github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk=
github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210820051608-ff94d0c9acb8 h1:/S06bCy/r/zuAe+uSjkywM7RHq4rXySApIom3uBVsIU=
github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210820051608-ff94d0c9acb8/go.mod h1:KwtZXt0JD+bP9bWW2ka0ir3Wp3oTEfZUTh22bs2sI4o=
github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210824090536-16d902a3c7e5 h1:7CJYiW8gKiI3IQOQSAZyqZq0GxB+bmrnZgk9QNZ1cPo=
github.com/tikv/client-go/v2 v2.0.0-alpha.0.20210824090536-16d902a3c7e5/go.mod h1:KwtZXt0JD+bP9bWW2ka0ir3Wp3oTEfZUTh22bs2sI4o=
github.com/tikv/pd v1.1.0-beta.0.20210323121136-78679e5e209d/go.mod h1:Jw9KG11C/23Rr7DW4XWQ7H5xOgGZo6DFL1OKAF4+Igw=
github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d h1:AFm1Dzw+QRUevWRfrFp45CPPkuK/zdSWcfxI10z+WVE=
github.com/tikv/pd v1.1.0-beta.0.20210818082359-acba1da0018d/go.mod h1:rammPjeZgpvfrQRPkijcx8tlxF1XM5+m6kRXrkDzCAA=
Expand Down
35 changes: 35 additions & 0 deletions kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"crypto/tls"
"time"

"github.com/pingcap/errors"
deadlockpb "github.com/pingcap/kvproto/pkg/deadlock"
"github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/kvproto/pkg/metapb"
Expand Down Expand Up @@ -70,6 +71,40 @@ type Retriever interface {
IterReverse(k Key) (Iterator, error)
}

// EmptyIterator is an iterator without any entry
type EmptyIterator struct{}

// Valid returns true if the current iterator is valid.
func (i *EmptyIterator) Valid() bool { return false }

// Key returns the current key. Always return nil for this iterator
func (i *EmptyIterator) Key() Key { return nil }

// Value returns the current value. Always return nil for this iterator
func (i *EmptyIterator) Value() []byte { return nil }

// Next goes the next position. Always return error for this iterator
func (i *EmptyIterator) Next() error { return errors.New("scanner iterator is invalid") }

// Close closes the iterator.
func (i *EmptyIterator) Close() {}

// EmptyRetriever is a retriever without any entry
type EmptyRetriever struct{}

// Get gets the value for key k from kv store. Always return nil for this retriever
func (r *EmptyRetriever) Get(_ context.Context, _ Key) ([]byte, error) {
return nil, ErrNotExist
}

// Iter creates an Iterator. Always return EmptyIterator for this retriever
func (r *EmptyRetriever) Iter(_ Key, _ Key) (Iterator, error) { return &EmptyIterator{}, nil }

// IterReverse creates a reversed Iterator. Always return EmptyIterator for this retriever
func (r *EmptyRetriever) IterReverse(_ Key) (Iterator, error) {
return &EmptyIterator{}, nil
}

// Mutator is the interface wraps the basic Set and Delete methods.
type Mutator interface {
// Set sets the value for key k as v into kv store.
Expand Down
121 changes: 121 additions & 0 deletions metrics/grafana/tidb.json
Original file line number Diff line number Diff line change
Expand Up @@ -12532,6 +12532,127 @@
"align": false,
"alignLevel": null
}
},
{
"aliasColors": {},
"bars": false,
"dashLength": 10,
"dashes": false,
"datasource": "${DS_TEST-CLUSTER}",
"description": "The duration of receiving response from TiKV.\nThis metrics can be high when there is no workload.\nBut if the value is too large, TiKV maybe not responding in time.",
"fieldConfig": {
"defaults": {},
"overrides": []
},
"fill": 1,
"fillGradient": 0,
"gridPos": {
"h": 7,
"w": 8,
"x": 0,
"y": 23
},
"hiddenSeries": false,
"id": 228,
"legend": {
"alignAsTable": true,
"avg": false,
"current": false,
"max": false,
"min": false,
"rightSide": true,
"show": true,
"total": false,
"values": false
},
"lines": true,
"linewidth": 1,
"links": [],
"nullPointMode": "null as zero",
"options": {
"alertThreshold": true
},
"percentage": false,
"pluginVersion": "7.5.7",
"pointradius": 2,
"points": false,
"renderer": "flot",
"seriesOverrides": [],
"spaceLength": 10,
"stack": false,
"steppedLine": false,
"targets": [
{
"exemplar": true,
"expr": "rate(tidb_tikvclient_batch_recv_latency_sum{tidb_cluster=\"$tidb_cluster\"}[1m]) / rate(tidb_tikvclient_batch_recv_latency_count{tidb_cluster=\"$tidb_cluster\"}[1m])",
"hide": false,
"interval": "",
"legendFormat": "{{instance}}-{{result}}",
"refId": "B"
},
{
"exemplar": true,
"expr": "histogram_quantile(0.99, sum(rate(tidb_tikvclient_batch_recv_latency_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance))",
"format": "time_series",
"hide": true,
"interval": "",
"intervalFactor": 1,
"legendFormat": "99-{{instance}}",
"refId": "A"
}
],
"thresholds": [
{
"$$hashKey": "object:367",
"colorMode": "warning",
"fill": true,
"line": false,
"op": "gt",
"value": 60000000000,
"yaxis": "left"
}
],
"timeFrom": null,
"timeRegions": [],
"timeShift": null,
"title": "Batch Receive Average Duration",
"tooltip": {
"shared": true,
"sort": 0,
"value_type": "individual"
},
"type": "graph",
"xaxis": {
"buckets": null,
"mode": "time",
"name": null,
"show": true,
"values": []
},
"yaxes": [
{
"$$hashKey": "object:58",
"format": "ns",
"label": null,
"logBase": 1,
"max": null,
"min": "0",
"show": true
},
{
"$$hashKey": "object:59",
"format": "short",
"label": null,
"logBase": 1,
"max": null,
"min": null,
"show": false
}
],
"yaxis": {
"align": false,
"alignLevel": null
}
}
],
"title": "Batch Client",
Expand Down
4 changes: 4 additions & 0 deletions planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1071,6 +1071,10 @@ func (p *LogicalJoin) constructInnerIndexScanTask(
}
}
}
// We set `StatsVersion` here and fill other fields in `(*copTask).finishIndexPlan`. Since `copTask.indexPlan` may
// change before calling `(*copTask).finishIndexPlan`, we don't know the stats information of `ts` currently and on
// the other hand, it may be hard to identify `StatsVersion` of `ts` in `(*copTask).finishIndexPlan`.
ts.stats = &property.StatsInfo{StatsVersion: ds.tableStats.StatsVersion}
// If inner cop task need keep order, the extraHandleCol should be set.
if cop.keepOrder && !ds.tableInfo.IsCommonHandle {
var needExtraProj bool
Expand Down
4 changes: 4 additions & 0 deletions planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -1194,6 +1194,10 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, candid
}.Init(ds.ctx, is.blockOffset)
ts.SetSchema(ds.schema.Clone())
ts.SetCost(cost)
// We set `StatsVersion` here and fill other fields in `(*copTask).finishIndexPlan`. Since `copTask.indexPlan` may
// change before calling `(*copTask).finishIndexPlan`, we don't know the stats information of `ts` currently and on
// the other hand, it may be hard to identify `StatsVersion` of `ts` in `(*copTask).finishIndexPlan`.
ts.stats = &property.StatsInfo{StatsVersion: ds.tableStats.StatsVersion}
cop.tablePlan = ts
}
cop.cst = cost
Expand Down
26 changes: 26 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import (
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/testkit"
Expand Down Expand Up @@ -4217,6 +4218,31 @@ func (s *testIntegrationSuite) TestOutputSkylinePruningInfo(c *C) {
}
}

func (s *testIntegrationSuite) TestIssue27083(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int primary key, b int, c int, index idx_b(b))")
tk.MustExec("insert into t values (1,2,3), (4,5,6), (7,8,9), (10, 11, 12), (13,14,15), (16, 17, 18)")
do, _ := session.GetDomain(s.store)
c.Assert(do.StatsHandle().DumpStatsDeltaToKV(handle.DumpAll), IsNil)
tk.MustExec("analyze table t")

var input []string
var output []struct {
SQL string
Plan []string
}
s.testData.GetTestCases(c, &input, &output)
for i, tt := range input {
s.testData.OnRecord(func() {
output[i].SQL = tt
output[i].Plan = s.testData.ConvertRowsToStrings(tk.MustQuery("explain format = 'brief' " + tt).Rows())
})
tk.MustQuery("explain format = 'brief' " + tt).Check(testkit.Rows(output[i].Plan...))
}
}

func (s *testIntegrationSuite) TestIssues27130(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
Loading

0 comments on commit 5369b4e

Please sign in to comment.