Skip to content

Commit

Permalink
*: fix revoke statement for CURRENT_USER() and refine error message (p…
Browse files Browse the repository at this point in the history
…ingcap#24052)

* *: fix revoke statement for CURRENT_USER() and refine error message

planner: support set tidb_allow_mpp to `2` or `ENFORCE` to enforce use mpp mode. (pingcap#24516)

store/tikv: remove use of SchemaAmender option in store/tikv (pingcap#24408)

*: the value of tikv-client.store-liveness-timeout should not less than 0 (pingcap#24244)

store/tikv: remove use of EnableAsyncCommit option in store/tikv (pingcap#24462)

txn: Add txn state's view (pingcap#22908)

planner: ignore lock for temporary table of PointGet and BatchPointGet (pingcap#24540)

store/tikv: remove use of ReplicaRead transaction option in store/tikv (pingcap#24409)

store/driver: move error to single package (pingcap#24549)

ddl: add check table compatibility for temporary table (pingcap#24501)

store/tikv: remove use of IsStatenessReadOnly option in store/tikv (pingcap#24464)

store/tikv: change backoff type for missed tiflash peer. (pingcap#24577)

store/tikv: remove use of MatchStoreLabels transaction option in store/tikv (pingcap#24465)

executor, meta: Allocate auto id for global temporary tables (pingcap#24506)

store/tikv: remove use of SampleStep option in store/tikv (pingcap#24461)

executor: add partition pruning tests for adding and dropping partition operations (pingcap#24573)

ddl: forbid partition on temporary mode before put into queue (pingcap#24565)

ddl: speedup test case TestIndexOnMultipleGeneratedColumn (pingcap#24487)

execution: Fix issue 24439 Inconsistent error with MySQL for GRANT CREATE USER ON <specific db>.* (pingcap#24485)

*: fix errcheck (pingcap#24463)

test: make TestExtractStartTs stable (pingcap#24585)

ddl: forbid recover/flashback temporary tables (pingcap#24518)

executor: fix point_get result on clustered index when new-row-format disabled but new-collation enabled (pingcap#24544)

executor: Improve the performance of appending not fixed columns (pingcap#20969)

*: typo fix (pingcap#24564)

planner/core: refresh stale regions in cache for batch cop response (pingcap#24457)

binlog: DML on temporary tables do not write binlog (pingcap#24570)

store/tikv: remove use of GuaranteeLinearizability option in store/tikv (pingcap#24605)

store/tikv: remove use of CollectRuntimeStats option in store/tikv (pingcap#24604)

store/tikv: move Backoffer into a single package (pingcap#24525)

variables: init cte max recursive deeps in a new session (pingcap#24609)

store/tikv: move transaction options out to /kv (pingcap#24619)

store/driver: move backoff driver into single package so we can use i… (pingcap#24624)

server: close the temporary session in HTTP API to avoid memory leak (pingcap#24339)

store/tikv: use latest PD TS plus one as min commit ts (pingcap#24579)

planner: fix incorrect TableDual plan built from nulleq (pingcap#24596)

ranger: fix the case which could have duplicate ranges (pingcap#24590)

 executor, store: Pass the SQL digest down to pessimistic lock request (pingcap#24380)

kv: remove UnionStore interface (pingcap#24625)

*: enable gosimple linter (pingcap#24617)

txn: avoid the gc resolving pessimistic locks of ongoing transactions (pingcap#24601)

util: fix wrong enum building for index range  (pingcap#24632)

sessionctx: change innodb large prefix default (pingcap#24555)

store: fix data race about KVStore.tikvClient (pingcap#24655)

executor, privileges: Add dynamic privileges to SHOW PRIVILEGES (pingcap#24646)

ddl: refactor rule [4/6] (pingcap#24007)

cmd: ddl_test modify retryCnt from 5 to 20 (pingcap#24662)

executor: add correctness tests about direct reading with ORDER BY and LIMIT (pingcap#24455)

store/tikv: remove options from unionstore (pingcap#24629)

planner: fix wrongly check for update statement (pingcap#24614)

store/tikv: remove CompareTS (pingcap#24657)

planner, privilege: Add security enhanced mode part 4 (pingcap#24416)

executor: add some test cases about partition table dynamic-mode with split-region (pingcap#24665)

planner: fix wrong column offsets when processing dynamic pruning for IndexJoin (pingcap#24659)

*: Add security enhanced mode part 3 (pingcap#24412)

store/tikv: resolve ReplicaReadType dependencies (pingcap#24653)

executor: add test cases about partition table with `expression` (pingcap#24628)

tablecodec: fix write wrong prefix index value when collation is ascii_bin/latin1_bin (pingcap#24578)

*: compatibility with staleread (pingcap#24285)

session: test that temporary tables will also be retried (pingcap#24505)

domain, session: Add new sysvarcache to replace global values cache (pingcap#24359)

ddl, transaction: DDL on temporary tables won't affect transactions (pingcap#24534)

*: implement tidb_bounded_staleness built-in function (pingcap#24328)

executor: add correctness tests for partition table with different joins (pingcap#24673)

expression: fix the spelling of word arithmetical (pingcap#24713)

store/copr: balance region for batch cop task (pingcap#24521)

store, metrics: Add metrics for safetTS updating (pingcap#24687)

sem: add tidbredact log to restricted variables (pingcap#24701)

session: fix dml_batch_size doesn't load the global variable (pingcap#24710)

store/tikv: retry TSO RPC (pingcap#24682)

expression, planner: push cast down to control function with enum type. (pingcap#24542)

executor: add correctness tests about IndexMerge (pingcap#24674)

variable: change default for DefDMLBatchSize tidbOptInt64 call (pingcap#24697)

planner: add partitioning pruning tests for range partitioning (pingcap#24554)

*: add option for enum push down (pingcap#24685)

txn: break dependency from store/tikv to tidb/kv cause by TransactionOption (pingcap#24656)

executor: enhancement for ListInDisk(support writing after reading) (pingcap#24379)

kv: move TxnScope into kv (pingcap#24715)

execution: fix the incorrect use of cached plan for point get (pingcap#24749)

executor: add correctness tests about direct reading with indexJoin (pingcap#24497)

variable:  fix sysvar datarace with deep copy (pingcap#24732)

*: Implementing RENAME USER (pingcap#24413)

infoschema, executor: Add the deadlock table (pingcap#24524)

docs: Some proposal for renaming and configurations for Lock View (pingcap#24718)

planner: add range partition boundaries tests with BETWEEN expression (pingcap#24598)

oracle: simplify timestamp utilities (pingcap#24688)

executor: fix wrong enum key in point get (pingcap#24618)

ranger: fix incorrect enum range for xxx_ci collation (pingcap#24661)

executor: add some test cases about dynamic-mode and apply operator (pingcap#24683)

store/tikv: remove Variables.Hook (pingcap#24758)

ddl: speed up the execution time of `TestBackwardCompatibility`. (pingcap#24704)

*: prepare errors for CTE (pingcap#24763)

expression: support cast real/int as real (pingcap#24670)

executor: add table name in log (pingcap#24666)

expression: add builtin function ``json_pretty`` (pingcap#24675)

ddl: make `TestDropLastVisibleColumns` stable (pingcap#24790)

* ddl: make `TestDropLastVisibleColumns` stable

*: support AS OF TIMESTAMP read-only begin statement (pingcap#24740)

executor: Fix unstable TestTiDBLastTxnInfoCommitMode (pingcap#24779)

planner: add tests for partition range boundaries for LT/GT (pingcap#24574)

test: record random seed in TestIssue20658 (pingcap#24782)

store/tikv/retry: define Config instead of BackoffType (pingcap#24692)

config: ignore tiflash when show config (pingcap#24770)

privileges: improve dynamic privs registration and tests (pingcap#24773)

README: remove the link of TiDB Monthly Update (pingcap#24791)

region_cache: filter peers on tombstone or dropped stores (pingcap#24726)

util/stmtsummary: remove import package tikv (pingcap#24776)

ddl: grammar check for create unsupported temporary table (pingcap#24723)

*: update go.etcd.io/bbolt (pingcap#24799)

ddl: speed up the execution time of `ddl test` and `Test Chunk pingcap#7 ddl-other` (pingcap#24780)

executor: remove the unnecessary use of fmt.Sprintf (pingcap#24815)

executor: fix index join panic on prefix index on some cases (pingcap#24568)
  • Loading branch information
tiancaiamao authored and Howie59 committed May 21, 2021
1 parent 2b3f157 commit f679a36
Show file tree
Hide file tree
Showing 320 changed files with 16,851 additions and 3,882 deletions.
1 change: 1 addition & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@ check-static: tools/bin/golangci-lint
--enable=unused \
--enable=structcheck \
--enable=deadcode \
--enable=gosimple \
$$($(PACKAGE_DIRECTORIES))

check-slow:tools/bin/gometalinter tools/bin/gosec
Expand Down
1 change: 0 additions & 1 deletion README.md
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,6 @@ In addition, you may enjoy following:
- [@PingCAP](https://twitter.com/PingCAP) on Twitter
- Question tagged [#tidb on StackOverflow](https://stackoverflow.com/questions/tagged/tidb)
- The PingCAP Team [English Blog](https://en.pingcap.com/blog) and [Chinese Blog](https://pingcap.com/blog-cn/)
- [TiDB Monthly](https://pingcap.com/weekly/)

For support, please contact [PingCAP](http://bit.ly/contact_us_via_github).

Expand Down
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
2 changes: 1 addition & 1 deletion cmd/ddltest/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ func (s *TestDDLSuite) SetUpSuite(c *C) {
s.procs = make([]*server, *serverNum)

// Set server restart retry count.
s.retryCount = 5
s.retryCount = 20

createLogFiles(c, *serverNum)
err = s.startServers()
Expand Down
3 changes: 0 additions & 3 deletions cmd/explaintest/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/log"
"github.com/pingcap/parser/ast"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -663,8 +662,6 @@ func main() {
log.Fatal(fmt.Sprintf("%s failed", sql), zap.Error(err))
}
}
// Wait global variables to reload.
time.Sleep(domain.GlobalVariableCacheExpiry)

if _, err = mdb.Exec("set sql_mode='STRICT_TRANS_TABLES'"); err != nil {
log.Fatal("set sql_mode='STRICT_TRANS_TABLES' failed", zap.Error(err))
Expand Down
6 changes: 3 additions & 3 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,9 @@ const (
DefHost = "0.0.0.0"
// DefStatusHost is the default status host of TiDB
DefStatusHost = "0.0.0.0"
// Def TableColumnCountLimit is limit of the number of columns in a table
// DefTableColumnCountLimit is limit of the number of columns in a table
DefTableColumnCountLimit = 1017
// Def TableColumnCountLimit is maximum limitation of the number of columns in a table
// DefMaxOfTableColumnCountLimit is maximum limitation of the number of columns in a table
DefMaxOfTableColumnCountLimit = 4096
)

Expand All @@ -73,7 +73,7 @@ var (
"tikv": true,
"unistore": true,
}
// checkTableBeforeDrop enable to execute `admin check table` before `drop table`.
// CheckTableBeforeDrop enable to execute `admin check table` before `drop table`.
CheckTableBeforeDrop = false
// checkBeforeDropLDFlag is a go build flag.
checkBeforeDropLDFlag = "None"
Expand Down
2 changes: 1 addition & 1 deletion ddl/backfilling.go
Original file line number Diff line number Diff line change
Expand Up @@ -677,7 +677,7 @@ func iterateSnapshotRows(store kv.Storage, priority int, t table.Table, version

ver := kv.Version{Ver: version}
snap := store.GetSnapshot(ver)
snap.SetOption(tikvstore.Priority, priority)
snap.SetOption(kv.Priority, priority)

it, err := snap.Iter(firstKey, upperBound)
if err != nil {
Expand Down
3 changes: 1 addition & 2 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ import (
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -1346,7 +1345,7 @@ func (w *updateColumnWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t
errInTxn = kv.RunInNewTxn(context.Background(), w.sessCtx.GetStore(), true, func(ctx context.Context, txn kv.Transaction) error {
taskCtx.addedCount = 0
taskCtx.scanCount = 0
txn.SetOption(tikvstore.Priority, w.priority)
txn.SetOption(kv.Priority, w.priority)

rowRecords, nextKey, taskDone, err := w.fetchRowColVals(txn, handleRange)
if err != nil {
Expand Down
21 changes: 12 additions & 9 deletions ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,15 +47,18 @@ type testColumnChangeSuite struct {
func (s *testColumnChangeSuite) SetUpSuite(c *C) {
SetWaitTimeWhenErrorOccurred(1 * time.Microsecond)
s.store = testCreateStore(c, "test_column_change")
s.dbInfo = &model.DBInfo{
Name: model.NewCIStr("test_column_change"),
ID: 1,
}
err := kv.RunInNewTxn(context.Background(), s.store, true, func(ctx context.Context, txn kv.Transaction) error {
t := meta.NewMeta(txn)
return errors.Trace(t.CreateDatabase(s.dbInfo))
})
c.Check(err, IsNil)
d := testNewDDLAndStart(
context.Background(),
c,
WithStore(s.store),
WithLease(testLease),
)
defer func() {
err := d.Stop()
c.Assert(err, IsNil)
}()
s.dbInfo = testSchemaInfo(c, d, "test_index_change")
testCreateSchema(c, testNewContext(d), d, s.dbInfo)
}

func (s *testColumnChangeSuite) TearDownSuite(c *C) {
Expand Down
3 changes: 1 addition & 2 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,8 +54,7 @@ func (s *testColumnSuite) SetUpSuite(c *C) {

s.dbInfo = testSchemaInfo(c, d, "test_column")
testCreateSchema(c, testNewContext(d), d, s.dbInfo)
err := d.Stop()
c.Assert(err, IsNil)
c.Assert(d.Stop(), IsNil)
}

func (s *testColumnSuite) TearDownSuite(c *C) {
Expand Down
2 changes: 0 additions & 2 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1063,7 +1063,6 @@ func (s *testStateChangeSuite) TestParallelAddGeneratedColumnAndAlterModifyColum
_, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0")
c.Assert(err, IsNil)
}()
domain.GetDomain(s.se).GetGlobalVarsCache().Disable()

sql1 := "ALTER TABLE t ADD COLUMN f INT GENERATED ALWAYS AS(a+1);"
sql2 := "ALTER TABLE t MODIFY COLUMN a tinyint;"
Expand All @@ -1083,7 +1082,6 @@ func (s *testStateChangeSuite) TestParallelAlterModifyColumnAndAddPK(c *C) {
_, err = s.se.Execute(context.Background(), "set global tidb_enable_change_column_type = 0")
c.Assert(err, IsNil)
}()
domain.GetDomain(s.se).GetGlobalVarsCache().Disable()

sql1 := "ALTER TABLE t ADD PRIMARY KEY (b) NONCLUSTERED;"
sql2 := "ALTER TABLE t MODIFY COLUMN b tinyint;"
Expand Down
Loading

0 comments on commit f679a36

Please sign in to comment.