Skip to content

Commit

Permalink
br: Merge master to br-stream (#30598)
Browse files Browse the repository at this point in the history
* ddl: handle the error from `addBatchDDLJobs()` correctly (#30401)

* br: fix the integration tests (#30423)

* util, cmd: remove unused filesort (#30438)

* *: update client-go for small backoff time (#30436)

* server: Fix unstable tests with FakeAuthSwitch (#30287)

* dumpling: fix dump failed when sequence exists (#30164)

* *: replace compareDatum by compare (#30421)

* lightning: fix gcs max key limit (#30393)

* expression, parser: add built-in func is_uuid (#30318)

* expression: migrate test-infra to testify for constant_fold_test.go (#30424)

* executor: fix pipelined window invalid memory address (#30418)

* makefile: add gotestsum for verify ci (#29848)

* server: close sql rows to fix unstable test (#30306)

* Makefile: add coverage record for BR and Dumpling (#30457)

* executor: track the mem usage of IndexMergeReader (#30210)

* infosync: close body when ReadAll encounters error (#30462)

* planner: show accessed partition when explain mpp query over partition table (#30367)

* *: Fix use of user identity in SHOW GRANTS + error messages (#30294)

* ddl: add not null flag for auto_increment column  (#30477)

* expression: make some unstable test serial (#30323)

* expression: migrate test-infra to testify for constant_propagation_test.go (#30430)

* executor: stable test TestSetDDLReorgBatchSize and TestSetDDLReorgWorkerCnt (#30480)

* statistics, util/ranger: add cardinality estimation trace for `GetRowCountBy...` (#30321)

* *: skip mysql client goroutine leak detection in integration ddl (#30467)

* executor,util: write slow query to slow log no matter what log level (#30461)

* executor: enable index_merge used in transaction. (#29875)

* logutil: add testcase for SlowQueryLogger.MaxDays/MaxSize/MaxBackups (#30316)

* expression: fix data race in builtin_other_vec_generated_test.go (#30503)

* expression: fix data race in the collationInfo (#30490)

* planner/core, session: fix error message of wrong variable scope (#30510)

* lightning: support Re/ReregisterMySQL by different tls name (#30463)

* executor: TestBatchGetandPointGetwithHashPartition test typo (#29669) (#29671)

* mockstore: improve log to avoid panic for nil pointer (#30513)

* *: replace compareDatum by compare, PR 10 (#30456)

* planner: Disable dynamic partition prune mode for all non-autocommit (#27532) (#30505)

* expression: change the log level of an confusing log from warn to debug (#30484)

* br: Check crypter.key valid before backup (#29991)

* *: replace compareDatum by compare, PR 11 (#30465)

* dumpling: fix default column collation with upstream when dump table (#30531)

* server: fix prepared cursor select (#30285)

* executor: HashJoinExec checks the buildError even if the probeSide is empty (#30471)

* parser, expression: follow mysql, increase interval precedence (#30528)

* makefile: set timeout 25m for make race (#30555)

* planner: fix the unstable test TestAnalyzeGlobalStatsWithOpts/2 (#30576)

* expression,types: Adjusts UNIX_TIMESTAMP() for non-existing DST values (#28739) (#30405)

* br: add res.Body.close to avoid leak (#30545)

* lightning: add back integration test lightning_error_summary (#30547)

* sessionctx/variable: small refactor (split large file) (#30511)

* ddl: let `admin cancel ddl jobs` run in a new transaction (#30549)

* *: Retry when placement PutBundles failed (#30590)

Co-authored-by: tangenta <[email protected]>
Co-authored-by: 3pointer <[email protected]>
Co-authored-by: wjHuang <[email protected]>
Co-authored-by: Lei Zhao <[email protected]>
Co-authored-by: Daniël van Eeden <[email protected]>
Co-authored-by: sylzd <[email protected]>
Co-authored-by: Jianjun Liao <[email protected]>
Co-authored-by: unconsolable <[email protected]>
Co-authored-by: tison <[email protected]>
Co-authored-by: Shenghui Wu <[email protected]>
Co-authored-by: Weizhen Wang <[email protected]>
Co-authored-by: guo-shaoge <[email protected]>
Co-authored-by: Ryan Leung <[email protected]>
Co-authored-by: xufei <[email protected]>
Co-authored-by: Morgan Tocker <[email protected]>
Co-authored-by: Zhou Kunqin <[email protected]>
Co-authored-by: 王超 <[email protected]>
Co-authored-by: TonsnakeLin <[email protected]>
Co-authored-by: Ehco <[email protected]>
Co-authored-by: Mattias Jonsson <[email protected]>
Co-authored-by: HuaiyuXu <[email protected]>
Co-authored-by: WizardXiao <[email protected]>
Co-authored-by: xhe <[email protected]>
Co-authored-by: Hangjie Mo <[email protected]>
Co-authored-by: Yuanjia Zhang <[email protected]>
Co-authored-by: glorv <[email protected]>
Co-authored-by: djshow832 <[email protected]>
Co-authored-by: Ti Chi Robot <[email protected]>
  • Loading branch information
1 parent bcdee32 commit f015eab
Show file tree
Hide file tree
Showing 133 changed files with 5,056 additions and 4,742 deletions.
2 changes: 2 additions & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -24,3 +24,5 @@ coverage.txt
var
fix.sql
export-20*/
*-coverage.xml
*-junit-report.xml
56 changes: 47 additions & 9 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -109,13 +109,6 @@ explaintest: server_check
ddltest:
@cd cmd/ddltest && $(GO) test -o ../../bin/ddltest -c

upload-coverage:
ifneq ($(CODECOV_TOKEN), "")
curl -LO ${FILE_SERVER_URL}/download/cicd/ci-tools/codecov
chmod +x codecov
./codecov -t ${CODECOV_TOKEN}
endif

devgotest: failpoint-enable
# grep regex: Filter out all tidb logs starting with:
# - '[20' (like [2021/09/15 ...] [INFO]..)
Expand All @@ -132,9 +125,29 @@ gotest: failpoint-enable
$(GOTEST) -ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -cover $(PACKAGES_TIDB_TESTS) -coverprofile=coverage.txt -check.p true > gotest.log || { $(FAILPOINT_DISABLE); cat 'gotest.log'; exit 1; }
@$(FAILPOINT_DISABLE)

gotest_in_verify_ci_part_1: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml
@echo "Running gotest_in_verify_ci_part_1."
@mkdir -p $(TEST_COVERAGE_DIR)
@export log_level=info; export TZ='Asia/Shanghai'; \
CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) --race \
-ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \
$(PACKAGES_TIDB_TESTS_EXPENSIVE) -check.p true || { $(FAILPOINT_DISABLE); exit 1; }
tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/tidb-coverage.xml"
@$(FAILPOINT_DISABLE)

gotest_in_verify_ci_part_2: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml
@echo "Running gotest_in_verify_ci_part_2."
@mkdir -p $(TEST_COVERAGE_DIR)
@export log_level=info; export TZ='Asia/Shanghai'; \
CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/tidb-junit-report.xml" -- -v -p $(P) --race \
-ldflags '$(TEST_LDFLAGS)' $(EXTRA_TEST_ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" \
$(PACKAGES_TIDB_TESTS_OTHERS) -check.p true || { $(FAILPOINT_DISABLE); exit 1; }
tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/tidb_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/tidb-coverage.xml"
@$(FAILPOINT_DISABLE)

race: failpoint-enable
@export log_level=debug; \
$(GOTEST) -timeout 20m -race $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; }
$(GOTEST) -timeout 25m -race $(PACKAGES) || { $(FAILPOINT_DISABLE); exit 1; }
@$(FAILPOINT_DISABLE)

leak: failpoint-enable
Expand Down Expand Up @@ -310,7 +323,16 @@ br_unit_test: export ARGS=$$($(BR_PACKAGES))
br_unit_test:
@make failpoint-enable
@export TZ='Asia/Shanghai';
$(GOTEST) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -tags leak $(ARGS) || ( make failpoint-disable && exit 1 )
$(GOTEST) $(RACE_FLAG) -ldflags '$(LDFLAGS)' -tags leak $(ARGS) -coverprofile=coverage.txt || ( make failpoint-disable && exit 1 )
@make failpoint-disable
br_unit_test_in_verify_ci: export ARGS=$$($(BR_PACKAGES))
br_unit_test_in_verify_ci: tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml
@make failpoint-enable
@export TZ='Asia/Shanghai';
@mkdir -p $(TEST_COVERAGE_DIR)
CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/br-junit-report.xml" -- $(RACE_FLAG) -ldflags '$(LDFLAGS)' \
-tags leak $(ARGS) -coverprofile="$(TEST_COVERAGE_DIR)/br_cov.unit_test.out" || ( make failpoint-disable && exit 1 )
tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/br_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/br-coverage.xml"
@make failpoint-disable

br_integration_test: br_bins build_br build_for_br_integration_test
Expand Down Expand Up @@ -369,6 +391,13 @@ dumpling_unit_test: export DUMPLING_ARGS=$$($(DUMPLING_PACKAGES))
dumpling_unit_test: failpoint-enable
$(DUMPLING_GOTEST) $(RACE_FLAG) -coverprofile=coverage.txt -covermode=atomic -tags leak $(DUMPLING_ARGS) || ( make failpoint-disable && exit 1 )
@make failpoint-disable
dumpling_unit_test_in_verify_ci: export DUMPLING_ARGS=$$($(DUMPLING_PACKAGES))
dumpling_unit_test_in_verify_ci: failpoint-enable tools/bin/gotestsum tools/bin/gocov tools/bin/gocov-xml
@mkdir -p $(TEST_COVERAGE_DIR)
CGO_ENABLED=1 tools/bin/gotestsum --junitfile "$(TEST_COVERAGE_DIR)/dumpling-junit-report.xml" -- -tags leak $(DUMPLING_ARGS) \
$(RACE_FLAG) -coverprofile="$(TEST_COVERAGE_DIR)/dumpling_cov.unit_test.out" || ( make failpoint-disable && exit 1 )
tools/bin/gocov convert "$(TEST_COVERAGE_DIR)/dumpling_cov.unit_test.out" | tools/bin/gocov-xml > "$(TEST_COVERAGE_DIR)/dumpling-coverage.xml"
@make failpoint-disable

dumpling_integration_test: dumpling_bins failpoint-enable build_dumpling
@make failpoint-disable
Expand All @@ -388,3 +417,12 @@ dumpling_bins:
@which bin/minio
@which bin/tidb-lightning
@which bin/sync_diff_inspector

tools/bin/gotestsum: tools/check/go.mod
cd tools/check && $(GO) build -o ../bin/gotestsum gotest.tools/gotestsum

tools/bin/gocov: tools/check/go.mod
cd tools/check && $(GO) build -o ../bin/gocov github.com/axw/gocov/gocov

tools/bin/gocov-xml: tools/check/go.mod
cd tools/check && $(GO) build -o ../bin/gocov-xml github.com/AlekSi/gocov-xml
5 changes: 5 additions & 0 deletions Makefile.common
Original file line number Diff line number Diff line change
Expand Up @@ -47,8 +47,11 @@ MAC := "Darwin"

PACKAGE_LIST := go list ./...
PACKAGE_LIST_TIDB_TESTS := go list ./... | grep -vE "github.com\/pingcap\/tidb\/br|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/dumpling"
PACKAGE_LIST_TEST_OTHERS := go list ./... | grep -vE "github.com\/pingcap\/tidb\/br|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/dumpling|github.com\/pingcap\/tidb\/executor|github.com\/pingcap\/tidb\/cmd|github.com\/pingcap\/tidb\/ddl"
PACKAGES ?= $$($(PACKAGE_LIST))
PACKAGES_TIDB_TESTS ?= $$($(PACKAGE_LIST_TIDB_TESTS))
PACKAGES_TIDB_TESTS_EXPENSIVE ?= "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/ddl"
PACKAGES_TIDB_TESTS_OTHERS ?= $$($(PACKAGE_LIST_TEST_OTHERS))
PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||'
PACKAGE_DIRECTORIES_TIDB_TESTS := $(PACKAGE_LIST_TIDB_TESTS) | sed 's|github.com/pingcap/$(PROJECT)/||'
FILES := $$(find $$($(PACKAGE_DIRECTORIES)) -name "*.go")
Expand Down Expand Up @@ -111,3 +114,5 @@ DUMPLING_LDFLAGS += -X "github.com/pingcap/tidb/dumpling/cli.GoVersion=$(shell g

DUMPLING_GOBUILD := CGO_ENABLED=0 GO111MODULE=on go build -trimpath -ldflags '$(DUMPLING_LDFLAGS)'
DUMPLING_GOTEST := CGO_ENABLED=1 GO111MODULE=on go test -ldflags '$(DUMPLING_LDFLAGS)'

TEST_COVERAGE_DIR := "test_coverage"
29 changes: 21 additions & 8 deletions br/pkg/lightning/config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -552,27 +552,37 @@ type Security struct {
KeyPath string `toml:"key-path" json:"key-path"`
// RedactInfoLog indicates that whether enabling redact log
RedactInfoLog bool `toml:"redact-info-log" json:"redact-info-log"`

// TLSConfigName is used to set tls config for lightning in DM, so we don't expose this field to user
// DM may running many lightning instances at same time, so we need to set different tls config name for each lightning
TLSConfigName string `toml:"-" json:"-"`
}

// RegistersMySQL registers (or deregisters) the TLS config with name "cluster"
// RegisterMySQL registers the TLS config with name "cluster" or security.TLSConfigName
// for use in `sql.Open()`. This method is goroutine-safe.
func (sec *Security) RegisterMySQL() error {
if sec == nil {
return nil
}
tlsConfig, err := common.ToTLSConfig(sec.CAPath, sec.CertPath, sec.KeyPath)
switch {
case err != nil:
if err != nil {
return errors.Trace(err)
case tlsConfig != nil:
}
if tlsConfig != nil {
// error happens only when the key coincides with the built-in names.
_ = gomysql.RegisterTLSConfig("cluster", tlsConfig)
default:
gomysql.DeregisterTLSConfig("cluster")
_ = gomysql.RegisterTLSConfig(sec.TLSConfigName, tlsConfig)
}
return nil
}

// DeregisterMySQL deregisters the TLS config with security.TLSConfigName
func (sec *Security) DeregisterMySQL() {
if sec == nil || len(sec.CAPath) == 0 {
return
}
gomysql.DeregisterTLSConfig(sec.TLSConfigName)
}

// A duration which can be deserialized from a TOML string.
// Implemented as https://github.com/BurntSushi/toml#using-the-encodingtextunmarshaler-interface
type Duration struct {
Expand Down Expand Up @@ -1124,7 +1134,10 @@ func (cfg *Config) CheckAndAdjustSecurity() error {
switch cfg.TiDB.TLS {
case "":
if len(cfg.TiDB.Security.CAPath) > 0 {
cfg.TiDB.TLS = "cluster"
if cfg.TiDB.Security.TLSConfigName == "" {
cfg.TiDB.Security.TLSConfigName = "cluster" // adjust this the default value
}
cfg.TiDB.TLS = cfg.TiDB.Security.TLSConfigName
} else {
cfg.TiDB.TLS = "false"
}
Expand Down
7 changes: 7 additions & 0 deletions br/pkg/lightning/config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -323,6 +323,13 @@ func (s *configTestSuite) TestAdjustSecuritySection(c *C) {
c.Assert(cfg.TiDB.Security.CAPath, Equals, tc.expectedCA, comment)
c.Assert(cfg.TiDB.TLS, Equals, tc.expectedTLS, comment)
}
// test different tls config name
cfg := config.NewConfig()
assignMinimalLegalValue(cfg)
cfg.Security.CAPath = "/path/to/ca.pem"
cfg.Security.TLSConfigName = "tidb-tls"
c.Assert(cfg.Adjust(context.Background()), IsNil)
c.Assert(cfg.TiDB.Security.TLSConfigName, Equals, cfg.TiDB.TLS)
}

func (s *configTestSuite) TestInvalidCSV(c *C) {
Expand Down
5 changes: 1 addition & 4 deletions br/pkg/lightning/lightning.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,10 +274,7 @@ func (l *Lightning) run(taskCtx context.Context, taskCfg *config.Config, g glue.
if taskCfg.TiDB.Security == nil {
return
}
taskCfg.TiDB.Security.CAPath = ""
if err := taskCfg.TiDB.Security.RegisterMySQL(); err != nil {
log.L().Warn("failed to deregister TLS config", log.ShortError(err))
}
taskCfg.TiDB.Security.DeregisterMySQL()
}()

// initiation of default glue should be after RegisterMySQL, which is ready to be called after taskCfg.Adjust
Expand Down
4 changes: 2 additions & 2 deletions br/pkg/lightning/restore/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -787,6 +787,7 @@ func (rc *Controller) restoreSchema(ctx context.Context) error {
os.Exit(0)
})

rc.checkpointsWg.Add(1) // checkpointsWg will be done in `rc.listenCheckpointUpdates`
go rc.listenCheckpointUpdates()

sysVars := ObtainImportantVariables(ctx, rc.tidbGlue.GetSQLExecutor(), !rc.isTiDBBackend())
Expand Down Expand Up @@ -993,7 +994,7 @@ func (rc *Controller) saveStatusCheckpoint(ctx context.Context, tableName string

// listenCheckpointUpdates will combine several checkpoints together to reduce database load.
func (rc *Controller) listenCheckpointUpdates() {
rc.checkpointsWg.Add(1)
defer rc.checkpointsWg.Done()

var lock sync.Mutex
coalesed := make(map[string]*checkpoints.TableCheckpointDiff)
Expand Down Expand Up @@ -1082,7 +1083,6 @@ func (rc *Controller) listenCheckpointUpdates() {
}
})
}
rc.checkpointsWg.Done()
}

// buildRunPeriodicActionAndCancelFunc build the runPeriodicAction func and a cancel func
Expand Down
1 change: 1 addition & 0 deletions br/pkg/lightning/restore/restore_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1118,6 +1118,7 @@ func (s *tableRestoreSuite) TestSaveStatusCheckpoint(c *C) {
saveCpCh: saveCpCh,
checkpointsDB: checkpoints.NewNullCheckpointsDB(),
}
rc.checkpointsWg.Add(1)
go rc.listenCheckpointUpdates()

start := time.Now()
Expand Down
11 changes: 10 additions & 1 deletion br/pkg/restore/split_client.go
Original file line number Diff line number Diff line change
Expand Up @@ -416,6 +416,11 @@ func (c *pdClient) getMaxReplica(ctx context.Context) (int, error) {
if err != nil {
return 0, errors.Trace(err)
}
defer func() {
if err = res.Body.Close(); err != nil {
log.Error("Response fail to close", zap.Error(err))
}
}()
var conf config.Config
if err := json.NewDecoder(res.Body).Decode(&conf); err != nil {
return 0, errors.Trace(err)
Expand Down Expand Up @@ -482,11 +487,15 @@ func (c *pdClient) GetPlacementRule(ctx context.Context, groupID, ruleID string)
if err != nil {
return rule, errors.Trace(err)
}
defer func() {
if err = res.Body.Close(); err != nil {
log.Error("Response fail to close", zap.Error(err))
}
}()
b, err := io.ReadAll(res.Body)
if err != nil {
return rule, errors.Trace(err)
}
res.Body.Close()
err = json.Unmarshal(b, &rule)
if err != nil {
return rule, errors.Trace(err)
Expand Down
7 changes: 1 addition & 6 deletions br/pkg/storage/gcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,11 +180,6 @@ func (s *gcsStorage) WalkDir(ctx context.Context, opt *WalkOption, fn func(strin
opt = &WalkOption{}
}

maxKeys := int64(1000)
if opt.ListCount > 0 {
maxKeys = opt.ListCount
}

prefix := path.Join(s.gcs.Prefix, opt.SubDir)
if len(prefix) > 0 && !strings.HasSuffix(prefix, "/") {
prefix += "/"
Expand All @@ -194,7 +189,7 @@ func (s *gcsStorage) WalkDir(ctx context.Context, opt *WalkOption, fn func(strin
// only need each object's name and size
query.SetAttrSelection([]string{"Name", "Size"})
iter := s.bucket.Objects(ctx, query)
for i := int64(0); i != maxKeys; i++ {
for {
attrs, err := iter.Next()
if err == iterator.Done {
break
Expand Down
26 changes: 26 additions & 0 deletions br/pkg/storage/gcs_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ package storage

import (
"context"
"fmt"
"io"
"os"

Expand Down Expand Up @@ -95,6 +96,31 @@ func (r *testStorageSuite) TestGCS(c *C) {
c.Assert(list, Equals, "keykey1key2")
c.Assert(totalSize, Equals, int64(42))

// test 1003 files
totalSize = 0
for i := 0; i < 1000; i += 1 {
err = stg.WriteFile(ctx, fmt.Sprintf("f%d", i), []byte("data"))
c.Assert(err, IsNil)
}
filesSet := make(map[string]struct{}, 1003)
err = stg.WalkDir(ctx, nil, func(name string, size int64) error {
filesSet[name] = struct{}{}
totalSize += size
return nil
})
c.Assert(err, IsNil)
c.Assert(totalSize, Equals, int64(42+4000))
_, ok := filesSet["key"]
c.Assert(ok, IsTrue)
_, ok = filesSet["key1"]
c.Assert(ok, IsTrue)
_, ok = filesSet["key2"]
c.Assert(ok, IsTrue)
for i := 0; i < 1000; i += 1 {
_, ok = filesSet[fmt.Sprintf("f%d", i)]
c.Assert(ok, IsTrue)
}

efr, err := stg.Open(ctx, "key2")
c.Assert(err, IsNil)

Expand Down
2 changes: 1 addition & 1 deletion br/pkg/task/common.go
Original file line number Diff line number Diff line change
Expand Up @@ -355,7 +355,7 @@ func (cfg *Config) parseCipherInfo(flags *pflag.FlagSet) error {
}

if !checkCipherKeyMatch(&cfg.CipherInfo) {
return errors.Annotate(err, "Cipher type and key not match")
return errors.Annotate(berrors.ErrInvalidArgument, "crypter method and key length not match")
}

return nil
Expand Down
Loading

0 comments on commit f015eab

Please sign in to comment.