From 403d277e3245b23e26a359700c8bfc9118846f55 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Thu, 2 Feb 2023 22:47:56 +0800 Subject: [PATCH] lightning: fix panic when nextKey twice (#40959) close pingcap/tidb#40934 --- br/pkg/lightning/backend/local/local.go | 8 ++++++-- br/pkg/lightning/backend/local/local_test.go | 15 +++++++++++++-- br/pkg/lightning/backend/local/localhelper.go | 10 +++++++++- .../data/cpeng.a-schema.sql | 2 +- .../lightning_local_backend/data/cpeng.a.1.sql | 2 +- .../lightning_local_backend/data/cpeng.a.2.sql | 2 +- .../lightning_local_backend/data/cpeng.a.3.sql | 2 +- br/tests/lightning_local_backend/run.sh | 5 +++-- kv/key.go | 2 ++ 9 files changed, 37 insertions(+), 11 deletions(-) diff --git a/br/pkg/lightning/backend/local/local.go b/br/pkg/lightning/backend/local/local.go index 05ae60c4497d7..6f0a2e6a130fd 100644 --- a/br/pkg/lightning/backend/local/local.go +++ b/br/pkg/lightning/backend/local/local.go @@ -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) { @@ -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 @@ -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) diff --git a/br/pkg/lightning/backend/local/local_test.go b/br/pkg/lightning/backend/local/local_test.go index 04d63ffb7479b..0294cd7c5f05e 100644 --- a/br/pkg/lightning/backend/local/local_test.go +++ b/br/pkg/lightning/backend/local/local_test.go @@ -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} diff --git a/br/pkg/lightning/backend/local/localhelper.go b/br/pkg/lightning/backend/local/localhelper.go index 8b5819175430b..cd8a27b9bda48 100644 --- a/br/pkg/lightning/backend/local/localhelper.go +++ b/br/pkg/lightning/backend/local/localhelper.go @@ -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" @@ -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") diff --git a/br/tests/lightning_local_backend/data/cpeng.a-schema.sql b/br/tests/lightning_local_backend/data/cpeng.a-schema.sql index fe3f493b6edfd..6c1f5ee154c58 100644 --- a/br/tests/lightning_local_backend/data/cpeng.a-schema.sql +++ b/br/tests/lightning_local_backend/data/cpeng.a-schema.sql @@ -1 +1 @@ -create table a (c int); +create table a (c VARCHAR(20) PRIMARY KEY); diff --git a/br/tests/lightning_local_backend/data/cpeng.a.1.sql b/br/tests/lightning_local_backend/data/cpeng.a.1.sql index 58829b7d87492..a75039e1304e3 100644 --- a/br/tests/lightning_local_backend/data/cpeng.a.1.sql +++ b/br/tests/lightning_local_backend/data/cpeng.a.1.sql @@ -1 +1 @@ -insert into a values (1); +insert into a values ('0000001'); diff --git a/br/tests/lightning_local_backend/data/cpeng.a.2.sql b/br/tests/lightning_local_backend/data/cpeng.a.2.sql index ccbcb5801f72b..a1b15acdecb11 100644 --- a/br/tests/lightning_local_backend/data/cpeng.a.2.sql +++ b/br/tests/lightning_local_backend/data/cpeng.a.2.sql @@ -1 +1 @@ -insert into a values (2); +insert into a values ('0000002'); diff --git a/br/tests/lightning_local_backend/data/cpeng.a.3.sql b/br/tests/lightning_local_backend/data/cpeng.a.3.sql index effdc8f3eab3c..0a457febecf55 100644 --- a/br/tests/lightning_local_backend/data/cpeng.a.3.sql +++ b/br/tests/lightning_local_backend/data/cpeng.a.3.sql @@ -1 +1 @@ -insert into a values (3),(4); +insert into a values ('0000003'),('0000004'); diff --git a/br/tests/lightning_local_backend/run.sh b/br/tests/lightning_local_backend/run.sh index 5177f98d1ecba..d59563637c4f4 100755 --- a/br/tests/lightning_local_backend/run.sh +++ b/br/tests/lightning_local_backend/run.sh @@ -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' diff --git a/kv/key.go b/kv/key.go index 6b5ca28400721..3c243a06ed2eb 100644 --- a/kv/key.go +++ b/kv/key.go @@ -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 @@ -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(),