diff --git a/ddl/reorg.go b/ddl/reorg.go index 1ff7af457cedb..b1437972976ed 100644 --- a/ddl/reorg.go +++ b/ddl/reorg.go @@ -62,18 +62,16 @@ type reorgCtx struct { // 0: job is not canceled. // 1: job is canceled. notifyCancelReorgJob int32 - // doneHandle is used to simulate the handle that has been processed. - + // doneKey is used to record the key that has been processed. doneKey atomic.Value // nullable kv.Key // element is used to record the current element in the reorg process, it can be // accessed by reorg-worker and daemon-worker concurrently. element atomic.Value - // warnings is used to store the warnings when doing the reorg job under - // a certain SQL Mode. mu struct { sync.Mutex + // warnings are used to store the warnings when doing the reorg job under certain SQL modes. warnings map[errors.ErrorID]*terror.Error warningsCount map[errors.ErrorID]int64 } @@ -241,7 +239,6 @@ func (w *worker) runReorgJob(rh *reorgHandler, reorgInfo *reorgInfo, tblInfo *mo } rowCount, _, _ := rc.getRowCountAndKey() logutil.BgLogger().Info("[ddl] run reorg job done", zap.Int64("handled rows", rowCount)) - // Update a job's RowCount. job.SetRowCount(rowCount) // Update a job's warnings. @@ -356,7 +353,7 @@ func getTableTotalCount(w *worker, tblInfo *model.TableInfo) int64 { func (dc *ddlCtx) isReorgRunnable(job *model.Job) error { if isChanClosed(dc.ctx.Done()) { - // Worker is closed. So it can't do the reorganizational job. + // Worker is closed. So it can't do the reorganization. return dbterror.ErrInvalidWorker.GenWithStack("worker is closed") } diff --git a/docs/design/2018-07-01-refactor-aggregate-framework.md b/docs/design/2018-07-01-refactor-aggregate-framework.md index b3dbdef80ae19..cc7bd5f715ca1 100644 --- a/docs/design/2018-07-01-refactor-aggregate-framework.md +++ b/docs/design/2018-07-01-refactor-aggregate-framework.md @@ -46,7 +46,7 @@ Advantages: - That the partial result can be any type also means the aggregate functions can use a map with the specific input type as keys. For example, use `map[types.MyDecimal]` to de-duplicate the input decimal values. In this way, the overhead on the encoding and decoding operations on the old framework can be reduced. - `UpdatePartialResult` is called with a batch of input records. The overhead caused by the per record function call on the framework can be saved. Since all the execution operators are using `Chunk` to store the input rows, in which the data belonging to the same column is stored consecutively in the memory, the aggregate functions should be executed one by one, take full utilization of the CPU caches, reduce the cache misses, and improve the execution performance. - For every state and every kind of input type, a specific aggregate function should be implemented to handle it. This means the CPU branch predictions on the aggregate state and input value types can reduce during the execution of `UpdatePartialResult`, utilize the CPU pipelines, and improve the execution speed. -- `AppendFinalResult2Chunk` directly finalizes the partial result to the chunk, without converting it to `Datum` and then converting the `Datum` back into `Chunk`. This saves a lot of object allocation, reduces the overhead of golang’s gc worker, and avoids the unnecessary value convertings between `Datum` and `Chunk`. +- `AppendFinalResult2Chunk` directly finalizes the partial result to the chunk, without converting it to `Datum` and then converting the `Datum` back into `Chunk`. This saves a lot of object allocation, reduces the overhead of golang’s gc worker, and avoids the unnecessary value conversions between `Datum` and `Chunk`. Disadvantages: - An aggregate function need to be implemented for every possible state and input type. This could introduces a lot of development work. And it takes more coding work to add a new aggregate function. diff --git a/docs/design/2018-10-20-join-reorder-dp-v1.md b/docs/design/2018-10-20-join-reorder-dp-v1.md index 77bea66cc044c..da8c00c834599 100644 --- a/docs/design/2018-10-20-join-reorder-dp-v1.md +++ b/docs/design/2018-10-20-join-reorder-dp-v1.md @@ -62,7 +62,7 @@ Here the set of node is shown as an integer. If the i-th bit of its binary repre ### How other systems solve the same issue? -DP-SUB is the most commonlly used solution in open source databases. +DP-SUB is the most commonly used solution in open source databases. ### What is the disadvantage of this design? diff --git a/docs/design/2019-04-11-indexmerge.md b/docs/design/2019-04-11-indexmerge.md index da6390949880a..8ba243da5e700 100644 --- a/docs/design/2019-04-11-indexmerge.md +++ b/docs/design/2019-04-11-indexmerge.md @@ -206,7 +206,7 @@ GetIndexMergePartialPath(IndexInfos, indexAccessPaths) { // condition : a < 1 or a > 2 or b < 1 or b > 10 // imPaths will be [a<1,a>2,b<1,b>10] and we can merge it and get [a<1 or a >2 , b < 1 or b > 10] // (2)IndexMergePath.tableFilters: -// <1> Remove a condition from PushdownConditions and the rest will be added to tableFitler. +// <1> Remove a condition from PushdownConditions and the rest will be added to tableFilter. // <2> After the merge operation, if any indexPath's tableFilter is not nil, we should add it into tableFilters CreateIndexMergeUnionPath(imPaths,PushdownConditions,cond,IndexInfos) { @@ -227,7 +227,7 @@ GetIndexMergeIntersectionPaths(pushDownConditions, usedConditionsInOr, indexInfo for cond in newConsiderConditions { indexPaths = buildAccessPath([]{cond}, indexInfos) if indexPaths == nil { - tableFiltes = append(tableFilters,cond) + tableFilters = append(tableFilters,cond) continue } indexPath := GetIndexMergePartialPath(indexPaths,indexInfos) @@ -260,7 +260,7 @@ The graph below illustrates an execution of IndexMerge scan. Execution Model -Every index plan in `PhysicalIndexMergeLookUpReader` will start an `IndexWorker` to execute the IndexScan plan and send handles to AndOrWorker. AndOrWorker is responsible for performing set operations (and, or) to getting final handles. Then `AndOrWoker` sends final handles to `TableWokers` to get rows from TiKV. +Every index plan in `PhysicalIndexMergeLookUpReader` will start an `IndexWorker` to execute the IndexScan plan and send handles to AndOrWorker. AndOrWorker is responsible for performing set operations (and, or) to getting final handles. Then `AndOrWoker` sends final handles to `TableWorkers` to get rows from TiKV. Here are some designs for index plans in pipeline mode to be executed without considering the order. diff --git a/docs/design/2019-11-14-tidb-builtin-diagnostics.md b/docs/design/2019-11-14-tidb-builtin-diagnostics.md index d421c625a03b7..c60b95e177162 100644 --- a/docs/design/2019-11-14-tidb-builtin-diagnostics.md +++ b/docs/design/2019-11-14-tidb-builtin-diagnostics.md @@ -190,7 +190,7 @@ In this proposal, we are opt to the second solution. Although it is more difficu 1. Add the `remote-metrics-storage` configuration to the PD and temporarily configure it as the address of the Prometheus Server. PD acts as a proxy, and the request is transferred to Prometheus for execution. The main considerations are as follows: - - PD will have its own implementation of the query interface to realize bootstraping. No other changes needed for TiDB. + - PD will have its own implementation of the query interface to realize bootstrapping. No other changes needed for TiDB. - With bootstrapping realized, users can still use SQL to query monitoring information and diagnostic frameworks without relying on the Prometheus component deployed by TiDB 2. Extract the modules for persisting and querying Prometheus time series data and embed it in PD. @@ -216,7 +216,7 @@ However, because TiKV plan to completely removes the HTTP API in the future, onl ```proto // Diagnostics service for TiDB cluster components. service Diagnostics { - // Searchs log in the target node + // Searches log in the target node rpc search_log(SearchLogRequest) returns (SearchLogResponse) {}; // Retrieves server info in the target node rpc server_info(ServerInfoRequest) returns (ServerInfoResponse) {}; diff --git a/docs/design/2020-01-24-collations.md b/docs/design/2020-01-24-collations.md index 63f850cecc702..a222035745b83 100644 --- a/docs/design/2020-01-24-collations.md +++ b/docs/design/2020-01-24-collations.md @@ -76,7 +76,7 @@ Before diving into the details, we should notice that ALL strings in MySQL(and T + update like/regex/string comparison related functions in `expression` package. + update the comparisons logic in Join/Aggregation executors in `executor` package. + update the codes in `UnionScan`(the internal buffer of transaction). - - SQL Optimizer: the optimizer may need to be alignd with the encoding changes in `planner` package. + - SQL Optimizer: the optimizer may need to be aligned with the encoding changes in `planner` package. - DDL/Schema: check if the new collation is supported according to the versions of tables/columns, the related codes are in `ddl` and `infoschema` package. - Misc + update string comparison related functions in `util` package. @@ -181,7 +181,7 @@ In this proposal, both of the compatibility issues can be solved in the new vers Based on the requirements listed above, the following behaviors are proposed: -1. Only TiDB clusters that initially boostrapped with the new TiDB version are allowed to enable the new collations. For old TiDB clusters, everything remains unchanged after the upgrade. +1. Only TiDB clusters that initially bootstrapped with the new TiDB version are allowed to enable the new collations. For old TiDB clusters, everything remains unchanged after the upgrade. 2. We can also provide a configuration entry for the users to choose between old/new collations when deploying the new clusters. @@ -197,7 +197,7 @@ The main reason to reject those options are: new collations that are not exist i ##### Option 1 -Add a series of new collatins named with the suffix "`_np_bin`"(meaning "NO PADDING BINARY"), for example, `utf8mb4_np_bin`. Such new collations don't exist in MySQL, but they're the "real ones" used by current TiDB. After upgrading to newer TiDB versions, all old collations are shown as "`_np_bin`", MySQL collations behave the same with MySQL. +Add a series of new collations named with the suffix "`_np_bin`"(meaning "NO PADDING BINARY"), for example, `utf8mb4_np_bin`. Such new collations don't exist in MySQL, but they're the "real ones" used by current TiDB. After upgrading to newer TiDB versions, all old collations are shown as "`_np_bin`", MySQL collations behave the same with MySQL. Pros: Requirement 1.a, 2, 3 and 4 are met. @@ -254,8 +254,8 @@ The plan depends on the option chosen in [Compatibility](#compatibility) chapter * For existing TiDB clusters with current binary collations, nothing need to be done if the users are happy with them. * For the potential migrations from MySQL to TiDB: - - If the collations used in MySQL have been implemented by TiDB, users from MySQL do not need to care about the collations when mirgrating to TiDB except Compatibility Option 2, in which the those collations need to be updated to their corresponding names. - - If there are colltions that are not supported by TiDB yet, users may need to change the them to the supported ones and check if no constraint is broken after the change. The check can be done following the approach mentioned in [this article](https://mysqlserverteam.com/mysql-8-0-collations-migrating-from-older-collations). + - If the collations used in MySQL have been implemented by TiDB, users from MySQL do not need to care about the collations when migrating to TiDB except Compatibility Option 2, in which the those collations need to be updated to their corresponding names. + - If there are collations that are not supported by TiDB yet, users may need to change the them to the supported ones and check if no constraint is broken after the change. The check can be done following the approach mentioned in [this article](https://mysqlserverteam.com/mysql-8-0-collations-migrating-from-older-collations). ## Testing Plans diff --git a/docs/design/2020-05-11-heap-profile-record.md b/docs/design/2020-05-11-heap-profile-record.md index dc06b21ee665c..c4a8f9885f141 100644 --- a/docs/design/2020-05-11-heap-profile-record.md +++ b/docs/design/2020-05-11-heap-profile-record.md @@ -24,7 +24,7 @@ And `runtime.MemProfileRate` controls the fraction of memory allocations that ar To verify whether `kvcache.(*SimpleLRUCache).Put` would reflect the real heap usage, I use following test to ensure it: -1. fufill the `SimpleLRUCache` by `set @randomString = ? with 20000 times`. +1. fulfill the `SimpleLRUCache` by `set @randomString = ? with 20000 times`. 2. profile the heap Usage of `github.com/pingcap/tidb/util/kvcache.(*SimpleLRUCache).Put` and the result is 2.55 MB Let's dig into the Put then we can find the where the heap consumed: @@ -60,7 +60,7 @@ ROUTINE ======================== github.com/pingcap/tidb/util/kvcache.(*SimpleLR . . 114: if l.quota == 0 { ``` -We can find that the `hash` (the key of cache) and the `element`(the value of the cache) totolly consume 2.55 MB. +We can find that the `hash` (the key of cache) and the `element`(the value of the cache) totally consume 2.55 MB. 3. we use [sizeof](https://github.com/templarbit/sizeof) (the result is similar, but lower, not exact) to calculate the size of each key and element is 80byte and 40byte. 4. As 2.28 MB (120 byte * 20000) is similar to the 2.55MB, we can ensure that the heap profile would reflect the heap usage of `SimpleLRUCache`. diff --git a/docs/design/2021-03-09-security-enhanced-mode.md b/docs/design/2021-03-09-security-enhanced-mode.md index 8ed0d457efbd3..337164a7bdc9e 100644 --- a/docs/design/2021-03-09-security-enhanced-mode.md +++ b/docs/design/2021-03-09-security-enhanced-mode.md @@ -20,7 +20,7 @@ ## Introduction -This document was created to discuss the design of Security Enhanced Mode. It comes from the DBaaS requirement that `SUPER` users must not be able to perform certain actions that could comprimise the system. +This document was created to discuss the design of Security Enhanced Mode. It comes from the DBaaS requirement that `SUPER` users must not be able to perform certain actions that could compromise the system. ### Terminology diff --git a/docs/design/README.md b/docs/design/README.md index 34fe07a336c6c..4a1be6ae690f0 100644 --- a/docs/design/README.md +++ b/docs/design/README.md @@ -16,7 +16,7 @@ A hastily-proposed design document can hurt its chances of acceptance. Low-quali Although there is no single way to prepare for submitting a design document, it is generally a good idea to pursue feedback from other project developers beforehand, to ascertain that the design document may be desirable; having a consistent impact on the project requires concerted effort toward consensus-building. -The most common preparations for writing and submitting an design document for now is [creating a disucss issue](https://github.com/pingcap/tidb/issues/new/choose), which is going to be converted into a tracking issue of the design implementation. +The most common preparations for writing and submitting a design document for now is [creating an issue for discussion](https://github.com/pingcap/tidb/issues/new/choose), which is going to be converted into a tracking issue of the design implementation. ## What is the process? diff --git a/executor/simple.go b/executor/simple.go index 6e2265c9678e0..0290c76769d3e 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -1293,6 +1293,14 @@ func (e *SimpleExec) executeDropUser(ctx context.Context, s *ast.DropUserStmt) e break } + // delete privileges from mysql.columns_priv + sql.Reset() + sqlexec.MustFormatSQL(sql, `DELETE FROM %n.%n WHERE Host = %? and User = %?;`, mysql.SystemDB, mysql.ColumnPrivTable, user.Hostname, user.Username) + if _, err = sqlExecutor.ExecuteInternal(context.TODO(), sql.String()); err != nil { + failedUsers = append(failedUsers, user.String()) + break + } + // delete relationship from mysql.role_edges sql.Reset() sqlexec.MustFormatSQL(sql, `DELETE FROM %n.%n WHERE TO_HOST = %? and TO_USER = %?;`, mysql.SystemDB, mysql.RoleEdgeTable, user.Hostname, user.Username) diff --git a/executor/simple_test.go b/executor/simple_test.go index 1ffa5aba69999..847d34668b8d1 100644 --- a/executor/simple_test.go +++ b/executor/simple_test.go @@ -988,6 +988,44 @@ func TestShowGrantsAfterDropRole(t *testing.T) { tk.MustQuery("SHOW GRANTS").Check(testkit.Rows("GRANT CREATE USER ON *.* TO 'u29473'@'%'")) } +func TestPrivilegesAfterDropUser(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1(id int, v int)") + defer tk.MustExec("drop table t1") + + tk.MustExec("CREATE USER u1 require ssl") + defer tk.MustExec("DROP USER IF EXISTS u1") + + tk.MustExec("GRANT CREATE ON test.* TO u1") + tk.MustExec("GRANT UPDATE ON test.t1 TO u1") + tk.MustExec("GRANT SYSTEM_VARIABLES_ADMIN ON *.* TO u1") + tk.MustExec("GRANT SELECT(v), UPDATE(v) on test.t1 TO u1") + + tk.MustQuery("SELECT COUNT(1) FROM mysql.global_grants WHERE USER='u1' AND HOST='%'").Check(testkit.Rows("1")) + tk.MustQuery("SELECT COUNT(1) FROM mysql.global_priv WHERE USER='u1' AND HOST='%'").Check(testkit.Rows("1")) + tk.MustQuery("SELECT COUNT(1) FROM mysql.tables_priv WHERE USER='u1' AND HOST='%'").Check(testkit.Rows("1")) + tk.MustQuery("SELECT COUNT(1) FROM mysql.columns_priv WHERE USER='u1' AND HOST='%'").Check(testkit.Rows("1")) + tk.Session().Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil) + tk.MustQuery("SHOW GRANTS FOR u1").Check(testkit.Rows( + "GRANT USAGE ON *.* TO 'u1'@'%'", + "GRANT CREATE ON test.* TO 'u1'@'%'", + "GRANT UPDATE ON test.t1 TO 'u1'@'%'", + "GRANT SELECT(v), UPDATE(v) ON test.t1 TO 'u1'@'%'", + "GRANT SYSTEM_VARIABLES_ADMIN ON *.* TO 'u1'@'%'", + )) + + tk.MustExec("DROP USER u1") + err := tk.QueryToErr("SHOW GRANTS FOR u1") + require.Equal(t, "[privilege:1141]There is no such grant defined for user 'u1' on host '%'", err.Error()) + tk.MustQuery("SELECT * FROM mysql.global_grants WHERE USER='u1' AND HOST='%'").Check(testkit.Rows()) + tk.MustQuery("SELECT * FROM mysql.global_priv WHERE USER='u1' AND HOST='%'").Check(testkit.Rows()) + tk.MustQuery("SELECT * FROM mysql.tables_priv WHERE USER='u1' AND HOST='%'").Check(testkit.Rows()) + tk.MustQuery("SELECT * FROM mysql.columns_priv WHERE USER='u1' AND HOST='%'").Check(testkit.Rows()) +} + func TestDropRoleAfterRevoke(t *testing.T) { store, clean := testkit.CreateMockStore(t) defer clean() diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index 90adccbc5fa64..3db97ef11f9c0 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -55,7 +55,7 @@ var ( const ( idLen = 8 prefixLen = 1 + idLen /*tableID*/ + 2 - // RecordRowKeyLen is public for calculating avgerage row size. + // RecordRowKeyLen is public for calculating average row size. RecordRowKeyLen = prefixLen + idLen /*handle*/ tablePrefixLength = 1 recordPrefixSepLength = 2