Skip to content

Commit 4df9aa8

Browse files
authored
br: fix debug decode backupmeta (#56627) (#57891)
close #56296
1 parent 6a147d5 commit 4df9aa8

File tree

8 files changed

+643
-12
lines changed

8 files changed

+643
-12
lines changed

br/cmd/br/debug.go

Lines changed: 14 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -283,6 +283,16 @@ func decodeBackupMetaCommand() *cobra.Command {
283283

284284
fieldName, _ := cmd.Flags().GetString("field")
285285
if fieldName == "" {
286+
if err := utils.DecodeMetaFile(ctx, s, &cfg.CipherInfo, backupMeta.FileIndex); err != nil {
287+
return errors.Trace(err)
288+
}
289+
if err := utils.DecodeMetaFile(ctx, s, &cfg.CipherInfo, backupMeta.RawRangeIndex); err != nil {
290+
return errors.Trace(err)
291+
}
292+
if err := utils.DecodeMetaFile(ctx, s, &cfg.CipherInfo, backupMeta.SchemaIndex); err != nil {
293+
return errors.Trace(err)
294+
}
295+
286296
// No field flag, write backupmeta to external storage in JSON format.
287297
backupMetaJSON, err := utils.MarshalBackupMeta(backupMeta)
288298
if err != nil {
@@ -292,7 +302,7 @@ func decodeBackupMetaCommand() *cobra.Command {
292302
if err != nil {
293303
return errors.Trace(err)
294304
}
295-
cmd.Printf("backupmeta decoded at %s\n", path.Join(cfg.Storage, metautil.MetaJSONFile))
305+
cmd.Printf("backupmeta decoded at %s\n", path.Join(s.URI(), metautil.MetaJSONFile))
296306
return nil
297307
}
298308

@@ -351,6 +361,9 @@ func encodeBackupMetaCommand() *cobra.Command {
351361
if err != nil {
352362
return errors.Trace(err)
353363
}
364+
if backupMetaJSON.Version == metautil.MetaV2 {
365+
return errors.Errorf("encoding backupmeta v2 is unimplemented")
366+
}
354367
backupMeta, err := proto.Marshal(backupMetaJSON)
355368
if err != nil {
356369
return errors.Trace(err)

br/pkg/metautil/metafile.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ const (
3636
// MetaFile represents file name
3737
MetaFile = "backupmeta"
3838
// MetaJSONFile represents backup meta json file name
39-
MetaJSONFile = "backupmeta.json"
39+
MetaJSONFile = "jsons/backupmeta.json"
4040
// MaxBatchSize represents the internal channel buffer size of MetaWriter and MetaReader.
4141
MaxBatchSize = 1024
4242

br/pkg/utils/BUILD.bazel

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ go_library(
66
"backoff.go",
77
"cdc.go",
88
"db.go",
9+
"debug.go",
910
"dyn_pprof_other.go",
1011
"dyn_pprof_unix.go",
1112
"env.go",
@@ -31,6 +32,7 @@ go_library(
3132
"//br/pkg/errors",
3233
"//br/pkg/logutil",
3334
"//br/pkg/metautil",
35+
"//br/pkg/storage",
3436
"//pkg/errno",
3537
"//pkg/kv",
3638
"//pkg/parser/model",
@@ -43,6 +45,7 @@ go_library(
4345
"//pkg/util/sqlexec",
4446
"@com_github_cheggaaa_pb_v3//:pb",
4547
"@com_github_docker_go_units//:go-units",
48+
"@com_github_gogo_protobuf//proto",
4649
"@com_github_google_uuid//:uuid",
4750
"@com_github_pingcap_errors//:errors",
4851
"@com_github_pingcap_failpoint//:failpoint",
@@ -76,6 +79,7 @@ go_test(
7679
"backoff_test.go",
7780
"cdc_test.go",
7881
"db_test.go",
82+
"debug_test.go",
7983
"env_test.go",
8084
"json_test.go",
8185
"key_test.go",
@@ -91,7 +95,7 @@ go_test(
9195
],
9296
embed = [":utils"],
9397
flaky = True,
94-
shard_count = 37,
98+
shard_count = 39,
9599
deps = [
96100
"//br/pkg/errors",
97101
"//br/pkg/metautil",
@@ -107,6 +111,7 @@ go_test(
107111
"//pkg/types",
108112
"//pkg/util/chunk",
109113
"//pkg/util/sqlexec",
114+
"@com_github_gogo_protobuf//proto",
110115
"@com_github_golang_protobuf//proto",
111116
"@com_github_pingcap_errors//:errors",
112117
"@com_github_pingcap_failpoint//:failpoint",

br/pkg/utils/debug.go

Lines changed: 91 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,91 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package utils
16+
17+
import (
18+
"bytes"
19+
"context"
20+
"crypto/sha256"
21+
"fmt"
22+
23+
"github.com/gogo/protobuf/proto"
24+
"github.com/pingcap/errors"
25+
backuppb "github.com/pingcap/kvproto/pkg/brpb"
26+
berrors "github.com/pingcap/tidb/br/pkg/errors"
27+
"github.com/pingcap/tidb/br/pkg/metautil"
28+
"github.com/pingcap/tidb/br/pkg/storage"
29+
"golang.org/x/sync/errgroup"
30+
)
31+
32+
const (
33+
// JSONFileFormat represents json file name format
34+
JSONFileFormat = "jsons/%s.json"
35+
)
36+
37+
// DecodeMetaFile decodes the meta file to json format, it is called by br debug
38+
func DecodeMetaFile(
39+
ctx context.Context,
40+
s storage.ExternalStorage,
41+
cipher *backuppb.CipherInfo,
42+
metaIndex *backuppb.MetaFile,
43+
) error {
44+
if metaIndex == nil {
45+
return nil
46+
}
47+
eg, ectx := errgroup.WithContext(ctx)
48+
workers := NewWorkerPool(8, "download files workers")
49+
for _, node := range metaIndex.MetaFiles {
50+
workers.ApplyOnErrorGroup(eg, func() error {
51+
content, err := s.ReadFile(ectx, node.Name)
52+
if err != nil {
53+
return errors.Trace(err)
54+
}
55+
56+
decryptContent, err := metautil.Decrypt(content, cipher, node.CipherIv)
57+
if err != nil {
58+
return errors.Trace(err)
59+
}
60+
61+
checksum := sha256.Sum256(decryptContent)
62+
if !bytes.Equal(node.Sha256, checksum[:]) {
63+
return berrors.ErrInvalidMetaFile.GenWithStackByArgs(fmt.Sprintf(
64+
"checksum mismatch expect %x, got %x", node.Sha256, checksum[:]))
65+
}
66+
67+
child := &backuppb.MetaFile{}
68+
if err = proto.Unmarshal(decryptContent, child); err != nil {
69+
return errors.Trace(err)
70+
}
71+
72+
// the max depth of the root metafile is only 1.
73+
// ASSERT: len(child.MetaFiles) == 0
74+
if len(child.MetaFiles) > 0 {
75+
return errors.Errorf("the metafile has unexpected level: %v", child)
76+
}
77+
78+
jsonContent, err := MarshalMetaFile(child)
79+
if err != nil {
80+
return errors.Trace(err)
81+
}
82+
83+
if err := s.WriteFile(ctx, fmt.Sprintf(JSONFileFormat, node.Name), jsonContent); err != nil {
84+
return errors.Trace(err)
85+
}
86+
87+
return errors.Trace(err)
88+
})
89+
}
90+
return eg.Wait()
91+
}

br/pkg/utils/debug_test.go

Lines changed: 183 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,183 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package utils_test
16+
17+
import (
18+
"context"
19+
"crypto/sha256"
20+
"fmt"
21+
"math/rand"
22+
"testing"
23+
24+
"github.com/gogo/protobuf/proto"
25+
backuppb "github.com/pingcap/kvproto/pkg/brpb"
26+
"github.com/pingcap/tidb/br/pkg/metautil"
27+
"github.com/pingcap/tidb/br/pkg/storage"
28+
"github.com/pingcap/tidb/br/pkg/utils"
29+
"github.com/stretchr/testify/require"
30+
)
31+
32+
func flushMetaFile(
33+
ctx context.Context,
34+
t *testing.T,
35+
fname string,
36+
metaFile *backuppb.MetaFile,
37+
storage storage.ExternalStorage,
38+
cipher *backuppb.CipherInfo,
39+
) *backuppb.File {
40+
content, err := metaFile.Marshal()
41+
require.NoError(t, err)
42+
43+
encyptedContent, iv, err := metautil.Encrypt(content, cipher)
44+
require.NoError(t, err)
45+
46+
err = storage.WriteFile(ctx, fname, encyptedContent)
47+
require.NoError(t, err)
48+
49+
checksum := sha256.Sum256(content)
50+
file := &backuppb.File{
51+
Name: fname,
52+
Sha256: checksum[:],
53+
Size_: uint64(len(content)),
54+
CipherIv: iv,
55+
}
56+
57+
return file
58+
}
59+
60+
func flushStatsFile(
61+
ctx context.Context,
62+
t *testing.T,
63+
fname string,
64+
statsFile *backuppb.StatsFile,
65+
storage storage.ExternalStorage,
66+
cipher *backuppb.CipherInfo,
67+
) *backuppb.StatsFileIndex {
68+
content, err := proto.Marshal(statsFile)
69+
require.NoError(t, err)
70+
71+
checksum := sha256.Sum256(content)
72+
sizeOri := uint64(len(content))
73+
encryptedContent, iv, err := metautil.Encrypt(content, cipher)
74+
require.NoError(t, err)
75+
76+
err = storage.WriteFile(ctx, fname, encryptedContent)
77+
require.NoError(t, err)
78+
79+
return &backuppb.StatsFileIndex{
80+
Name: fname,
81+
Sha256: checksum[:],
82+
SizeEnc: uint64(len(encryptedContent)),
83+
SizeOri: sizeOri,
84+
CipherIv: iv,
85+
InlineData: []byte(fmt.Sprintf("%d", rand.Int())),
86+
}
87+
}
88+
89+
func TestDecodeMetaFile(t *testing.T) {
90+
ctx := context.Background()
91+
base := t.TempDir()
92+
s, err := storage.NewLocalStorage(base)
93+
require.NoError(t, err)
94+
cipher := &backuppb.CipherInfo{CipherType: 1}
95+
file1 := flushMetaFile(ctx, t, "data", &backuppb.MetaFile{
96+
DataFiles: []*backuppb.File{
97+
{
98+
Name: "1.sst",
99+
Sha256: []byte("1.sst"),
100+
StartKey: []byte("start"),
101+
EndKey: []byte("end"),
102+
EndVersion: 1,
103+
Crc64Xor: 1,
104+
TotalKvs: 2,
105+
TotalBytes: 3,
106+
Cf: "write",
107+
CipherIv: []byte("1.sst"),
108+
},
109+
},
110+
}, s, cipher)
111+
stats := flushStatsFile(ctx, t, "stats", &backuppb.StatsFile{Blocks: []*backuppb.StatsBlock{
112+
{
113+
PhysicalId: 1,
114+
JsonTable: []byte("1"),
115+
},
116+
{
117+
PhysicalId: 2,
118+
JsonTable: []byte("2"),
119+
},
120+
}}, s, cipher)
121+
metaFile2 := &backuppb.MetaFile{
122+
Schemas: []*backuppb.Schema{
123+
{
124+
Db: []byte(`{"db_name":{"L":"test","O":"test"},"id":1,"state":5}`),
125+
Table: []byte(`{"id":2,"state":5}`),
126+
Crc64Xor: 1,
127+
TotalKvs: 2,
128+
TotalBytes: 3,
129+
TiflashReplicas: 4,
130+
Stats: []byte(`{"a":1}`),
131+
StatsIndex: []*backuppb.StatsFileIndex{stats},
132+
},
133+
},
134+
}
135+
file2 := flushMetaFile(ctx, t, "schema", metaFile2, s, cipher)
136+
137+
{
138+
err = utils.DecodeMetaFile(ctx, s, cipher, &backuppb.MetaFile{MetaFiles: []*backuppb.File{file1}})
139+
require.NoError(t, err)
140+
content, err := s.ReadFile(ctx, "jsons/data.json")
141+
require.NoError(t, err)
142+
metaFile, err := utils.UnmarshalMetaFile(content)
143+
require.NoError(t, err)
144+
require.Equal(t, 1, len(metaFile.DataFiles))
145+
require.Equal(t, "1.sst", metaFile.DataFiles[0].Name)
146+
require.Equal(t, []byte("1.sst"), metaFile.DataFiles[0].Sha256)
147+
require.Equal(t, []byte("start"), metaFile.DataFiles[0].StartKey)
148+
require.Equal(t, []byte("end"), metaFile.DataFiles[0].EndKey)
149+
require.Equal(t, uint64(1), metaFile.DataFiles[0].EndVersion)
150+
require.Equal(t, uint64(1), metaFile.DataFiles[0].Crc64Xor)
151+
require.Equal(t, uint64(2), metaFile.DataFiles[0].TotalKvs)
152+
require.Equal(t, uint64(3), metaFile.DataFiles[0].TotalBytes)
153+
require.Equal(t, "write", metaFile.DataFiles[0].Cf)
154+
require.Equal(t, []byte("1.sst"), metaFile.DataFiles[0].CipherIv)
155+
}
156+
157+
{
158+
err = utils.DecodeMetaFile(ctx, s, cipher, &backuppb.MetaFile{MetaFiles: []*backuppb.File{file2}})
159+
require.NoError(t, err)
160+
{
161+
content, err := s.ReadFile(ctx, "jsons/schema.json")
162+
require.NoError(t, err)
163+
metaFile, err := utils.UnmarshalMetaFile(content)
164+
require.NoError(t, err)
165+
require.Equal(t, 1, len(metaFile.Schemas))
166+
require.Equal(t, metaFile2.Schemas[0].Db, metaFile.Schemas[0].Db)
167+
require.Equal(t, metaFile2.Schemas[0].Table, metaFile.Schemas[0].Table)
168+
require.Equal(t, uint64(1), metaFile.Schemas[0].Crc64Xor)
169+
require.Equal(t, uint64(2), metaFile.Schemas[0].TotalKvs)
170+
require.Equal(t, uint64(3), metaFile.Schemas[0].TotalBytes)
171+
require.Equal(t, uint32(4), metaFile.Schemas[0].TiflashReplicas)
172+
require.Equal(t, metaFile2.Schemas[0].Stats, metaFile.Schemas[0].Stats)
173+
statsIndex := metaFile.Schemas[0].StatsIndex
174+
require.Equal(t, 1, len(statsIndex))
175+
require.Equal(t, stats.Name, statsIndex[0].Name)
176+
require.Equal(t, stats.Sha256, statsIndex[0].Sha256)
177+
require.Equal(t, stats.SizeEnc, statsIndex[0].SizeEnc)
178+
require.Equal(t, stats.SizeOri, statsIndex[0].SizeOri)
179+
require.Equal(t, stats.CipherIv, statsIndex[0].CipherIv)
180+
require.Equal(t, stats.InlineData, statsIndex[0].InlineData)
181+
}
182+
}
183+
}

0 commit comments

Comments
 (0)