Skip to content

Commit

Permalink
br: fix debug decode backupmeta (#56627)
Browse files Browse the repository at this point in the history
close #56296
  • Loading branch information
Leavrth authored Nov 13, 2024
1 parent 2ad93c2 commit 6b72ec4
Show file tree
Hide file tree
Showing 9 changed files with 798 additions and 13 deletions.
18 changes: 17 additions & 1 deletion br/cmd/br/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -285,6 +285,19 @@ func decodeBackupMetaCommand() *cobra.Command {

fieldName, _ := cmd.Flags().GetString("field")
if fieldName == "" {
if err := metautil.DecodeMetaFile(ctx, s, &cfg.CipherInfo, backupMeta.FileIndex); err != nil {
return errors.Trace(err)
}
if err := metautil.DecodeMetaFile(ctx, s, &cfg.CipherInfo, backupMeta.RawRangeIndex); err != nil {
return errors.Trace(err)
}
if err := metautil.DecodeMetaFile(ctx, s, &cfg.CipherInfo, backupMeta.SchemaIndex); err != nil {
return errors.Trace(err)
}
if err := metautil.DecodeStatsFile(ctx, s, &cfg.CipherInfo, backupMeta.Schemas); err != nil {
return errors.Trace(err)
}

// No field flag, write backupmeta to external storage in JSON format.
backupMetaJSON, err := utils.MarshalBackupMeta(backupMeta)
if err != nil {
Expand All @@ -294,7 +307,7 @@ func decodeBackupMetaCommand() *cobra.Command {
if err != nil {
return errors.Trace(err)
}
cmd.Printf("backupmeta decoded at %s\n", path.Join(cfg.Storage, metautil.MetaJSONFile))
cmd.Printf("backupmeta decoded at %s\n", path.Join(s.URI(), metautil.MetaJSONFile))
return nil
}

Expand Down Expand Up @@ -353,6 +366,9 @@ func encodeBackupMetaCommand() *cobra.Command {
if err != nil {
return errors.Trace(err)
}
if backupMetaJSON.Version == metautil.MetaV2 {
return errors.Errorf("encoding backupmeta v2 is unimplemented")
}
backupMeta, err := proto.Marshal(backupMetaJSON)
if err != nil {
return errors.Trace(err)
Expand Down
5 changes: 4 additions & 1 deletion br/pkg/metautil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "metautil",
srcs = [
"debug.go",
"load.go",
"metafile.go",
"statsfile.go",
Expand Down Expand Up @@ -38,14 +39,15 @@ go_test(
name = "metautil_test",
timeout = "short",
srcs = [
"debug_test.go",
"load_test.go",
"main_test.go",
"metafile_test.go",
"statsfile_test.go",
],
embed = [":metautil"],
flaky = True,
shard_count = 9,
shard_count = 10,
deps = [
"//br/pkg/storage",
"//br/pkg/utils",
Expand All @@ -56,6 +58,7 @@ go_test(
"//pkg/tablecodec",
"//pkg/testkit/testsetup",
"//pkg/util",
"@com_github_gogo_protobuf//proto",
"@com_github_golang_protobuf//proto",
"@com_github_pingcap_kvproto//pkg/brpb",
"@com_github_pingcap_kvproto//pkg/encryptionpb",
Expand Down
134 changes: 134 additions & 0 deletions br/pkg/metautil/debug.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
// Copyright 2024 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.

package metautil

import (
"bytes"
"context"
"crypto/sha256"
"fmt"

"github.com/gogo/protobuf/proto"
"github.com/pingcap/errors"
backuppb "github.com/pingcap/kvproto/pkg/brpb"
berrors "github.com/pingcap/tidb/br/pkg/errors"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/br/pkg/utils"
tidbutil "github.com/pingcap/tidb/pkg/util"
"golang.org/x/sync/errgroup"
)

const (
// JSONFileFormat represents json file name format
JSONFileFormat = "jsons/%s.json"
)

// DecodeStatsFile decodes the stats file to json format, it is called by br debug
func DecodeStatsFile(
ctx context.Context,
s storage.ExternalStorage,
cipher *backuppb.CipherInfo,
schemas []*backuppb.Schema,
) error {
for _, schema := range schemas {
for _, statsIndex := range schema.StatsIndex {
if len(statsIndex.Name) == 0 {
continue
}
content, err := s.ReadFile(ctx, statsIndex.Name)
if err != nil {
return errors.Trace(err)
}
decryptContent, err := utils.Decrypt(content, cipher, statsIndex.CipherIv)
if err != nil {
return errors.Trace(err)
}
checksum := sha256.Sum256(decryptContent)
if !bytes.Equal(statsIndex.Sha256, checksum[:]) {
return berrors.ErrInvalidMetaFile.GenWithStackByArgs(fmt.Sprintf(
"checksum mismatch expect %x, got %x", statsIndex.Sha256, checksum[:]))
}
statsFileBlocks := &backuppb.StatsFile{}
if err := proto.Unmarshal(decryptContent, statsFileBlocks); err != nil {
return errors.Trace(err)
}
jsonContent, err := utils.MarshalStatsFile(statsFileBlocks)
if err != nil {
return errors.Trace(err)
}
if err := s.WriteFile(ctx, fmt.Sprintf(JSONFileFormat, statsIndex.Name), jsonContent); err != nil {
return errors.Trace(err)
}
}
}
return nil
}

// DecodeMetaFile decodes the meta file to json format, it is called by br debug
func DecodeMetaFile(
ctx context.Context,
s storage.ExternalStorage,
cipher *backuppb.CipherInfo,
metaIndex *backuppb.MetaFile,
) error {
if metaIndex == nil {
return nil
}
eg, ectx := errgroup.WithContext(ctx)
workers := tidbutil.NewWorkerPool(8, "download files workers")
for _, node := range metaIndex.MetaFiles {
workers.ApplyOnErrorGroup(eg, func() error {
content, err := s.ReadFile(ectx, node.Name)
if err != nil {
return errors.Trace(err)
}

decryptContent, err := utils.Decrypt(content, cipher, node.CipherIv)
if err != nil {
return errors.Trace(err)
}

checksum := sha256.Sum256(decryptContent)
if !bytes.Equal(node.Sha256, checksum[:]) {
return berrors.ErrInvalidMetaFile.GenWithStackByArgs(fmt.Sprintf(
"checksum mismatch expect %x, got %x", node.Sha256, checksum[:]))
}

child := &backuppb.MetaFile{}
if err = proto.Unmarshal(decryptContent, child); err != nil {
return errors.Trace(err)
}

// the max depth of the root metafile is only 1.
// ASSERT: len(child.MetaFiles) == 0
if len(child.MetaFiles) > 0 {
return errors.Errorf("the metafile has unexpected level: %v", child)
}

jsonContent, err := utils.MarshalMetaFile(child)
if err != nil {
return errors.Trace(err)
}

if err := s.WriteFile(ctx, fmt.Sprintf(JSONFileFormat, node.Name), jsonContent); err != nil {
return errors.Trace(err)
}

err = DecodeStatsFile(ctx, s, cipher, child.Schemas)
return errors.Trace(err)
})
}
return eg.Wait()
}
Loading

0 comments on commit 6b72ec4

Please sign in to comment.