Skip to content

Commit

Permalink
Merge branch 'release-6.0' into fix_cte60
Browse files Browse the repository at this point in the history
  • Loading branch information
wjhuang2016 authored Mar 31, 2022
2 parents 27feece + 238f79b commit 358487f
Show file tree
Hide file tree
Showing 28 changed files with 356 additions and 113 deletions.
1 change: 1 addition & 0 deletions br/pkg/glue/glue.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ type Session interface {
CreateTable(ctx context.Context, dbName model.CIStr, table *model.TableInfo) error
CreatePlacementPolicy(ctx context.Context, policy *model.PolicyInfo) error
Close()
GetGlobalVariable(name string) (string, error)
}

// BatchCreateTableSession is an interface to batch create table parallelly
Expand Down
5 changes: 5 additions & 0 deletions br/pkg/gluetidb/glue.go
Original file line number Diff line number Diff line change
Expand Up @@ -203,6 +203,11 @@ func (gs *tidbSession) Close() {
gs.se.Close()
}

// GetGlobalVariables implements glue.Session.
func (gs *tidbSession) GetGlobalVariable(name string) (string, error) {
return gs.se.GetSessionVars().GlobalVarsAccessor.GetTiDBTableValue(name)
}

// showCreateTable shows the result of SHOW CREATE TABLE from a TableInfo.
func (gs *tidbSession) showCreateTable(tbl *model.TableInfo) (string, error) {
table := tbl.Clone()
Expand Down
13 changes: 10 additions & 3 deletions br/pkg/restore/split.go
Original file line number Diff line number Diff line change
Expand Up @@ -438,11 +438,18 @@ func PaginateScanRegion(
}

var regions []*RegionInfo
err := utils.WithRetry(ctx, func() error {
var err error
// we don't need to return multierr. since there only 3 times retry.
// in most case 3 times retry have the same error. so we just return the last error.
// actually we'd better remove all multierr in br/lightning.
// because it's not easy to check multierr equals normal error.
// see https://github.com/pingcap/tidb/issues/33419.
_ = utils.WithRetry(ctx, func() error {
regions = []*RegionInfo{}
scanStartKey := startKey
for {
batch, err := client.ScanRegions(ctx, scanStartKey, endKey, limit)
var batch []*RegionInfo
batch, err = client.ScanRegions(ctx, scanStartKey, endKey, limit)
if err != nil {
return errors.Trace(err)
}
Expand All @@ -458,7 +465,7 @@ func PaginateScanRegion(
break
}
}
if err := CheckRegionConsistency(startKey, endKey, regions); err != nil {
if err = CheckRegionConsistency(startKey, endKey, regions); err != nil {
log.Warn("failed to scan region, retrying", logutil.ShortError(err))
return err
}
Expand Down
13 changes: 13 additions & 0 deletions br/pkg/restore/split_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ type TestClient struct {
supportBatchScatter bool

scattered map[uint64]bool
InjectErr bool
}

func NewTestClient(
Expand Down Expand Up @@ -215,6 +216,10 @@ func (c *TestClient) GetOperator(ctx context.Context, regionID uint64) (*pdpb.Ge
}

func (c *TestClient) ScanRegions(ctx context.Context, key, endKey []byte, limit int) ([]*restore.RegionInfo, error) {
if c.InjectErr {
return nil, errors.New("mock scan error")
}

infos := c.regionsInfo.ScanRange(key, endKey, limit)
regions := make([]*restore.RegionInfo, 0, len(infos))
for _, info := range infos {
Expand Down Expand Up @@ -567,12 +572,17 @@ func TestRegionConsistency(t *testing.T) {
}

type fakeRestorer struct {
mu sync.Mutex

errorInSplit bool
splitRanges []rtree.Range
restoredFiles []*backuppb.File
}

func (f *fakeRestorer) SplitRanges(ctx context.Context, ranges []rtree.Range, rewriteRules *restore.RewriteRules, updateCh glue.Progress, isRawKv bool) error {
f.mu.Lock()
defer f.mu.Unlock()

if ctx.Err() != nil {
return ctx.Err()
}
Expand All @@ -587,6 +597,9 @@ func (f *fakeRestorer) SplitRanges(ctx context.Context, ranges []rtree.Range, re
}

func (f *fakeRestorer) RestoreFiles(ctx context.Context, files []*backuppb.File, rewriteRules *restore.RewriteRules, updateCh glue.Progress) error {
f.mu.Lock()
defer f.mu.Unlock()

if ctx.Err() != nil {
return ctx.Err()
}
Expand Down
10 changes: 10 additions & 0 deletions br/pkg/restore/util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
backuppb "github.com/pingcap/kvproto/pkg/brpb"
"github.com/pingcap/kvproto/pkg/import_sstpb"
"github.com/pingcap/kvproto/pkg/metapb"
berrors "github.com/pingcap/tidb/br/pkg/errors"
"github.com/pingcap/tidb/br/pkg/restore"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/codec"
Expand Down Expand Up @@ -228,6 +229,7 @@ func TestPaginateScanRegion(t *testing.T) {
var batch []*restore.RegionInfo
_, err := restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{}, []byte{}, 3)
require.Error(t, err)
require.True(t, berrors.ErrPDBatchScanRegion.Equal(err))
require.Regexp(t, ".*scan region return empty result.*", err.Error())

regionMap, regions = makeRegions(1)
Expand Down Expand Up @@ -268,12 +270,20 @@ func TestPaginateScanRegion(t *testing.T) {

_, err = restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), []byte{2}, []byte{1}, 3)
require.Error(t, err)
require.True(t, berrors.ErrRestoreInvalidRange.Equal(err))
require.Regexp(t, ".*startKey >= endKey.*", err.Error())

tc := NewTestClient(stores, regionMap, 0)
tc.InjectErr = true
_, err = restore.PaginateScanRegion(ctx, tc, regions[1].Region.EndKey, regions[5].Region.EndKey, 3)
require.Error(t, err)
require.Regexp(t, ".*mock scan error.*", err.Error())

// make the regionMap losing some region, this will cause scan region check fails
delete(regionMap, uint64(3))
_, err = restore.PaginateScanRegion(ctx, NewTestClient(stores, regionMap, 0), regions[1].Region.EndKey, regions[5].Region.EndKey, 3)
require.Error(t, err)
require.True(t, berrors.ErrPDBatchScanRegion.Equal(err))
require.Regexp(t, ".*region endKey not equal to next region startKey.*", err.Error())

}
11 changes: 11 additions & 0 deletions br/pkg/task/backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -261,6 +261,16 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig
statsHandle = mgr.GetDomain().StatsHandle()
}

se, err := g.CreateSession(mgr.GetStorage())
if err != nil {
return errors.Trace(err)
}
newCollationEnable, err := se.GetGlobalVariable(tidbNewCollationEnabled)
if err != nil {
return errors.Trace(err)
}
log.Info("get newCollationEnable for check during restore", zap.String("newCollationEnable", newCollationEnable))

client, err := backup.NewBackupClient(ctx, mgr)
if err != nil {
return errors.Trace(err)
Expand Down Expand Up @@ -351,6 +361,7 @@ func RunBackup(c context.Context, g glue.Glue, cmdName string, cfg *BackupConfig
m.ClusterId = req.ClusterId
m.ClusterVersion = clusterVersion
m.BrVersion = brVersion
m.NewCollationsEnabled = newCollationEnable
})

log.Info("get placement policies", zap.Int("count", len(policies)))
Expand Down
2 changes: 2 additions & 0 deletions br/pkg/task/common.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,8 @@ const (
crypterAES128KeyLen = 16
crypterAES192KeyLen = 24
crypterAES256KeyLen = 32

tidbNewCollationEnabled = "new_collation_enabled"
)

// TLSConfig is the common configuration for TLS connection.
Expand Down
42 changes: 42 additions & 0 deletions br/pkg/task/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ package task

import (
"context"
"strings"
"time"

"github.com/opentracing/opentracing-go"
Expand All @@ -21,6 +22,7 @@ import (
"github.com/pingcap/tidb/br/pkg/utils"
"github.com/pingcap/tidb/br/pkg/version"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/spf13/pflag"
"go.uber.org/multierr"
"go.uber.org/zap"
Expand Down Expand Up @@ -265,6 +267,42 @@ func CheckRestoreDBAndTable(client *restore.Client, cfg *RestoreConfig) error {
return nil
}

func CheckNewCollationEnable(
backupNewCollationEnable string,
g glue.Glue,
storage kv.Storage,
CheckRequirements bool,
) error {
if backupNewCollationEnable == "" {
if CheckRequirements {
return errors.Annotatef(berrors.ErrUnknown,
"NewCollactionEnable not found in backupmeta. "+
"if you ensure the NewCollactionEnable config of backup cluster is as same as restore cluster, "+
"use --check-requirements=false to skip")
} else {
log.Warn("no NewCollactionEnable in backup")
return nil
}
}

se, err := g.CreateSession(storage)
if err != nil {
return errors.Trace(err)
}

newCollationEnable, err := se.GetGlobalVariable(tidbNewCollationEnabled)
if err != nil {
return errors.Trace(err)
}

if !strings.EqualFold(backupNewCollationEnable, newCollationEnable) {
return errors.Annotatef(berrors.ErrUnknown,
"newCollationEnable not match, upstream:%v, downstream: %v",
backupNewCollationEnable, newCollationEnable)
}
return nil
}

func isFullRestore(cmdName string) bool {
return cmdName == FullRestoreCmd
}
Expand Down Expand Up @@ -315,6 +353,10 @@ func RunRestore(c context.Context, g glue.Glue, cmdName string, cfg *RestoreConf
return errors.Trace(versionErr)
}
}
if err = CheckNewCollationEnable(backupMeta.GetNewCollationsEnabled(), g, mgr.GetStorage(), cfg.CheckRequirements); err != nil {
return errors.Trace(err)
}

reader := metautil.NewMetaReader(backupMeta, s, &cfg.CipherInfo)
if err = client.InitBackupMeta(c, backupMeta, u, s, reader); err != nil {
return errors.Trace(err)
Expand Down
1 change: 1 addition & 0 deletions br/pkg/version/version.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,7 @@ func CheckClusterVersion(ctx context.Context, client pd.Client, checker VerCheck
if err := checkTiFlashVersion(s); err != nil {
return errors.Trace(err)
}
continue
}

tikvVersionString := removeVAndHash(s.Version)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# config of tidb

# Schema lease duration
# There are lot of ddl in the tests, setting this
# to 360s to test whether BR is gracefully shutdown.
lease = "360s"

new_collations_enabled_on_first_bootstrap = false

[security]
ssl-ca = "/tmp/backup_restore_test/certs/ca.pem"
ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem"
ssl-key = "/tmp/backup_restore_test/certs/tidb.key"
cluster-ssl-ca = "/tmp/backup_restore_test/certs/ca.pem"
cluster-ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem"
cluster-ssl-key = "/tmp/backup_restore_test/certs/tidb.key"
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# config of tidb

# Schema lease duration
# There are lot of ddl in the tests, setting this
# to 360s to test whether BR is gracefully shutdown.
lease = "360s"

new_collations_enabled_on_first_bootstrap = true

[security]
ssl-ca = "/tmp/backup_restore_test/certs/ca.pem"
ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem"
ssl-key = "/tmp/backup_restore_test/certs/tidb.key"
cluster-ssl-ca = "/tmp/backup_restore_test/certs/ca.pem"
cluster-ssl-cert = "/tmp/backup_restore_test/certs/tidb.pem"
cluster-ssl-key = "/tmp/backup_restore_test/certs/tidb.key"
102 changes: 102 additions & 0 deletions br/tests/br_check_new_collocation_enable/run.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,102 @@
#!/bin/sh
#
# Copyright 2019 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"

cur=$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )
source $cur/../_utils/run_services

PROGRESS_FILE="$TEST_DIR/progress_unit_file"
rm -rf $PROGRESS_FILE

run_sql "CREATE DATABASE $DB;"

run_sql "CREATE TABLE $DB.usertable1 ( \
YCSB_KEY varchar(64) NOT NULL, \
FIELD0 varchar(1) DEFAULT NULL, \
PRIMARY KEY (YCSB_KEY) \
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;"

run_sql "INSERT INTO $DB.usertable1 VALUES (\"a\", \"b\");"
run_sql "INSERT INTO $DB.usertable1 VALUES (\"aa\", \"b\");"

run_sql "CREATE TABLE $DB.usertable2 ( \
YCSB_KEY varchar(64) NOT NULL, \
FIELD0 varchar(1) DEFAULT NULL, \
PRIMARY KEY (YCSB_KEY) \
) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin;"

run_sql "INSERT INTO $DB.usertable2 VALUES (\"c\", \"d\");"

# backup db
echo "backup start ... with brv4.0.8 without NewCollactionEnable"
bin/brv4.0.8 backup db --db "$DB" -s "local://$TEST_DIR/$DB" \
--ca "$TEST_DIR/certs/ca.pem" \
--cert "$TEST_DIR/certs/br.pem" \
--key "$TEST_DIR/certs/br.key" \
--pd $PD_ADDR \
--check-requirements=false

# restore db from v4.0.8 version without `newCollationEnable`
echo "restore start ... without NewCollactionEnable in backupmeta"
restore_fail=0
error_str="NewCollactionEnable not found in backupmeta"
test_log="new_collotion_enable_test.log"
unset BR_LOG_TO_TERM
run_br restore db --db $DB -s "local://$TEST_DIR/$DB" --pd $PD_ADDR --log-file $test_log || restore_fail=1
if [ $restore_fail -ne 1 ]; then
echo "TEST: [$TEST_NAME] test restore failed!"
exit 1
fi

if ! grep -i "$error_str" $test_log; then
echo "${error_str} not found in log"
echo "TEST: [$TEST_NAME] test restore failed!"
exit 1
fi

rm -rf "$test_log"

# backup with NewCollationEable = false
echo "Restart cluster with new_collation_enable=false"
start_services --tidb-cfg $cur/config/new_collation_enable_false.toml

echo "backup start ... witch NewCollactionEnable=false in TiDB"
run_br --pd $PD_ADDR backup db --db "$DB" -s "local://$cur/${DB}_2"

echo "Restart cluster with new_collation_enable=true"
start_services --tidb-cfg $cur/config/new_collation_enable_true.toml

echo "restore start ... with NewCollactionEnable=True in TiDB"
restore_fail=0
test_log2="new_collotion_enable_test2.log"
error_str="newCollationEnable not match"
unset BR_LOG_TO_TERM
run_br restore db --db $DB -s "local://$cur/${DB}_2" --pd $PD_ADDR --log-file $test_log2 || restore_fail=1
if [ $restore_fail -ne 1 ]; then
echo "TEST: [$TEST_NAME] test restore failed!"
exit 1
fi

if ! grep -i "$error_str" $test_log2; then
echo "${error_str} not found in log"
echo "TEST: [$TEST_NAME] test restore failed!"
exit 1
fi

rm -rf "$test_log2"
rm -rf "$cur/${DB}_2"
6 changes: 6 additions & 0 deletions br/tests/br_s3/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,12 @@ for p in $(seq 2); do
exit 1
fi

target_log="get newCollationEnable for check during restore"
if ! grep -i "$target_log" $BACKUP_LOG; then
echo "${target_log} not found in log"
exit 1
fi

for i in $(seq $DB_COUNT); do
run_sql "DROP DATABASE $DB${i};"
done
Expand Down
Loading

0 comments on commit 358487f

Please sign in to comment.