Skip to content

Commit

Permalink
Merge branch 'master' into mvindex-empty-array
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored Feb 3, 2023
2 parents d5ae03e + b91e2d9 commit ab62491
Show file tree
Hide file tree
Showing 16 changed files with 157 additions and 102 deletions.
8 changes: 6 additions & 2 deletions br/pkg/lightning/backend/local/local.go
Original file line number Diff line number Diff line change
Expand Up @@ -1656,6 +1656,9 @@ func (local *local) ImportEngine(ctx context.Context, engineUUID uuid.UUID, regi
// the table when table is created.
needSplit := len(unfinishedRanges) > 1 || lfTotalSize > regionSplitSize || lfLength > regionSplitKeys
// split region by given ranges
failpoint.Inject("failToSplit", func(_ failpoint.Value) {
needSplit = true
})
for i := 0; i < maxRetryTimes; i++ {
err = local.SplitAndScatterRegionInBatches(ctx, unfinishedRanges, lf.tableInfo, needSplit, regionSplitSize, maxBatchSplitRanges)
if err == nil || common.IsContextCanceledError(err) {
Expand Down Expand Up @@ -2063,7 +2066,8 @@ func nextKey(key []byte) []byte {

// in tikv <= 4.x, tikv will truncate the row key, so we should fetch the next valid row key
// See: https://github.com/tikv/tikv/blob/f7f22f70e1585d7ca38a59ea30e774949160c3e8/components/raftstore/src/coprocessor/split_observer.rs#L36-L41
if tablecodec.IsRecordKey(key) {
// we only do this for IntHandle, which is checked by length
if tablecodec.IsRecordKey(key) && len(key) == tablecodec.RecordRowKeyLen {
tableID, handle, _ := tablecodec.DecodeRecordKey(key)
nextHandle := handle.Next()
// int handle overflow, use the next table prefix as nextKey
Expand All @@ -2073,7 +2077,7 @@ func nextKey(key []byte) []byte {
return tablecodec.EncodeRowKeyWithHandle(tableID, nextHandle)
}

// if key is an index, directly append a 0x00 to the key.
// for index key and CommonHandle, directly append a 0x00 to the key.
res := make([]byte, 0, len(key)+1)
res = append(res, key...)
res = append(res, 0)
Expand Down
15 changes: 13 additions & 2 deletions br/pkg/lightning/backend/local/local_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,10 +115,21 @@ func TestNextKey(t *testing.T) {
require.NoError(t, err)
nextHdl, err := tidbkv.NewCommonHandle(nextKeyBytes)
require.NoError(t, err)
expectNextKey := []byte(tablecodec.EncodeRowKeyWithHandle(1, nextHdl))
require.Equal(t, expectNextKey, nextKey(key))
nextValidKey := []byte(tablecodec.EncodeRowKeyWithHandle(1, nextHdl))
// nextKey may return a key that can't be decoded, but it must not be larger than the valid next key.
require.True(t, bytes.Compare(nextKey(key), nextValidKey) <= 0, "datums: %v", datums)
}

// a special case that when len(string datum) % 8 == 7, nextKey twice should not panic.
keyBytes, err := codec.EncodeKey(stmtCtx, nil, types.NewStringDatum("1234567"))
require.NoError(t, err)
h, err := tidbkv.NewCommonHandle(keyBytes)
require.NoError(t, err)
key = tablecodec.EncodeRowKeyWithHandle(1, h)
nextOnce := nextKey(key)
// should not panic
_ = nextKey(nextOnce)

// dIAAAAAAAAD/PV9pgAAAAAD/AAABA4AAAAD/AAAAAQOAAAD/AAAAAAEAAAD8
// a index key with: table: 61, index: 1, int64: 1, int64: 1
a := []byte{116, 128, 0, 0, 0, 0, 0, 0, 255, 61, 95, 105, 128, 0, 0, 0, 0, 255, 0, 0, 1, 3, 128, 0, 0, 0, 255, 0, 0, 0, 1, 3, 128, 0, 0, 255, 0, 0, 0, 0, 1, 0, 0, 0, 252}
Expand Down
10 changes: 9 additions & 1 deletion br/pkg/lightning/backend/local/localhelper.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (

"github.com/docker/go-units"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
sst "github.com/pingcap/kvproto/pkg/import_sstpb"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/kvproto/pkg/pdpb"
Expand Down Expand Up @@ -379,7 +380,14 @@ func fetchTableRegionSizeStats(ctx context.Context, db *sql.DB, tableID int64) (
return stats, errors.Trace(err)
}

func (local *local) BatchSplitRegions(ctx context.Context, region *split.RegionInfo, keys [][]byte) (*split.RegionInfo, []*split.RegionInfo, error) {
func (local *local) BatchSplitRegions(
ctx context.Context,
region *split.RegionInfo,
keys [][]byte,
) (*split.RegionInfo, []*split.RegionInfo, error) {
failpoint.Inject("failToSplit", func(_ failpoint.Value) {
failpoint.Return(nil, nil, errors.New("retryable error"))
})
region, newRegions, err := local.splitCli.BatchSplitRegionsWithOrigin(ctx, region, keys)
if err != nil {
return nil, nil, errors.Annotatef(err, "batch split regions failed")
Expand Down
47 changes: 16 additions & 31 deletions br/pkg/restore/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,6 @@ package restore
import (
"context"
"io"
"sync/atomic"

"github.com/pingcap/errors"
"github.com/pingcap/kvproto/pkg/metapb"
Expand Down Expand Up @@ -80,29 +79,27 @@ func NewStoreMeta(storeId uint64) StoreMeta {

// for test
type Recovery struct {
allStores []*metapb.Store
StoreMetas []StoreMeta
RecoveryPlan map[uint64][]*recovpb.RecoverRegionRequest
MaxAllocID uint64
mgr *conn.Mgr
progress glue.Progress
concurrency uint32
totalFlashbackRegions uint64
allStores []*metapb.Store
StoreMetas []StoreMeta
RecoveryPlan map[uint64][]*recovpb.RecoverRegionRequest
MaxAllocID uint64
mgr *conn.Mgr
progress glue.Progress
concurrency uint32
}

func NewRecovery(allStores []*metapb.Store, mgr *conn.Mgr, progress glue.Progress, concurrency uint32) Recovery {
totalStores := len(allStores)
var StoreMetas = make([]StoreMeta, totalStores)
var regionRecovers = make(map[uint64][]*recovpb.RecoverRegionRequest, totalStores)
return Recovery{
allStores: allStores,
StoreMetas: StoreMetas,
RecoveryPlan: regionRecovers,
MaxAllocID: 0,
mgr: mgr,
progress: progress,
concurrency: concurrency,
totalFlashbackRegions: 0}
allStores: allStores,
StoreMetas: StoreMetas,
RecoveryPlan: regionRecovers,
MaxAllocID: 0,
mgr: mgr,
progress: progress,
concurrency: concurrency}
}

func (recovery *Recovery) newRecoveryClient(ctx context.Context, storeAddr string) (recovpb.RecoverDataClient, *grpc.ClientConn, error) {
Expand Down Expand Up @@ -305,12 +302,8 @@ func (recovery *Recovery) WaitApply(ctx context.Context) (err error) {

// prepare the region for flashback the data, the purpose is to stop region service, put region in flashback state
func (recovery *Recovery) PrepareFlashbackToVersion(ctx context.Context, resolveTS uint64, startTS uint64) (err error) {
var totalRegions atomic.Uint64
totalRegions.Store(0)

handler := func(ctx context.Context, r tikvstore.KeyRange) (rangetask.TaskStat, error) {
stats, err := ddl.SendPrepareFlashbackToVersionRPC(ctx, recovery.mgr.GetStorage().(tikv.Storage), resolveTS, startTS, r)
totalRegions.Add(uint64(stats.CompletedRegions))
return stats, err
}

Expand All @@ -321,23 +314,16 @@ func (recovery *Recovery) PrepareFlashbackToVersion(ctx context.Context, resolve
log.Error("region flashback prepare get error")
return errors.Trace(err)
}

recovery.totalFlashbackRegions = totalRegions.Load()
recovery.progress.Inc()
log.Info("region flashback prepare complete", zap.Int("regions", runner.CompletedRegions()))

return nil
}

// flashback the region data to version resolveTS
func (recovery *Recovery) FlashbackToVersion(ctx context.Context, resolveTS uint64, commitTS uint64) (err error) {
var completedRegions atomic.Uint64

// only know the total progress of tikv, progress is total state of the whole restore flow.
ratio := int(recovery.totalFlashbackRegions) / len(recovery.allStores)

handler := func(ctx context.Context, r tikvstore.KeyRange) (rangetask.TaskStat, error) {
stats, err := ddl.SendFlashbackToVersionRPC(ctx, recovery.mgr.GetStorage().(tikv.Storage), resolveTS, commitTS-1, commitTS, r)
completedRegions.Add(uint64(stats.CompletedRegions))
return stats, err
}

Expand All @@ -352,13 +338,12 @@ func (recovery *Recovery) FlashbackToVersion(ctx context.Context, resolveTS uint
return errors.Trace(err)
}

recovery.progress.IncBy(int64(completedRegions.Load()) / int64(ratio))

log.Info("region flashback complete",
zap.Uint64("resolveTS", resolveTS),
zap.Uint64("commitTS", commitTS),
zap.Int("regions", runner.CompletedRegions()))

recovery.progress.Inc()
return nil
}

Expand Down
6 changes: 3 additions & 3 deletions br/pkg/task/restore_data.go
Original file line number Diff line number Diff line change
Expand Up @@ -139,9 +139,9 @@ func RunResolveKvData(c context.Context, g glue.Glue, cmdName string, cfg *Resto
}

log.Debug("total tikv", zap.Int("total", numBackupStore), zap.String("progress file", cfg.ProgressFile))
// progress = read meta + send recovery + iterate tikv + flashback.
progress := g.StartProgress(ctx, cmdName, int64(numBackupStore*4), !cfg.LogProgress)
go progressFileWriterRoutine(ctx, progress, int64(numBackupStore*4), cfg.ProgressFile)
// progress = read meta + send recovery + iterate tikv + (1 * prepareflashback + 1 * flashback)
progress := g.StartProgress(ctx, cmdName, int64(numBackupStore*3+2), !cfg.LogProgress)
go progressFileWriterRoutine(ctx, progress, int64(numBackupStore*3+2), cfg.ProgressFile)

// restore tikv data from a snapshot volume
var totalRegions int
Expand Down
2 changes: 1 addition & 1 deletion br/tests/lightning_local_backend/data/cpeng.a-schema.sql
Original file line number Diff line number Diff line change
@@ -1 +1 @@
create table a (c int);
create table a (c VARCHAR(20) PRIMARY KEY);
2 changes: 1 addition & 1 deletion br/tests/lightning_local_backend/data/cpeng.a.1.sql
Original file line number Diff line number Diff line change
@@ -1 +1 @@
insert into a values (1);
insert into a values ('0000001');
2 changes: 1 addition & 1 deletion br/tests/lightning_local_backend/data/cpeng.a.2.sql
Original file line number Diff line number Diff line change
@@ -1 +1 @@
insert into a values (2);
insert into a values ('0000002');
2 changes: 1 addition & 1 deletion br/tests/lightning_local_backend/data/cpeng.a.3.sql
Original file line number Diff line number Diff line change
@@ -1 +1 @@
insert into a values (3),(4);
insert into a values ('0000003'),('0000004');
5 changes: 3 additions & 2 deletions br/tests/lightning_local_backend/run.sh
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,11 @@ grep -Fq 'table(s) [`cpeng`.`a`, `cpeng`.`b`] are not empty' $TEST_DIR/lightning


# First, verify that inject with not leader error is fine.
export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=1*return("notleader")'
export GO_FAILPOINTS='github.com/pingcap/tidb/br/pkg/lightning/backend/local/FailIngestMeta=1*return("notleader");github.com/pingcap/tidb/br/pkg/lightning/backend/local/failToSplit=2*return("")'
rm -f "$TEST_DIR/lightning-local.log"
run_sql 'DROP DATABASE IF EXISTS cpeng;'
run_lightning --backend local --enable-checkpoint=1 --log-file "$TEST_DIR/lightning-local.log" --config "tests/$TEST_NAME/config.toml"
run_lightning --backend local --enable-checkpoint=1 --log-file "$TEST_DIR/lightning-local.log" --config "tests/$TEST_NAME/config.toml" -L debug
grep -Eq "split regions.*retryable error" "$TEST_DIR/lightning-local.log"

# Check that everything is correctly imported
run_sql 'SELECT count(*), sum(c) FROM cpeng.a'
Expand Down
6 changes: 3 additions & 3 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@ type Domain struct {
expiredTimeStamp4PC types.Time
logBackupAdvancer *daemon.OwnerDaemon
historicalStatsWorker *HistoricalStatsWorker
ttlJobManager *ttlworker.JobManager
ttlJobManager atomic.Pointer[ttlworker.JobManager]

serverID uint64
serverIDSession *concurrency.Session
Expand Down Expand Up @@ -2513,7 +2513,7 @@ func (do *Domain) StartTTLJobManager() {
}()

ttlJobManager := ttlworker.NewJobManager(do.ddl.GetID(), do.sysSessionPool, do.store, do.etcdClient)
do.ttlJobManager = ttlJobManager
do.ttlJobManager.Store(ttlJobManager)
ttlJobManager.Start()

<-do.exit
Expand All @@ -2528,7 +2528,7 @@ func (do *Domain) StartTTLJobManager() {

// TTLJobManager returns the ttl job manager on this domain
func (do *Domain) TTLJobManager() *ttlworker.JobManager {
return do.ttlJobManager
return do.ttlJobManager.Load()
}

func init() {
Expand Down
2 changes: 2 additions & 0 deletions kv/key.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,7 @@ type Handle interface {
// IntValue returns the int64 value if IsInt is true, it panics if IsInt returns false.
IntValue() int64
// Next returns the minimum handle that is greater than this handle.
// The returned handle is not guaranteed to be able to decode.
Next() Handle
// Equal returns if the handle equals to another handle, it panics if the types are different.
Equal(h Handle) bool
Expand Down Expand Up @@ -299,6 +300,7 @@ func (*CommonHandle) IntValue() int64 {
}

// Next implements the Handle interface.
// Note that the returned encoded field is not guaranteed to be able to decode.
func (ch *CommonHandle) Next() Handle {
return &CommonHandle{
encoded: Key(ch.encoded).PrefixNext(),
Expand Down
59 changes: 35 additions & 24 deletions meta/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,25 @@ var (
ErrInvalidString = dbterror.ClassMeta.NewStd(errno.ErrInvalidCharacterString)
)

// DDLTableVersion is to display ddl related table versions
type DDLTableVersion int

const (
// InitDDLTableVersion is the original version.
InitDDLTableVersion DDLTableVersion = 0
// BaseDDLTableVersion is for support concurrent DDL, it added tidb_ddl_job, tidb_ddl_reorg and tidb_ddl_history.
BaseDDLTableVersion DDLTableVersion = 1
// MDLTableVersion is for support MDL tables.
MDLTableVersion DDLTableVersion = 2
// BackfillTableVersion is for support distributed reorg stage, it added tidb_ddl_backfill, tidb_ddl_backfill_history.
BackfillTableVersion DDLTableVersion = 3
)

// Bytes returns the byte slice.
func (ver DDLTableVersion) Bytes() []byte {
return []byte(strconv.Itoa(int(ver)))
}

// Meta is for handling meta information in a transaction.
type Meta struct {
txn *structure.TxStructure
Expand Down Expand Up @@ -619,15 +638,25 @@ func (m *Meta) CreateTableOrView(dbID int64, tableInfo *model.TableInfo) error {
}

// SetDDLTables write a key into storage.
func (m *Meta) SetDDLTables() error {
err := m.txn.Set(mDDLTableVersion, []byte("1"))
func (m *Meta) SetDDLTables(ddlTableVersion DDLTableVersion) error {
err := m.txn.Set(mDDLTableVersion, ddlTableVersion.Bytes())
return errors.Trace(err)
}

// SetMDLTables write a key into storage.
func (m *Meta) SetMDLTables() error {
err := m.txn.Set(mDDLTableVersion, []byte("2"))
return errors.Trace(err)
// CheckDDLTableVersion check if the tables related to concurrent DDL exists.
func (m *Meta) CheckDDLTableVersion() (DDLTableVersion, error) {
v, err := m.txn.Get(mDDLTableVersion)
if err != nil {
return -1, errors.Trace(err)
}
if string(v) == "" {
return InitDDLTableVersion, nil
}
ver, err := strconv.Atoi(string(v))
if err != nil {
return -1, errors.Trace(err)
}
return DDLTableVersion(ver), nil
}

// CreateMySQLDatabaseIfNotExists creates mysql schema and return its DB ID.
Expand Down Expand Up @@ -666,24 +695,6 @@ func (m *Meta) GetSystemDBID() (int64, error) {
return 0, nil
}

// CheckDDLTableExists check if the tables related to concurrent DDL exists.
func (m *Meta) CheckDDLTableExists() (bool, error) {
v, err := m.txn.Get(mDDLTableVersion)
if err != nil {
return false, errors.Trace(err)
}
return len(v) != 0, nil
}

// CheckMDLTableExists check if the tables related to concurrent DDL exists.
func (m *Meta) CheckMDLTableExists() (bool, error) {
v, err := m.txn.Get(mDDLTableVersion)
if err != nil {
return false, errors.Trace(err)
}
return bytes.Equal(v, []byte("2")), nil
}

// SetMetadataLock sets the metadata lock.
func (m *Meta) SetMetadataLock(b bool) error {
var data []byte
Expand Down
Loading

0 comments on commit ab62491

Please sign in to comment.