Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into stale-r…
Browse files Browse the repository at this point in the history
…ead-ddl-fix
  • Loading branch information
crazycs520 committed Sep 8, 2023
2 parents ab82a5a + fa0c5ce commit c8f5b20
Show file tree
Hide file tree
Showing 210 changed files with 19,627 additions and 14,701 deletions.
3 changes: 2 additions & 1 deletion .github/licenserc.yml
Original file line number Diff line number Diff line change
Expand Up @@ -48,5 +48,6 @@ header:
- "build/image/.ci_bazel"
- "**/OWNERS"
- "OWNERS_ALIASES"
- "disttask/framework/mock/*_mock.go"
- "disttask/framework/mock/**/*_mock.go"
- "util/sqlexec/mock/*_mock.go"
comment: on-failure
12 changes: 6 additions & 6 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -6937,13 +6937,13 @@ def go_deps():
name = "com_github_tikv_client_go_v2",
build_file_proto_mode = "disable_global",
importpath = "github.com/tikv/client-go/v2",
sha256 = "9cf5877cb0b43d73140e280ad9c80dccd9684e89659a358ee75702469368cf95",
strip_prefix = "github.com/tikv/client-go/v2@v2.0.8-0.20230829002846-295094e5b534",
sha256 = "d6ce005b65261e495ee771f5cfdbc75e409c739ff0b89f4edcd187c3c0cb000f",
strip_prefix = "github.com/tikv/client-go/v2@v2.0.8-0.20230905034839-5dd12b06bc3c",
urls = [
"http://bazel-cache.pingcap.net:8080/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20230829002846-295094e5b534.zip",
"http://ats.apps.svc/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20230829002846-295094e5b534.zip",
"https://cache.hawkingrei.com/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20230829002846-295094e5b534.zip",
"https://storage.googleapis.com/pingcapmirror/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20230829002846-295094e5b534.zip",
"http://bazel-cache.pingcap.net:8080/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20230905034839-5dd12b06bc3c.zip",
"http://ats.apps.svc/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20230905034839-5dd12b06bc3c.zip",
"https://cache.hawkingrei.com/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20230905034839-5dd12b06bc3c.zip",
"https://storage.googleapis.com/pingcapmirror/gomod/github.com/tikv/client-go/v2/com_github_tikv_client_go_v2-v2.0.8-0.20230905034839-5dd12b06bc3c.zip",
],
)
go_repository(
Expand Down
6 changes: 4 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -384,9 +384,11 @@ mock_lightning: tools/bin/mockgen
tools/bin/mockgen -package mock github.com/pingcap/tidb/br/pkg/utils TaskRegister > br/pkg/mock/task_register.go

gen_mock: tools/bin/mockgen
tools/bin/mockgen -package mock github.com/pingcap/tidb/disttask/framework/scheduler TaskTable,SubtaskExecutor,Pool,Scheduler,InternalScheduler > disttask/framework/mock/scheduler_mock.go
tools/bin/mockgen -package mock github.com/pingcap/tidb/disttask/framework/scheduler TaskTable,Pool,Scheduler,Extension > disttask/framework/mock/scheduler_mock.go
tools/bin/mockgen -package execute github.com/pingcap/tidb/disttask/framework/scheduler/execute SubtaskExecutor,MiniTaskExecutor > disttask/framework/mock/execute/execute_mock.go
tools/bin/mockgen -package mock github.com/pingcap/tidb/disttask/framework/planner LogicalPlan,PipelineSpec > disttask/framework/mock/plan_mock.go

tools/bin/mockgen -package mock github.com/pingcap/tidb/util/sqlexec RestrictedSQLExecutor > util/sqlexec/mock/restricted_sql_executor_mock.go

# There is no FreeBSD environment for GitHub actions. So cross-compile on Linux
# but that doesn't work with CGO_ENABLED=1, so disable cgo. The reason to have
# cgo enabled on regular builds is performance.
Expand Down
40 changes: 32 additions & 8 deletions WORKSPACE
Original file line number Diff line number Diff line change
Expand Up @@ -4,10 +4,10 @@ http_archive(
name = "bazel_skylib",
sha256 = "66ffd9315665bfaafc96b52278f57c7e2dd09f5ede279ea6d39b2be471e7e3aa",
urls = [
"http://bazel-cache.pingcap.net:8080/bazelbuild/bazel-skylib/releases/download/1.4.2/bazel-skylib-1.4.2.tar.gz",
"http://bazel-cache.pingcap.net:8080/gomod/rules/bazel-skylib-1.4.2.tar.gz",
"http://ats.apps.svc/gomod/rules/bazel-skylib-1.4.2.tar.gz",
"https://mirror.bazel.build/github.com/bazelbuild/bazel-skylib/releases/download/1.4.2/bazel-skylib-1.4.2.tar.gz",
"https://github.com/bazelbuild/bazel-skylib/releases/download/1.4.2/bazel-skylib-1.4.2.tar.gz",
"http://ats.apps.svc/bazelbuild/bazel-skylib/releases/download/1.4.2/bazel-skylib-1.4.2.tar.gz",
],
)

Expand Down Expand Up @@ -79,9 +79,9 @@ http_archive(
strip_prefix = "protobuf-3.15.5",
# latest, as of 2021-03-08
urls = [
"http://bazel-cache.pingcap.net:8080/protocolbuffers/protobuf/archive/v3.15.5.tar.gz",
"https://github.com/protocolbuffers/protobuf/archive/v3.15.5.tar.gz",
"http://bazel-cache.pingcap.net:8080/gomod/rules/protobuf-3.15.5.tar.gz ",
"https://mirror.bazel.build/github.com/protocolbuffers/protobuf/archive/v3.15.5.tar.gz",
"https://github.com/protocolbuffers/protobuf/archive/v3.15.5.tar.gz",
],
)

Expand All @@ -93,8 +93,8 @@ http_archive(
name = "remote_java_tools",
sha256 = "f58a358ca694a41416a9b6a92b852935ad301d8882e5d22f4f11134f035317d5",
urls = [
"http://bazel-cache.pingcap.net:8080/bazelbuild/java_tools/releases/download/java_v12.6/java_tools-v12.6.zip",
"http://ats.apps.svc/bazel_java_tools/releases/java/v12.6/java_tools-v12.6.zip",
"http://bazel-cache.pingcap.net:8080/gomod/rules/java_tools-v12.6.zip",
"http://ats.apps.svc/gomod/rules/java_tools-v12.6.zip",
"https://mirror.bazel.build/bazel_java_tools/releases/java/v12.6/java_tools-v12.6.zip",
"https://github.com/bazelbuild/java_tools/releases/download/java_v12.6/java_tools-v12.6.zip",
],
Expand All @@ -104,9 +104,33 @@ http_archive(
name = "remote_java_tools_linux",
sha256 = "64294e91fe940c77e6d35818b4c3a1f07d78e33add01e330188d907032687066",
urls = [
"http://bazel-cache.pingcap.net:8080/bazelbuild/java_tools/releases/download/java_v12.6/java_tools_linux-v12.6.zip",
"http://ats.apps.svc/bazel_java_tools/releases/java/v12.6/java_tools_linux-v12.6.zip",
"http://bazel-cache.pingcap.net:8080/gomod/rules/java_tools_linux-v12.6.zip",
"http://ats.apps.svc/gomod/rules/java_tools_linux-v12.6.zip",
"https://mirror.bazel.build/bazel_java_tools/releases/java/v12.6/java_tools_linux-v12.6.zip",
"https://github.com/bazelbuild/java_tools/releases/download/java_v12.6/java_tools_linux-v12.6.zip",
],
)

http_archive(
name = "rules_proto",
sha256 = "aa1ee19226f707d44bee44c720915199c20c84a23318bb0597ed4e5c873ccbd5",
strip_prefix = "rules_proto-40298556293ae502c66579620a7ce867d5f57311",
urls = [
"http://bazel-cache.pingcap.net:8080/gomod/rules/rules_proto/rules_proto-40298556293ae502c66579620a7ce867d5f57311.tar.gz",
"http://ats.apps.svc/gomod/gomod/rules/rules_proto/rules_proto-40298556293ae502c66579620a7ce867d5f57311.tar.gz",
"https://mirror.bazel.build/github.com/bazelbuild/rules_proto/archive/40298556293ae502c66579620a7ce867d5f57311.tar.gz",
"https://github.com/bazelbuild/rules_proto/archive/40298556293ae502c66579620a7ce867d5f57311.tar.gz",
],
)

http_archive(
name = "rules_java",
sha256 = "f5a3e477e579231fca27bf202bb0e8fbe4fc6339d63b38ccb87c2760b533d1c3",
strip_prefix = "rules_java-981f06c3d2bd10225e85209904090eb7b5fb26bd",
urls = [
"http://bazel-cache.pingcap.net:8080/gomod/rules/rules_java/rules_java-981f06c3d2bd10225e85209904090eb7b5fb26bd.tar.gz",
"http://ats.apps.svc/bazelbuild/gomod/rules/rules_java/rules_java-981f06c3d2bd10225e85209904090eb7b5fb26bd.tar.gz",
"https://mirror.bazel.build/github.com/bazelbuild/rules_java/archive/981f06c3d2bd10225e85209904090eb7b5fb26bd.tar.gz",
"https://github.com/bazelbuild/rules_java/archive/981f06c3d2bd10225e85209904090eb7b5fb26bd.tar.gz",
],
)
2 changes: 1 addition & 1 deletion bindinfo/internal/testutil.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,6 @@ func UtilNormalizeWithDefaultDB(t *testing.T, sql string) (normalized, digest st
testParser := parser.New()
stmt, err := testParser.ParseOneStmt(sql, "", "")
require.NoError(t, err)
normalized, digestResult := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(stmt, "test", ""))
normalized, digestResult := parser.NormalizeDigestForBinding(utilparser.RestoreWithDefaultDB(stmt, "test", ""))
return normalized, digestResult.String()
}
2 changes: 1 addition & 1 deletion bindinfo/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,7 @@ go_test(
],
flaky = True,
race = "on",
shard_count = 34,
shard_count = 35,
deps = [
"//bindinfo",
"//bindinfo/internal",
Expand Down
50 changes: 47 additions & 3 deletions bindinfo/tests/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -414,7 +414,7 @@ func TestBindingSymbolList(t *testing.T) {
require.True(t, tk.MustUseIndex("select a, b from t where a = 3 limit 1, 100", "ib(b)"))

// Normalize
sql, hash := parser.NormalizeDigest("select a, b from test . t where a = 1 limit 0, 1")
sql, hash := parser.NormalizeDigestForBinding("select a, b from test . t where a = 1 limit 0, 1")

bindData := dom.BindHandle().GetBindRecord(hash.String(), sql, "test")
require.NotNil(t, bindData)
Expand All @@ -429,6 +429,50 @@ func TestBindingSymbolList(t *testing.T) {
require.NotNil(t, bind.UpdateTime)
}

// TestBindingInListWithSingleLiteral tests sql with "IN (Lit)", fixes #44298
func TestBindingInListWithSingleLiteral(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int, b int, INDEX ia (a), INDEX ib (b));")
tk.MustExec("insert into t value(1, 1);")

// GIVEN
sqlcmd := "select a, b from t where a in (1)"
binding := `create global binding for select a, b from t where a in (1, 2, 3) using select a, b from t use index (ib) where a in (1, 2, 3)`

// before binding
tk.MustQuery(sqlcmd)
require.Equal(t, "t:ia", tk.Session().GetSessionVars().StmtCtx.IndexNames[0])
require.True(t, tk.MustUseIndex(sqlcmd, "ia(a)"))

tk.MustExec(binding)

// after binding
tk.MustQuery(sqlcmd)
require.Equal(t, "t:ib", tk.Session().GetSessionVars().StmtCtx.IndexNames[0])
require.True(t, tk.MustUseIndex(sqlcmd, "ib(b)"))

tk.MustQuery("select @@last_plan_from_binding").Check(testkit.Rows("1"))

// Normalize
sql, hash := parser.NormalizeDigestForBinding("select a, b from test . t where a in (1)")

bindData := dom.BindHandle().GetBindRecord(hash.String(), sql, "test")
require.NotNil(t, bindData)
require.Equal(t, "select `a` , `b` from `test` . `t` where `a` in ( ... )", bindData.OriginalSQL)
bind := bindData.Bindings[0]
require.Equal(t, "SELECT `a`,`b` FROM `test`.`t` USE INDEX (`ib`) WHERE `a` IN (1,2,3)", bind.BindSQL)
require.Equal(t, "test", bindData.Db)
require.Equal(t, bindinfo.Enabled, bind.Status)
require.NotNil(t, bind.Charset)
require.NotNil(t, bind.Collation)
require.NotNil(t, bind.CreateTime)
require.NotNil(t, bind.UpdateTime)
}

func TestDMLSQLBind(t *testing.T) {
store := testkit.CreateMockStore(t)

Expand Down Expand Up @@ -538,7 +582,7 @@ func TestErrorBind(t *testing.T) {
_, err := tk.Exec("create global binding for select * from t where i>100 using select * from t use index(index_t) where i>100")
require.NoError(t, err, "err %v", err)

sql, hash := parser.NormalizeDigest("select * from test . t where i > ?")
sql, hash := parser.NormalizeDigestForBinding("select * from test . t where i > ?")
bindData := dom.BindHandle().GetBindRecord(hash.String(), sql, "test")
require.NotNil(t, bindData)
require.Equal(t, "select * from `test` . `t` where `i` > ?", bindData.OriginalSQL)
Expand Down Expand Up @@ -1304,7 +1348,7 @@ func TestBindSQLDigest(t *testing.T) {
parser4binding := parser.New()
originNode, err := parser4binding.ParseOneStmt(c.origin, "utf8mb4", "utf8mb4_general_ci")
require.NoError(t, err)
_, sqlDigestWithDB := parser.NormalizeDigest(utilparser.RestoreWithDefaultDB(originNode, "test", c.origin))
_, sqlDigestWithDB := parser.NormalizeDigestForBinding(utilparser.RestoreWithDefaultDB(originNode, "test", c.origin))
require.Equal(t, res[0][9], sqlDigestWithDB.String())
}
}
Expand Down
16 changes: 16 additions & 0 deletions br/pkg/lightning/backend/backend.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,8 @@ type EngineConfig struct {
TableInfo *checkpoints.TidbTableInfo
// local backend specified configuration
Local LocalEngineConfig
// local backend external engine specified configuration
External *ExternalEngineConfig
// KeepSortDir indicates whether to keep the temporary sort directory
// when opening the engine, instead of removing it.
KeepSortDir bool
Expand All @@ -99,6 +101,20 @@ type LocalEngineConfig struct {
CompactConcurrency int
}

// ExternalEngineConfig is the configuration used for local backend external engine.
type ExternalEngineConfig struct {
StorageURI string
DataFiles []string
StatFiles []string
MinKey []byte
MaxKey []byte
SplitKeys [][]byte
// TotalFileSize can be an estimated value.
TotalFileSize int64
// TotalKVCount can be an estimated value.
TotalKVCount int64
}

// CheckCtx contains all parameters used in CheckRequirements
type CheckCtx struct {
DBMetas []*mydump.MDDatabaseMeta
Expand Down
31 changes: 22 additions & 9 deletions br/pkg/lightning/backend/external/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,8 @@ type Engine struct {
storage storage.ExternalStorage
dataFiles []string
statsFiles []string
minKey []byte
maxKey []byte
splitKeys [][]byte
bufPool *membuf.Pool

Expand All @@ -49,8 +51,8 @@ type Engine struct {
dupDetectOpt common.DupDetectOpt
ts uint64

totalKVSize int64
totalKVLength int64
totalKVSize int64
totalKVCount int64

importedKVSize *atomic.Int64
importedKVCount *atomic.Int64
Expand All @@ -61,26 +63,32 @@ func NewExternalEngine(
storage storage.ExternalStorage,
dataFiles []string,
statsFiles []string,
minKey []byte,
maxKey []byte,
splitKeys [][]byte,
keyAdapter common.KeyAdapter,
duplicateDetection bool,
duplicateDB *pebble.DB,
dupDetectOpt common.DupDetectOpt,
ts uint64,
totalKVSize int64,
totakKVLength int64,
totalKVCount int64,
) common.Engine {
return &Engine{
storage: storage,
dataFiles: dataFiles,
statsFiles: statsFiles,
minKey: minKey,
maxKey: maxKey,
splitKeys: splitKeys,
bufPool: membuf.NewPool(),
keyAdapter: keyAdapter,
duplicateDetection: duplicateDetection,
duplicateDB: duplicateDB,
dupDetectOpt: dupDetectOpt,
ts: ts,
totalKVSize: totalKVSize,
totalKVLength: totakKVLength,
totalKVCount: totalKVCount,
importedKVSize: atomic.NewInt64(0),
importedKVCount: atomic.NewInt64(0),
}
Expand Down Expand Up @@ -177,13 +185,13 @@ func (e *Engine) createMergeIter(ctx context.Context, start kv.Key) (*MergeKVIte
return iter, nil
}

// KVStatistics returns the total kv size and total kv length.
func (e *Engine) KVStatistics() (totalKVSize int64, totalKVLength int64) {
return e.totalKVSize, e.totalKVLength
// KVStatistics returns the total kv size and total kv count.
func (e *Engine) KVStatistics() (totalKVSize int64, totalKVCount int64) {
return e.totalKVSize, e.totalKVCount
}

// ImportedStatistics returns the imported kv size and imported kv length.
func (e *Engine) ImportedStatistics() (importedKVSize int64, importedKVLength int64) {
// ImportedStatistics returns the imported kv size and imported kv count.
func (e *Engine) ImportedStatistics() (importedSize int64, importedKVCount int64) {
return e.importedKVSize.Load(), e.importedKVCount.Load()
}

Expand All @@ -192,6 +200,11 @@ func (e *Engine) ID() string {
return "external"
}

// GetKeyRange implements common.Engine.
func (e *Engine) GetKeyRange() (firstKey []byte, lastKey []byte, err error) {
return e.minKey, e.maxKey, nil
}

// SplitRanges split the ranges by split keys provided by external engine.
func (e *Engine) SplitRanges(
startKey, endKey []byte,
Expand Down
28 changes: 14 additions & 14 deletions br/pkg/lightning/backend/external/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,20 +69,20 @@ notExhausted:
lenStatFiles := len(statFiles)
require.Equal(t, lenDataFiles, lenStatFiles)
require.Greater(t, lenDataFiles, 0)
require.Greater(t, len(splitKeys), 0)

// splitKeys should be strictly increasing
for i := 1; i < len(splitKeys); i++ {
cmp := bytes.Compare(splitKeys[i], splitKeys[i-1])
require.Equal(t, 1, cmp, "splitKeys: %v", splitKeys)
}
// first splitKeys should be strictly greater than lastEndKey
cmp := bytes.Compare(splitKeys[0], lastEndKey)
require.Equal(t, 1, cmp, "splitKeys: %v, lastEndKey: %v", splitKeys, lastEndKey)
// last splitKeys should be strictly less than endKey
if endKey != nil {
cmp = bytes.Compare(splitKeys[len(splitKeys)-1], endKey)
require.Equal(t, -1, cmp, "splitKeys: %v, endKey: %v", splitKeys, endKey)
if len(splitKeys) > 0 {
// splitKeys should be strictly increasing
for i := 1; i < len(splitKeys); i++ {
cmp := bytes.Compare(splitKeys[i], splitKeys[i-1])
require.Equal(t, 1, cmp, "splitKeys: %v", splitKeys)
}
// first splitKeys should be strictly greater than lastEndKey
cmp := bytes.Compare(splitKeys[0], lastEndKey)
require.Equal(t, 1, cmp, "splitKeys: %v, lastEndKey: %v", splitKeys, lastEndKey)
// last splitKeys should be strictly less than endKey
if endKey != nil {
cmp = bytes.Compare(splitKeys[len(splitKeys)-1], endKey)
require.Equal(t, -1, cmp, "splitKeys: %v, endKey: %v", splitKeys, endKey)
}
}

lastEndKey = endKey
Expand Down
2 changes: 2 additions & 0 deletions br/pkg/lightning/backend/local/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ go_library(
"//br/pkg/errors",
"//br/pkg/lightning/backend",
"//br/pkg/lightning/backend/encode",
"//br/pkg/lightning/backend/external",
"//br/pkg/lightning/backend/kv",
"//br/pkg/lightning/checkpoints",
"//br/pkg/lightning/common",
Expand All @@ -40,6 +41,7 @@ go_library(
"//br/pkg/membuf",
"//br/pkg/pdutil",
"//br/pkg/restore/split",
"//br/pkg/storage",
"//br/pkg/utils",
"//br/pkg/version",
"//distsql",
Expand Down
Loading

0 comments on commit c8f5b20

Please sign in to comment.