Skip to content

Commit

Permalink
Merge branch 'master' into fk-explain-analyze
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored Nov 30, 2022
2 parents 30db64e + f5487e3 commit fd22612
Show file tree
Hide file tree
Showing 16 changed files with 289 additions and 24 deletions.
4 changes: 2 additions & 2 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -2915,8 +2915,8 @@ def go_deps():
name = "com_github_pingcap_kvproto",
build_file_proto_mode = "disable_global",
importpath = "github.com/pingcap/kvproto",
sum = "h1:LzIZsQpXQlj8yF7+yvyOg680OaPq7bmPuDuszgXfHsw=",
version = "v0.0.0-20221129023506-621ec37aac7a",
sum = "h1:46ZD6xzQWJ8Jkeal/U7SqkX030Mgs8DAn6QV/9zbqOQ=",
version = "v0.0.0-20221130022225-6c56ac56fe5f",
)
go_repository(
name = "com_github_pingcap_log",
Expand Down
1 change: 1 addition & 0 deletions autoid_service/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
"//util/logutil",
"//util/mathutil",
"@com_github_pingcap_errors//:errors",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_kvproto//pkg/autoid",
"@io_etcd_go_etcd_client_v3//:client",
"@org_golang_google_grpc//:grpc",
Expand Down
26 changes: 22 additions & 4 deletions autoid_service/autoid.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"time"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/autoid"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
Expand Down Expand Up @@ -401,9 +402,16 @@ func (s *Service) getAlloc(dbID, tblID int64, isUnsigned bool) *autoIDValue {

func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (*autoid.AutoIDResponse, error) {
if s.leaderShip != nil && !s.leaderShip.IsOwner() {
logutil.BgLogger().Info("[autoid service] Alloc AutoID fail, not leader")
return nil, errors.New("not leader")
}

failpoint.Inject("mockErr", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(nil, errors.New("mock reload failed"))
}
})

val := s.getAlloc(req.DbID, req.TblID, req.IsUnsigned)

if req.N == 0 {
Expand All @@ -426,10 +434,13 @@ func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (*
val.end = currentEnd
return nil
})
if err != nil {
return &autoid.AutoIDResponse{Errmsg: []byte(err.Error())}, nil
}
return &autoid.AutoIDResponse{
Min: currentEnd,
Max: currentEnd,
}, err
}, nil
}

val.Lock()
Expand All @@ -443,10 +454,13 @@ func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (*
min, max, err = val.alloc4Signed(ctx, s.store, req.DbID, req.TblID, req.IsUnsigned, req.N, req.Increment, req.Offset)
}

if err != nil {
return &autoid.AutoIDResponse{Errmsg: []byte(err.Error())}, nil
}
return &autoid.AutoIDResponse{
Min: min,
Max: max,
}, err
}, nil
}

func (alloc *autoIDValue) forceRebase(ctx context.Context, store kv.Storage, dbID, tblID, requiredBase int64, isUnsigned bool) error {
Expand Down Expand Up @@ -478,14 +492,15 @@ func (alloc *autoIDValue) forceRebase(ctx context.Context, store kv.Storage, dbI
// req.N = 0 is handled specially, it is used to return the current auto ID value.
func (s *Service) Rebase(ctx context.Context, req *autoid.RebaseRequest) (*autoid.RebaseResponse, error) {
if s.leaderShip != nil && !s.leaderShip.IsOwner() {
logutil.BgLogger().Info("[autoid service] Rebase() fail, not leader")
return nil, errors.New("not leader")
}

val := s.getAlloc(req.DbID, req.TblID, req.IsUnsigned)
if req.Force {
err := val.forceRebase(ctx, s.store, req.DbID, req.TblID, req.Base, req.IsUnsigned)
if err != nil {
return nil, errors.Trace(err)
return &autoid.RebaseResponse{Errmsg: []byte(err.Error())}, nil
}
}

Expand All @@ -495,5 +510,8 @@ func (s *Service) Rebase(ctx context.Context, req *autoid.RebaseRequest) (*autoi
} else {
err = val.rebase4Signed(ctx, s.store, req.DbID, req.TblID, req.Base)
}
return &autoid.RebaseResponse{}, err
if err != nil {
return &autoid.RebaseResponse{Errmsg: []byte(err.Error())}, nil
}
return &autoid.RebaseResponse{}, nil
}
2 changes: 2 additions & 0 deletions br/pkg/gluetidb/glue.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,8 @@ func (gs *tidbSession) CreateTables(ctx context.Context, tables map[string][]*mo
d := domain.GetDomain(gs.se).DDL()
var dbName model.CIStr

// Disable foreign key check when batch create tables.
gs.se.GetSessionVars().ForeignKeyChecks = false
for db, tablesInDB := range tables {
dbName = model.NewCIStr(db)
queryBuilder := strings.Builder{}
Expand Down
55 changes: 55 additions & 0 deletions br/tests/br_foreign_key/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
#!/bin/sh
#
# Copyright 2022 PingCAP, Inc.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

set -eu
DB="$TEST_NAME"

run_sql "set @@global.tidb_enable_foreign_key=1;"
run_sql "set @@global.foreign_key_checks=1;"
run_sql "set @@foreign_key_checks=1;"
run_sql "create schema $DB;"
run_sql "create table $DB.t1 (id int key);"
run_sql "create table $DB.t2 (id int key, a int, b int, foreign key fk_1 (a) references t1(id) ON UPDATE SET NULL ON DELETE SET NULL, foreign key fk_2 (b) references t1(id) ON DELETE CASCADE ON UPDATE CASCADE);"
run_sql "insert into $DB.t1 values (1), (2), (3);"
run_sql "insert into $DB.t2 values (1, 1, 1), (2, 2, 2), (3, 3, 3);"
run_sql "update $DB.t1 set id=id+10 where id in (1, 3);"
run_sql "delete from $DB.t1 where id = 2;"

echo "backup start..."
run_br backup db --db "$DB" -s "local://$TEST_DIR/$DB" --pd $PD_ADDR

run_sql "drop schema $DB;"

echo "restore start..."
run_br restore db --db $DB -s "local://$TEST_DIR/$DB" --pd $PD_ADDR

set -x

run_sql "select count(*) from $DB.t1;"
check_contains 'count(*): 2'

run_sql "select count(*) from $DB.t2;"
check_contains 'count(*): 2'

run_sql "select id, a, b from $DB.t2;"
check_contains 'id: 1'
check_contains 'id: 3'
check_contains 'a: NULL'
check_contains 'b: 11'
check_contains 'b: 13'
check_contains 'b: 14'

run_sql "drop schema $DB"
134 changes: 134 additions & 0 deletions docs/design/2022-11-22-view-hint.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
# View Hint Design Doc
- Author: Reminiscent
- Tracking Issue: https://github.com/pingcap/tidb/issues/37887

## Background
Hints that specify a table generally refer to tables in the DELETE, SELECT, or UPDATE query block in which the hint occurs, not to tables inside any views referenced by the statement. So we introduce the view hint to specify the table in view instead of embedding the hint in the view.

In Oracle, there are three ways to use the global hint. (Node: the `{}` part is only used for explanation)
```SQL
CREATE OR REPLACE VIEW v AS
SELECT {SEL$2} * from e1 join (select {SEL$3} * from e3) e2 on e1.a = e2.a;

SELECT {SEL$1} * FROM v;

A. SELECT /*+ INDEX(v.e2.e3 idx) */ * FROM v; // /*+ INDEX(@SEL$1 v.e2.e3 idx) */

B. SELECT /*+ INDEX(@SEL$2 e2.e3 idx) */ * FROM v;

C. SELECT /*+ INDEX(@SEL$3 e3 idx) */ * FROM v;
```

Compared with TiDB, Oracle has two differences:
1. Oracle can use `alias.table` to represent in subquery, such as e2.e3. Besides, TiDB can use `db.table` to represent a table.
2. The count for query block number treats view like a subquery, which means the select parts in view are counted.

Based on the difference, there are some reasons why TiDB can not just use the grammar from Oracle:
1. Compatibility
1. Grammar.
1. We can not use the `alias.table` to represent in subquery, such as e2.e3.
2. We can use `db.table` to represent a table. So if we want to use the `view.table` to represent a table in view, we should change the grammar or it will conflict with db.table.
2. The count for the query block.
1. Previously, the query block in view would not be counted. But now, if we take the view into consideration, it will change the origin count. For example, in the following part. The count of the query block for the `select a from t1` will be changed from `@SEL_2` to `@SEL_3`. So if we use the query block related hints for this part, it will be invalid or represent the content in the view.

```SQL
CREATE OR REPLACE VIEW v AS
SELECT {SEL$2} * FROM t;

SELECT {SEL$1} * FROM v JOIN (select {SEL$3} a from t1) t2 on v.a = t2.a;
```

So based on the above reasons, we should introduce another way to let hint take effect in the view.

## Detailed Design
### How does origin hint framework work?
1. Parser: parse the sql text and get the basic information about the hint. Handle hint syntax error in this phase.
2. Optimizer:
1. Divide and mark the query block. Besides, group the hints in the same query blocks.
2. In the plan builder phase, when we try to build select. We will handle the hints in the current query block. Including doing some simple checks and building the hints structure which can be used by planner.
3. When we build some logical operators, we will use the hints which belongs to the current query block. And tt will use the table in hint to match the table in the plan node. For example, when we build the `DataSource` operator, it will generate the possible access path based on the index hints. When we build the `Aggregation` operator, it will set the aggregation algorithm based on the agg hints. And for the `Join` operator, it will store the hint in the join node and use the hint information in the physical optimization phase. The warning about which table is not used in the hint will be recorded in this phase.
4. Use the hint information in the physical optimization phase to determine which physical algorithm should be used. And if the hint can not take effect, it will report warning. For example, if the join can not use the index join, but we set the index join hint in the sql text. It will report related warnings.

### View Hint Design
Based on the goal and current infrastructure for hint. I extend the current usage of the qb_name hint to a bigger scope to support the view hint.

An example to show the usage of the current `qb_name` hint.
```SQL
select /*+ stream_agg(@qb) merge_join(t1@qb)*/ * from (select /*+ qb_name(qb) */ count(*) from t1 join t2 on t1.a = t2.a) tt;
```
1. First, we define the name for some query blocks.
2. Then we can use the query block name to represent the query block.

Based on the meaning of `qb_name` hint now, we can expand it to support the view. The basic idea is the same here. We define the query block name in the view first. And then we can use the query block name to represent the contents in the view. Now the grammar is expanded from
`qb_name(name)` in the query block which you want to rename
To
`qb_name(name, viewName@queryBlockNum . {viewName}@queryBlockNum . ...)` in the first query block to represent any query block. Besides, we will reset the count for query block in every view. It means, for every view, it always counts from 1 and it will not effect the outer part.
For example:
```SQL
create table t(a int, b int);
create table t1(a int, b int);
create table t2(a int, b int);

create view v as select {@SEL_1}{5} t.a, t.b from t join (select {@SEL_2}{6} t1.a from t1 join t2 on t1.b=t2.b) tt on t.a = tt.a;

create view v1 as select {@SEL_1}{3} t.a, t.b from t join (select {@SEL_2}{4} from t1 join v on t1.b=v.b) tt on t.a = tt.a;

create view v2 as select {@SEL_1}{1} t.a, t.b from t join (select {@SEL_2}{2} t1.a from t1 join v1 join v3 on t1.b=v1.b) tt on t.a = tt.a;

select {@SEL_1} * from v2;

/* We can use the following part to represent the {1} - {6} */
1: qb_name(v2_sel1, v2@sel_1 . @sel_1)
2: qb_name(v2_sel2, v2@sel_1 . @sel_2)
3: qb_name(v1_sel1, v2@sel_1 . v1@sel_2 . @sel_1)
4: qb_name(v1_sel2, v2@sel_1 . v1@sel_2 . @sel_2)
5: qb_name(v_sel1, v2@sel_1 . v1@sel_2 . v@sel_2 . @sel_1)
6: qb_name(v_sel2, v2@sel_1 . v1@sel_2 . v@sel_2 . @sel_2)
```
Take the previous as example:
```SQL
CREATE OR REPLACE VIEW v AS
SELECT * from e1 join (select count(*) from e3) e2 on e1.a = e2.a;


/* In Oracle */
A1. SELECT /*+ INDEX(v.e2.e3 idx) */ * FROM v;

A2. SELECT /*+ INDEX(@SEL$1 v.e2.e3 idx) */ * FROM v;

B. SELECT /*+ INDEX(@SEL$2 e2.e3 idx) */ * FROM v;

C. SELECT /*+ INDEX(@SEL$3 e3 idx) */ * FROM v;

/* In TiDB */
SELECT /*+ qb_name(viewSub, v@sel_1 . @sel_2) use_index(e3@viewSub, idx) hash_agg(viewSub) */ * FROM v;
```

### Implementation
Parser part is easy to implement. Just to expand the origin `qb_name` hint grammar. The only problem maybe is how to express the nested view(use dot or blank or something else).

For the planner part:
1. At the beginning of the optimization, we should handle the query block name hint for view and the other method hints for view. And group these hints based on the query block name.
2. When we try to build the data source from the view, we have to traverse all of the query blocks for views. Check whether the view name in hint can match the data source or not. If there are some hints that can match, we pass it to the `buildDataSourceFromView`.
3. When we try to build the view plan, we first handle the hints which are passed by the caller. Distinguish which hints belong to the current view and which belongs to the nested view. If the hint belongs to the current view, we transform the hint to the normal hint. If the hints belong to the nested view. Then we will do the same thing, like step2.

Besides the planner part, we need support to show the query block for a sql to increase usability. The user can copy the result and use it in hint directly.

### Support Scope
1. We can support almost all physical algorithm's hints. Like join hints/ agg hints/ index etc.
2. Do not support the leading hints which may be across the view. But we can support the leading hint in the same view.

### Pros and Cons
Pros:
1. No compatibility problems. Just expand the usage of the existing hint.
2. It is easier to implement. It can use the origin hints' infrastructure as much as possible.
3. It can support almost all the hints which can take effect in the query block. Oracle can only support the join order, join method and access path hints.

Cons:
1. It may not be easy to write the query block name hint for a view.
2. The user should define the query block name hint first.

## Reference
[Oracle Global Hint](https://docs.oracle.com/cd/E18283_01/server.112/e16638/hintsref.htm#i27644)


16 changes: 14 additions & 2 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -469,8 +469,20 @@ func (a *ExecStmt) Exec(ctx context.Context) (_ sqlexec.RecordSet, err error) {
if !ok {
oriIso = "REPEATABLE-READ"
}
terror.Log(sctx.GetSessionVars().SetSystemVar(variable.TiDBBuildStatsConcurrency, "1"))
sctx.GetSessionVars().SetDistSQLScanConcurrency(1)
autoConcurrency, err1 := sctx.GetSessionVars().GetSessionOrGlobalSystemVar(ctx, variable.TiDBAutoBuildStatsConcurrency)
terror.Log(err1)
if err1 == nil {
terror.Log(sctx.GetSessionVars().SetSystemVar(variable.TiDBBuildStatsConcurrency, autoConcurrency))
}
sVal, err2 := sctx.GetSessionVars().GetSessionOrGlobalSystemVar(ctx, variable.TiDBSysProcScanConcurrency)
terror.Log(err2)
if err2 == nil {
concurrency, err3 := strconv.ParseInt(sVal, 10, 64)
terror.Log(err3)
if err3 == nil {
sctx.GetSessionVars().SetDistSQLScanConcurrency(int(concurrency))
}
}
sctx.GetSessionVars().SetIndexSerialScanConcurrency(1)
terror.Log(sctx.GetSessionVars().SetSystemVar(variable.TxnIsolation, ast.ReadCommitted))
defer func() {
Expand Down
1 change: 1 addition & 0 deletions executor/autoidtest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ go_test(
"//sessionctx/variable",
"//testkit",
"//testkit/testutil",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//tikv",
"@org_uber_go_goleak//:goleak",
Expand Down
13 changes: 13 additions & 0 deletions executor/autoidtest/autoid_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"strings"
"testing"

"github.com/pingcap/failpoint"
ddltestutil "github.com/pingcap/tidb/ddl/testutil"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/session"
Expand Down Expand Up @@ -736,3 +737,15 @@ func TestAlterTableAutoIDCache(t *testing.T) {
_, err = tk.Exec("alter table t_473 auto_id_cache = 1")
require.Error(t, err)
}

func TestMockAutoIDServiceError(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("USE test;")
tk.MustExec("create table t_mock_err (id int key auto_increment) auto_id_cache 1")

failpoint.Enable("github.com/pingcap/tidb/autoid_service/mockErr", `return(true)`)
defer failpoint.Disable("github.com/pingcap/tidb/autoid_service/mockErr")
// Cover a bug that the autoid client retry non-retryable errors forever cause dead loop.
tk.MustExecToErr("insert into t_mock_err values (),()") // mock error, instead of dead loop
}
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ require (
github.com/pingcap/errors v0.11.5-0.20220729040631-518f63d66278
github.com/pingcap/failpoint v0.0.0-20220423142525-ae43b7f4e5c3
github.com/pingcap/fn v0.0.0-20200306044125-d5540d389059
github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a
github.com/pingcap/kvproto v0.0.0-20221130022225-6c56ac56fe5f
github.com/pingcap/log v1.1.1-0.20221116035753-734d527bc87c
github.com/pingcap/sysutil v0.0.0-20220114020952-ea68d2dbf5b4
github.com/pingcap/tidb/parser v0.0.0-20211011031125-9b13dc409c5e
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -779,8 +779,8 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN
github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw=
github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w=
github.com/pingcap/kvproto v0.0.0-20221026112947-f8d61344b172/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI=
github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a h1:LzIZsQpXQlj8yF7+yvyOg680OaPq7bmPuDuszgXfHsw=
github.com/pingcap/kvproto v0.0.0-20221129023506-621ec37aac7a/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI=
github.com/pingcap/kvproto v0.0.0-20221130022225-6c56ac56fe5f h1:46ZD6xzQWJ8Jkeal/U7SqkX030Mgs8DAn6QV/9zbqOQ=
github.com/pingcap/kvproto v0.0.0-20221130022225-6c56ac56fe5f/go.mod h1:OYtxs0786qojVTmkVeufx93xe+jUgm56GUYRIKnmaGI=
github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8=
github.com/pingcap/log v0.0.0-20200511115504-543df19646ad/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8=
github.com/pingcap/log v0.0.0-20210625125904-98ed8e2eb1c7/go.mod h1:8AanEdAHATuRurdGxZXBz0At+9avep+ub7U1AGYLIMM=
Expand Down
3 changes: 2 additions & 1 deletion meta/autoid/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//autoid_service",
"//config",
"//errno",
"//kv",
"//meta",
Expand All @@ -31,7 +32,7 @@ go_library(
"@com_github_tikv_client_go_v2//util",
"@io_etcd_go_etcd_client_v3//:client",
"@org_golang_google_grpc//:grpc",
"@org_golang_google_grpc//credentials/insecure",
"@org_golang_google_grpc//credentials",
"@org_uber_go_zap//:zap",
],
)
Expand Down
Loading

0 comments on commit fd22612

Please sign in to comment.