Skip to content

Commit

Permalink
Merge branch 'master' into fix_39344_2
Browse files Browse the repository at this point in the history
  • Loading branch information
Yisaer authored Nov 29, 2022
2 parents 2cc606b + 2c0c130 commit 9f5d8cd
Show file tree
Hide file tree
Showing 227 changed files with 5,074 additions and 2,059 deletions.
62 changes: 0 additions & 62 deletions .github/workflows/integration-test-br-compatibility.yml

This file was deleted.

8 changes: 4 additions & 4 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -2936,8 +2936,8 @@ def go_deps():
name = "com_github_pingcap_tipb",
build_file_proto_mode = "disable_global",
importpath = "github.com/pingcap/tipb",
sum = "h1:Yoo8j5xQGxjlsC3yt0ndsiAz0WZXED9rzsKmEN0U0DY=",
version = "v0.0.0-20221020071514-cd933387bcb5",
sum = "h1:DbmCfCbcavo0JG+gSp0ySvv1ub/c/j3hsnYzyYPzONo=",
version = "v0.0.0-20221123081521-2fb828910813",
)
go_repository(
name = "com_github_pkg_browser",
Expand Down Expand Up @@ -3519,8 +3519,8 @@ def go_deps():
name = "com_github_tikv_client_go_v2",
build_file_proto_mode = "disable_global",
importpath = "github.com/tikv/client-go/v2",
sum = "h1:5FFJAKukKDTsLqrEeeDgC89aDAteGEFXBHwKRa3wnnQ=",
version = "v2.0.3-0.20221125022819-f05c6886bbad",
sum = "h1:nfdLYzkmYR2b0XurXSXcVFk06eO//P4VtkqoADX2tR4=",
version = "v2.0.3-0.20221128025602-81939ec8b2bb",
)
go_repository(
name = "com_github_tikv_pd_client",
Expand Down
14 changes: 7 additions & 7 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -415,7 +415,7 @@ bazel_coverage_test: failpoint-enable bazel_ci_prepare

bazel_build: bazel_ci_prepare
mkdir -p bin
bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) \
bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) --remote_download_minimal \
//... --//build:with_nogo_flag=true
bazel $(BAZEL_GLOBAL_CONFIG) build $(BAZEL_CMD_CONFIG) \
//cmd/importer:importer //tidb-server:tidb-server //tidb-server:tidb-server-check --//build:with_nogo_flag=true
Expand All @@ -442,27 +442,27 @@ bazel_golangcilinter:
-- run $$($(PACKAGE_DIRECTORIES)) --config ./.golangci.yaml

bazel_brietest: failpoint-enable bazel_ci_prepare
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \
-- //tests/realtikvtest/brietest/...

bazel_pessimistictest: failpoint-enable bazel_ci_prepare
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \
-- //tests/realtikvtest/pessimistictest/...

bazel_sessiontest: failpoint-enable bazel_ci_prepare
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \
-- //tests/realtikvtest/sessiontest/...

bazel_statisticstest: failpoint-enable bazel_ci_prepare
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \
-- //tests/realtikvtest/statisticstest/...

bazel_txntest: failpoint-enable bazel_ci_prepare
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \
-- //tests/realtikvtest/txntest/...

bazel_addindextest: failpoint-enable bazel_ci_prepare
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --test_arg=-with-real-tikv \
bazel $(BAZEL_GLOBAL_CONFIG) test $(BAZEL_CMD_CONFIG) --remote_download_minimal --test_arg=-with-real-tikv \
-- //tests/realtikvtest/addindextest/...

bazel_lint: bazel_prepare
Expand Down
14 changes: 12 additions & 2 deletions WORKSPACE
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ http_archive(
],
)

load("@io_bazel_rules_go//go:deps.bzl", "go_register_toolchains", "go_rules_dependencies")
load("@io_bazel_rules_go//go:deps.bzl", "go_download_sdk", "go_register_toolchains", "go_rules_dependencies")
load("@bazel_gazelle//:deps.bzl", "gazelle_dependencies")
load("//:DEPS.bzl", "go_deps")

Expand All @@ -27,9 +27,19 @@ go_deps()

go_rules_dependencies()

go_download_sdk(
name = "go_sdk",
urls = [
"http://ats.apps.svc/golang/{}",
"http://bazel-cache.pingcap.net:8080/golang/{}",
"https://mirrors.aliyun.com/golang/{}",
"https://dl.google.com/go/{}",
],
version = "1.19.3",
)

go_register_toolchains(
nogo = "@//build:tidb_nogo",
version = "1.19.3",
)

gazelle_dependencies()
Expand Down
1 change: 1 addition & 0 deletions autoid_service/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"//meta",
"//metrics",
"//owner",
"//parser/model",
"//util/logutil",
"//util/mathutil",
"@com_github_pingcap_errors//:errors",
Expand Down
11 changes: 6 additions & 5 deletions autoid_service/autoid.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/owner"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/mathutil"
clientv3 "go.etcd.io/etcd/client/v3"
Expand Down Expand Up @@ -76,7 +77,7 @@ func (alloc *autoIDValue) alloc4Unsigned(ctx context.Context, store kv.Storage,

ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID()
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5)
var err1 error
newBase, err1 = idAcc.Get()
if err1 != nil {
Expand Down Expand Up @@ -137,7 +138,7 @@ func (alloc *autoIDValue) alloc4Signed(ctx context.Context,

ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID()
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5)
var err1 error
newBase, err1 = idAcc.Get()
if err1 != nil {
Expand Down Expand Up @@ -188,7 +189,7 @@ func (alloc *autoIDValue) rebase4Unsigned(ctx context.Context,
startTime := time.Now()
ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID()
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5)
currentEnd, err1 := idAcc.Get()
if err1 != nil {
return err1
Expand Down Expand Up @@ -221,7 +222,7 @@ func (alloc *autoIDValue) rebase4Signed(ctx context.Context, store kv.Storage, d
var newBase, newEnd int64
ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID()
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5)
currentEnd, err1 := idAcc.Get()
if err1 != nil {
return err1
Expand Down Expand Up @@ -451,7 +452,7 @@ func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (*
func (alloc *autoIDValue) forceRebase(ctx context.Context, store kv.Storage, dbID, tblID, requiredBase int64, isUnsigned bool) error {
ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta)
err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error {
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID()
idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5)
currentEnd, err1 := idAcc.Get()
if err1 != nil {
return err1
Expand Down
7 changes: 3 additions & 4 deletions bindinfo/session_handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,13 +33,12 @@ import (

// SessionHandle is used to handle all session sql bind operations.
type SessionHandle struct {
ch *bindCache
parser *parser.Parser
ch *bindCache
}

// NewSessionBindHandle creates a new SessionBindHandle.
func NewSessionBindHandle(parser *parser.Parser) *SessionHandle {
sessionHandle := &SessionHandle{parser: parser}
func NewSessionBindHandle() *SessionHandle {
sessionHandle := &SessionHandle{}
sessionHandle.ch = newBindCache()
return sessionHandle
}
Expand Down
1 change: 0 additions & 1 deletion br/cmd/br/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,5 @@ func newStreamRestoreCommand() *cobra.Command {
}
task.DefineFilterFlags(command, filterOutSysAndMemTables, true)
task.DefineStreamRestoreFlags(command)
command.Hidden = true
return command
}
6 changes: 4 additions & 2 deletions br/pkg/backup/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -290,10 +290,12 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) {
ranges = ranger.FullIntRange(false)
}

retRanges := make([]kv.KeyRange, 0, 1+len(tbl.Indices))
kvRanges, err := distsql.TableHandleRangesToKVRanges(nil, []int64{tblID}, tbl.IsCommonHandle, ranges, nil)
if err != nil {
return nil, errors.Trace(err)
}
retRanges = kvRanges.AppendSelfTo(retRanges)

for _, index := range tbl.Indices {
if index.State != model.StatePublic {
Expand All @@ -304,9 +306,9 @@ func appendRanges(tbl *model.TableInfo, tblID int64) ([]kv.KeyRange, error) {
if err != nil {
return nil, errors.Trace(err)
}
kvRanges = append(kvRanges, idxRanges...)
retRanges = idxRanges.AppendSelfTo(retRanges)
}
return kvRanges, nil
return retRanges, nil
}

// BuildBackupRangeAndSchema gets KV range and schema of tables.
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/checksum/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ func TestChecksum(t *testing.T) {
first = false
ranges, err := backup.BuildTableRanges(tableInfo3)
require.NoError(t, err)
require.Equalf(t, ranges[:1], req.KeyRanges, "%v", req.KeyRanges)
require.Equalf(t, ranges[:1], req.KeyRanges.FirstPartitionRange(), "%v", req.KeyRanges.FirstPartitionRange())
}
return nil
}))
Expand Down
5 changes: 3 additions & 2 deletions br/pkg/errors/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,8 +83,9 @@ var (
ErrStorageInvalidPermission = errors.Normalize("external storage permission", errors.RFCCodeText("BR:ExternalStorage:ErrStorageInvalidPermission"))

// Snapshot restore
ErrRestoreTotalKVMismatch = errors.Normalize("restore total tikvs mismatch", errors.RFCCodeText("BR:EBS:ErrRestoreTotalKVMismatch"))
ErrRestoreInvalidPeer = errors.Normalize("restore met a invalid peer", errors.RFCCodeText("BR:EBS:ErrRestoreInvalidPeer"))
ErrRestoreTotalKVMismatch = errors.Normalize("restore total tikvs mismatch", errors.RFCCodeText("BR:EBS:ErrRestoreTotalKVMismatch"))
ErrRestoreInvalidPeer = errors.Normalize("restore met a invalid peer", errors.RFCCodeText("BR:EBS:ErrRestoreInvalidPeer"))
ErrRestoreRegionWithoutPeer = errors.Normalize("restore met a region without any peer", errors.RFCCodeText("BR:EBS:ErrRestoreRegionWithoutPeer"))

// Errors reported from TiKV.
ErrKVStorage = errors.Normalize("tikv storage occur I/O error", errors.RFCCodeText("BR:KV:ErrKVStorage"))
Expand Down
1 change: 1 addition & 0 deletions br/pkg/lightning/backend/kv/allocator.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ type panickingAllocator struct {
func NewPanickingAllocators(base int64) autoid.Allocators {
sharedBase := &base
return autoid.NewAllocators(
false,
&panickingAllocator{base: sharedBase, ty: autoid.RowIDAllocType},
&panickingAllocator{base: sharedBase, ty: autoid.AutoIncrementType},
&panickingAllocator{base: sharedBase, ty: autoid.AutoRandomType},
Expand Down
2 changes: 1 addition & 1 deletion br/pkg/lightning/backend/local/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ go_test(
],
embed = [":local"],
flaky = True,
shard_count = 20,
shard_count = 40,
deps = [
"//br/pkg/errors",
"//br/pkg/lightning/backend",
Expand Down
Loading

0 comments on commit 9f5d8cd

Please sign in to comment.