Skip to content

Commit

Permalink
Merge branch 'master' into fix-sysvar-race
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo committed May 19, 2021
2 parents 3ed42bd + 89cf970 commit 30e92fe
Show file tree
Hide file tree
Showing 51 changed files with 1,837 additions and 260 deletions.
3 changes: 1 addition & 2 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/store/tikv/mockstore/cluster"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -1343,7 +1342,7 @@ func getMaxTableHandle(ctx *testMaxTableRowIDContext, store kv.Storage) (kv.Hand
c := ctx.c
d := ctx.d
tbl := ctx.tbl
curVer, err := store.CurrentVersion(oracle.GlobalTxnScope)
curVer, err := store.CurrentVersion(kv.GlobalTxnScope)
c.Assert(err, IsNil)
maxHandle, emptyTable, err := d.GetTableMaxHandle(curVer.Ver, tbl.(table.PhysicalTable))
c.Assert(err, IsNil)
Expand Down
3 changes: 1 addition & 2 deletions ddl/delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/pingcap/tidb/ddl/util"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sqlexec"
Expand Down Expand Up @@ -451,7 +450,7 @@ func doBatchInsert(s sqlexec.SQLExecutor, jobID int64, tableIDs []int64, ts uint

// getNowTS gets the current timestamp, in TSO.
func getNowTSO(ctx sessionctx.Context) (uint64, error) {
currVer, err := ctx.GetStore().CurrentVersion(oracle.GlobalTxnScope)
currVer, err := ctx.GetStore().CurrentVersion(kv.GlobalTxnScope)
if err != nil {
return 0, errors.Trace(err)
}
Expand Down
3 changes: 1 addition & 2 deletions ddl/reorg.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -534,7 +533,7 @@ func getTableRange(d *ddlCtx, tbl table.PhysicalTable, snapshotVer uint64, prior
}

func getValidCurrentVersion(store kv.Storage) (ver kv.Version, err error) {
ver, err = store.CurrentVersion(oracle.GlobalTxnScope)
ver, err = store.CurrentVersion(kv.GlobalTxnScope)
if err != nil {
return ver, errors.Trace(err)
} else if ver.Ver <= 0 {
Expand Down
9 changes: 4 additions & 5 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
Expand Down Expand Up @@ -236,7 +235,7 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req
}
builder.txnScope = sv.TxnCtx.TxnScope
builder.IsStaleness = sv.TxnCtx.IsStaleness
if builder.IsStaleness && builder.txnScope != oracle.GlobalTxnScope {
if builder.IsStaleness && builder.txnScope != kv.GlobalTxnScope {
builder.MatchStoreLabels = []*metapb.StoreLabel{
{
Key: placement.DCLabelKey,
Expand Down Expand Up @@ -279,9 +278,9 @@ func (builder *RequestBuilder) SetFromInfoSchema(is infoschema.InfoSchema) *Requ

func (builder *RequestBuilder) verifyTxnScope() error {
if builder.txnScope == "" {
builder.txnScope = oracle.GlobalTxnScope
builder.txnScope = kv.GlobalTxnScope
}
if builder.txnScope == oracle.GlobalTxnScope || builder.is == nil {
if builder.txnScope == kv.GlobalTxnScope || builder.is == nil {
return nil
}
visitPhysicalTableID := make(map[int64]struct{})
Expand Down Expand Up @@ -600,7 +599,7 @@ func CommonHandleRangesToKVRanges(sc *stmtctx.StatementContext, tids []int64, ra

// VerifyTxnScope verify whether the txnScope and visited physical table break the leader rule's dcLocation.
func VerifyTxnScope(txnScope string, physicalTableID int64, is infoschema.InfoSchema) bool {
if txnScope == "" || txnScope == oracle.GlobalTxnScope {
if txnScope == "" || txnScope == kv.GlobalTxnScope {
return true
}
bundle, ok := is.BundleByName(placement.GroupID(physicalTableID))
Expand Down
3 changes: 1 addition & 2 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/telemetry"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/dbterror"
Expand Down Expand Up @@ -336,7 +335,7 @@ func (do *Domain) Reload() error {
defer do.m.Unlock()

startTime := time.Now()
ver, err := do.store.CurrentVersion(oracle.GlobalTxnScope)
ver, err := do.store.CurrentVersion(kv.GlobalTxnScope)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions domain/domain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -347,7 +347,7 @@ func (*testSuite) TestT(c *C) {

// for schemaValidator
schemaVer := dom.SchemaValidator.(*schemaValidator).LatestSchemaVersion()
ver, err := store.CurrentVersion(oracle.GlobalTxnScope)
ver, err := store.CurrentVersion(kv.GlobalTxnScope)
c.Assert(err, IsNil)
ts := ver.Ver

Expand All @@ -360,7 +360,7 @@ func (*testSuite) TestT(c *C) {
c.Assert(succ, Equals, ResultSucc)
time.Sleep(ddlLease)

ver, err = store.CurrentVersion(oracle.GlobalTxnScope)
ver, err = store.CurrentVersion(kv.GlobalTxnScope)
c.Assert(err, IsNil)
ts = ver.Ver
_, succ = dom.SchemaValidator.Check(ts, schemaVer, nil)
Expand Down
2 changes: 1 addition & 1 deletion domain/infosync/info.go
Original file line number Diff line number Diff line change
Expand Up @@ -554,7 +554,7 @@ func (is *InfoSyncer) ReportMinStartTS(store kv.Storage) {
pl := is.manager.ShowProcessList()

// Calculate the lower limit of the start timestamp to avoid extremely old transaction delaying GC.
currentVer, err := store.CurrentVersion(oracle.GlobalTxnScope)
currentVer, err := store.CurrentVersion(kv.GlobalTxnScope)
if err != nil {
logutil.BgLogger().Error("update minStartTS failed", zap.Error(err))
return
Expand Down
3 changes: 1 addition & 2 deletions executor/batch_point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
driver "github.com/pingcap/tidb/store/driver/txn"
"github.com/pingcap/tidb/store/tikv"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -122,7 +121,7 @@ func (e *BatchPointGetExec) Open(context.Context) error {
snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID)
isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness
snapshot.SetOption(kv.IsStalenessReadOnly, isStaleness)
if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != oracle.GlobalTxnScope {
if isStaleness && e.ctx.GetSessionVars().TxnCtx.TxnScope != kv.GlobalTxnScope {
snapshot.SetOption(kv.MatchStoreLabels, []*metapb.StoreLabel{
{
Key: placement.DCLabelKey,
Expand Down
80 changes: 78 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2693,11 +2693,11 @@ func (s *testSuiteP2) TestHistoryRead(c *C) {
// SnapshotTS Is not updated if check failed.
c.Assert(tk.Se.GetSessionVars().SnapshotTS, Equals, uint64(0))

curVer1, _ := s.store.CurrentVersion(oracle.GlobalTxnScope)
curVer1, _ := s.store.CurrentVersion(kv.GlobalTxnScope)
time.Sleep(time.Millisecond)
snapshotTime := time.Now()
time.Sleep(time.Millisecond)
curVer2, _ := s.store.CurrentVersion(oracle.GlobalTxnScope)
curVer2, _ := s.store.CurrentVersion(kv.GlobalTxnScope)
tk.MustExec("insert history_read values (2)")
tk.MustQuery("select * from history_read").Check(testkit.Rows("1", "2"))
tk.MustExec("set @@tidb_snapshot = '" + snapshotTime.Format("2006-01-02 15:04:05.999999") + "'")
Expand Down Expand Up @@ -8153,3 +8153,79 @@ func (s *testSerialSuite) TestIssue24210(c *C) {
c.Assert(err, IsNil)

}

func (s testSerialSuite) TestExprBlackListForEnum(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 enum('a','b','c'), b enum('a','b','c'), c int, index idx(b,a));")
tk.MustExec("insert into t values(1,1,1),(2,2,2),(3,3,3);")

checkFuncPushDown := func(rows [][]interface{}, keyWord string) bool {
for _, line := range rows {
// Agg/Expr push down
if line[2].(string) == "cop[tikv]" && strings.Contains(line[4].(string), keyWord) {
return true
}
// access index
if line[2].(string) == "cop[tikv]" && strings.Contains(line[3].(string), keyWord) {
return true
}
}
return false
}

// Test agg(enum) push down
tk.MustExec("insert into mysql.expr_pushdown_blacklist(name) values('enum');")
tk.MustExec("admin reload expr_pushdown_blacklist;")
rows := tk.MustQuery("desc format='brief' select /*+ HASH_AGG() */ max(a) from t;").Rows()
c.Assert(checkFuncPushDown(rows, "max"), IsFalse)
rows = tk.MustQuery("desc format='brief' select /*+ STREAM_AGG() */ max(a) from t;").Rows()
c.Assert(checkFuncPushDown(rows, "max"), IsFalse)

tk.MustExec("delete from mysql.expr_pushdown_blacklist;")
tk.MustExec("admin reload expr_pushdown_blacklist;")
rows = tk.MustQuery("desc format='brief' select /*+ HASH_AGG() */ max(a) from t;").Rows()
c.Assert(checkFuncPushDown(rows, "max"), IsTrue)
rows = tk.MustQuery("desc format='brief' select /*+ STREAM_AGG() */ max(a) from t;").Rows()
c.Assert(checkFuncPushDown(rows, "max"), IsTrue)

// Test expr(enum) push down
tk.MustExec("insert into mysql.expr_pushdown_blacklist(name) values('enum');")
tk.MustExec("admin reload expr_pushdown_blacklist;")
rows = tk.MustQuery("desc format='brief' select * from t where a + b;").Rows()
c.Assert(checkFuncPushDown(rows, "plus"), IsFalse)
rows = tk.MustQuery("desc format='brief' select * from t where a + b;").Rows()
c.Assert(checkFuncPushDown(rows, "plus"), IsFalse)

tk.MustExec("delete from mysql.expr_pushdown_blacklist;")
tk.MustExec("admin reload expr_pushdown_blacklist;")
rows = tk.MustQuery("desc format='brief' select * from t where a + b;").Rows()
c.Assert(checkFuncPushDown(rows, "plus"), IsTrue)
rows = tk.MustQuery("desc format='brief' select * from t where a + b;").Rows()
c.Assert(checkFuncPushDown(rows, "plus"), IsTrue)

// Test enum index
tk.MustExec("insert into mysql.expr_pushdown_blacklist(name) values('enum');")
tk.MustExec("admin reload expr_pushdown_blacklist;")
rows = tk.MustQuery("desc format='brief' select * from t where b = 1;").Rows()
c.Assert(checkFuncPushDown(rows, "index:idx(b)"), IsFalse)
rows = tk.MustQuery("desc format='brief' select * from t where b = 'a';").Rows()
c.Assert(checkFuncPushDown(rows, "index:idx(b)"), IsFalse)
rows = tk.MustQuery("desc format='brief' select * from t where b > 1;").Rows()
c.Assert(checkFuncPushDown(rows, "index:idx(b)"), IsFalse)
rows = tk.MustQuery("desc format='brief' select * from t where b > 'a';").Rows()
c.Assert(checkFuncPushDown(rows, "index:idx(b)"), IsFalse)

tk.MustExec("delete from mysql.expr_pushdown_blacklist;")
tk.MustExec("admin reload expr_pushdown_blacklist;")
rows = tk.MustQuery("desc format='brief' select * from t where b = 1 and a = 1;").Rows()
c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue)
rows = tk.MustQuery("desc format='brief' select * from t where b = 'a' and a = 'a';").Rows()
c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue)
rows = tk.MustQuery("desc format='brief' select * from t where b = 1 and a > 1;").Rows()
c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue)
rows = tk.MustQuery("desc format='brief' select * from t where b = 1 and a > 'a'").Rows()
c.Assert(checkFuncPushDown(rows, "index:idx(b, a)"), IsTrue)
}
Loading

0 comments on commit 30e92fe

Please sign in to comment.