2 Star 2 Fork 1

cockroachdb/cockroach

加入 Gitee
与超过 1200万 开发者一起发现、参与优秀开源项目,私有仓库也完全免费 :)
免费加入
克隆/下载
backup.go 27.35 KB
一键复制 编辑 原始数据 按行查看 历史
123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896
// Copyright 2016 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/LICENSE
package sqlccl
import (
"bytes"
"io/ioutil"
"sort"
"time"
"github.com/pkg/errors"
"golang.org/x/net/context"
"golang.org/x/sync/errgroup"
"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl/intervalccl"
"github.com/cockroachdb/cockroach/pkg/gossip"
"github.com/cockroachdb/cockroach/pkg/internal/client"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/jobs"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/interval"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
)
const (
// BackupDescriptorName is the file name used for serialized
// BackupDescriptor protos.
BackupDescriptorName = "BACKUP"
// BackupDescriptorCheckpointName is the file name used to store the
// serialized BackupDescriptor proto while the backup is in progress.
BackupDescriptorCheckpointName = "BACKUP-CHECKPOINT"
// BackupFormatInitialVersion is the first version of backup and its files.
BackupFormatInitialVersion uint32 = 0
)
// BackupCheckpointInterval is the interval at which backup progress is saved
// to durable storage.
var BackupCheckpointInterval = time.Minute
// BackupImplicitSQLDescriptors are descriptors for tables that are implicitly
// included in every backup, plus their parent database descriptors.
var BackupImplicitSQLDescriptors = []sqlbase.Descriptor{
*sqlbase.WrapDescriptor(&sqlbase.SystemDB),
*sqlbase.WrapDescriptor(&sqlbase.DescriptorTable),
*sqlbase.WrapDescriptor(&sqlbase.UsersTable),
}
// exportStorageFromURI returns an ExportStorage for the given URI.
func exportStorageFromURI(ctx context.Context, uri string) (storageccl.ExportStorage, error) {
conf, err := storageccl.ExportStorageConfFromURI(uri)
if err != nil {
return nil, err
}
return storageccl.MakeExportStorage(ctx, conf)
}
// readBackupDescriptorFromURI creates an export store from the given URI, then
// reads and unmarshals a BackupDescriptor at the standard location in the
// export storage.
func readBackupDescriptorFromURI(ctx context.Context, uri string) (BackupDescriptor, error) {
exportStore, err := exportStorageFromURI(ctx, uri)
if err != nil {
return BackupDescriptor{}, err
}
defer exportStore.Close()
backupDesc, err := readBackupDescriptor(ctx, exportStore, BackupDescriptorName)
if err != nil {
return BackupDescriptor{}, err
}
backupDesc.Dir = exportStore.Conf()
// TODO(dan): Sanity check this BackupDescriptor: non-empty EndTime,
// non-empty Paths, and non-overlapping Spans and keyranges in Files.
return backupDesc, nil
}
// readBackupDescriptor reads and unmarshals a BackupDescriptor from filename in
// the provided export store.
func readBackupDescriptor(
ctx context.Context, exportStore storageccl.ExportStorage, filename string,
) (BackupDescriptor, error) {
r, err := exportStore.ReadFile(ctx, filename)
if err != nil {
return BackupDescriptor{}, err
}
defer r.Close()
descBytes, err := ioutil.ReadAll(r)
if err != nil {
return BackupDescriptor{}, err
}
var backupDesc BackupDescriptor
if err := backupDesc.Unmarshal(descBytes); err != nil {
return BackupDescriptor{}, err
}
return backupDesc, err
}
// ValidatePreviousBackups checks that the timestamps of previous backups are
// consistent. The most recently backed-up time is returned.
func ValidatePreviousBackups(
ctx context.Context, uris []string, spans []roachpb.Span,
) (hlc.Timestamp, error) {
if len(uris) == 0 || len(uris) == 1 && uris[0] == "" {
// Full backup.
return hlc.Timestamp{}, nil
}
backups := make([]BackupDescriptor, len(uris))
for i, uri := range uris {
desc, err := readBackupDescriptorFromURI(ctx, uri)
if err != nil {
return hlc.Timestamp{}, errors.Wrapf(err, "failed to read backup from %q", uri)
}
backups[i] = desc
}
// This reuses Restore's logic for lining up all the start and end
// timestamps to validate the previous backups that this one is incremental
// from.
lowWaterMark := keys.MinKey
_, endTime, err := makeImportSpans(spans, backups, lowWaterMark)
return endTime, errors.Wrap(err, "invalid previous backups (a new full backup may be required if a table has been created, dropped or truncated)")
}
func allSQLDescriptors(ctx context.Context, txn *client.Txn) ([]sqlbase.Descriptor, error) {
startKey := roachpb.Key(keys.MakeTablePrefix(keys.DescriptorTableID))
endKey := startKey.PrefixEnd()
rows, err := txn.Scan(ctx, startKey, endKey, 0)
if err != nil {
// NB: Don't wrap this error, as wrapped HandledRetryableTxnErrors are not
// automatically retried by db.Txn.
//
// TODO(benesch): teach the KV layer to use errors.Cause.
return nil, err
}
sqlDescs := make([]sqlbase.Descriptor, len(rows))
for i, row := range rows {
if err := row.ValueProto(&sqlDescs[i]); err != nil {
return nil, errors.Wrapf(err, "%s: unable to unmarshal SQL descriptor", row.Key)
}
}
return sqlDescs, nil
}
func allRangeDescriptors(ctx context.Context, txn *client.Txn) ([]roachpb.RangeDescriptor, error) {
rows, err := txn.Scan(ctx, keys.Meta2Prefix, keys.MetaMax, 0)
if err != nil {
// NB: Don't wrap this error, as wrapped HandledRetryableTxnErrors are not
// automatically retried by db.Txn.
//
// TODO(benesch): teach the KV layer to use errors.Cause.
return nil, err
}
rangeDescs := make([]roachpb.RangeDescriptor, len(rows))
for i, row := range rows {
if err := row.ValueProto(&rangeDescs[i]); err != nil {
return nil, errors.Wrapf(err, "%s: unable to unmarshal range descriptor", row.Key)
}
}
return rangeDescs, nil
}
// spansForAllTableIndexes returns non-overlapping spans for every index and
// table passed in. They would normally overlap if any of them are interleaved.
func spansForAllTableIndexes(tables []*sqlbase.TableDescriptor) []roachpb.Span {
sstIntervalTree := interval.NewTree(interval.ExclusiveOverlapper)
for _, table := range tables {
for _, index := range table.AllNonDropIndexes() {
if err := sstIntervalTree.Insert(intervalSpan(table.IndexSpan(index.ID)), false); err != nil {
panic(errors.Wrap(err, "IndexSpan"))
}
}
}
var spans []roachpb.Span
_ = sstIntervalTree.Do(func(r interval.Interface) bool {
spans = append(spans, roachpb.Span{
Key: roachpb.Key(r.Range().Start),
EndKey: roachpb.Key(r.Range().End),
})
return false
})
return spans
}
// coveringFromSpans creates an intervalccl.Covering with a fixed payload from a
// slice of roachpb.Spans.
func coveringFromSpans(spans []roachpb.Span, payload interface{}) intervalccl.Covering {
var covering intervalccl.Covering
for _, span := range spans {
covering = append(covering, intervalccl.Range{
Start: []byte(span.Key),
End: []byte(span.EndKey),
Payload: payload,
})
}
return covering
}
// splitAndFilterSpans returns the spans that represent the set difference
// (includes - excludes) while also guaranteeing that each output span does not
// cross the endpoint of a RangeDescriptor in ranges.
func splitAndFilterSpans(
includes []roachpb.Span, excludes []roachpb.Span, ranges []roachpb.RangeDescriptor,
) []roachpb.Span {
type includeMarker struct{}
type excludeMarker struct{}
includeCovering := coveringFromSpans(includes, includeMarker{})
excludeCovering := coveringFromSpans(excludes, excludeMarker{})
var rangeCovering intervalccl.Covering
for _, rangeDesc := range ranges {
rangeCovering = append(rangeCovering, intervalccl.Range{
Start: []byte(rangeDesc.StartKey),
End: []byte(rangeDesc.EndKey),
})
}
splits := intervalccl.OverlapCoveringMerge(
[]intervalccl.Covering{includeCovering, excludeCovering, rangeCovering},
)
var out []roachpb.Span
for _, split := range splits {
include := false
exclude := false
for _, payload := range split.Payload.([]interface{}) {
switch payload.(type) {
case includeMarker:
include = true
case excludeMarker:
exclude = true
}
}
if include && !exclude {
out = append(out, roachpb.Span{
Key: roachpb.Key(split.Start),
EndKey: roachpb.Key(split.End),
})
}
}
return out
}
func backupJobDescription(
backup *parser.Backup, to string, incrementalFrom []string,
) (string, error) {
b := &parser.Backup{
AsOf: backup.AsOf,
Options: backup.Options,
Targets: backup.Targets,
}
to, err := storageccl.SanitizeExportStorageURI(to)
if err != nil {
return "", err
}
b.To = parser.NewDString(to)
for _, from := range incrementalFrom {
sanitizedFrom, err := storageccl.SanitizeExportStorageURI(from)
if err != nil {
return "", err
}
b.IncrementalFrom = append(b.IncrementalFrom, parser.NewDString(sanitizedFrom))
}
return parser.AsStringWithFlags(b, parser.FmtSimpleQualified), nil
}
// clusterNodeCount returns the approximate number of nodes in the cluster.
func clusterNodeCount(g *gossip.Gossip) int {
var nodes int
for k := range g.GetInfoStatus().Infos {
if gossip.IsNodeIDKey(k) {
nodes++
}
}
return nodes
}
type backupFileDescriptors []BackupDescriptor_File
func (r backupFileDescriptors) Len() int { return len(r) }
func (r backupFileDescriptors) Swap(i, j int) { r[i], r[j] = r[j], r[i] }
func (r backupFileDescriptors) Less(i, j int) bool {
if cmp := bytes.Compare(r[i].Span.Key, r[j].Span.Key); cmp != 0 {
return cmp < 0
}
return bytes.Compare(r[i].Span.EndKey, r[j].Span.EndKey) < 0
}
func writeBackupDescriptor(
ctx context.Context,
exportStore storageccl.ExportStorage,
filename string,
desc *BackupDescriptor,
) error {
sort.Sort(backupFileDescriptors(desc.Files))
descBuf, err := desc.Marshal()
if err != nil {
return err
}
if err := exportStore.WriteFile(ctx, filename, bytes.NewReader(descBuf)); err != nil {
return err
}
return nil
}
func resolveTargetsToDescriptors(
ctx context.Context, p sql.PlanHookState, endTime hlc.Timestamp, targets parser.TargetList,
) ([]sqlbase.Descriptor, error) {
var err error
var sqlDescs []sqlbase.Descriptor
db := p.ExecCfg().DB
{
// TODO(andrei): Plumb a gatewayNodeID in here and also find a way to
// express that whatever this txn does should not count towards lease
// placement stats.
txn := client.NewTxn(db, 0 /* gatewayNodeID */)
opt := client.TxnExecOptions{AutoRetry: true, AutoCommit: true}
err := txn.Exec(ctx, opt, func(ctx context.Context, txn *client.Txn, opt *client.TxnExecOptions) error {
var err error
txn.SetFixedTimestamp(ctx, endTime)
sqlDescs, err = allSQLDescriptors(ctx, txn)
return err
})
if err != nil {
return nil, err
}
}
sessionDatabase := p.EvalContext().Database
if sqlDescs, err = descriptorsMatchingTargets(sessionDatabase, sqlDescs, targets); err != nil {
return nil, err
}
sqlDescs = append(sqlDescs, BackupImplicitSQLDescriptors...)
// Dedupe. Duplicate descriptors will cause restore to fail.
{
descsByID := make(map[sqlbase.ID]sqlbase.Descriptor)
for _, sqlDesc := range sqlDescs {
descsByID[sqlDesc.GetID()] = sqlDesc
}
sqlDescs = sqlDescs[:0]
for _, sqlDesc := range descsByID {
sqlDescs = append(sqlDescs, sqlDesc)
}
}
// Ensure interleaved tables appear after their parent. Since parents must be
// created before their children, simply sorting by ID accomplishes this.
sort.Slice(sqlDescs, func(i, j int) bool { return sqlDescs[i].GetID() < sqlDescs[j].GetID() })
return sqlDescs, nil
}
// backup exports a snapshot of every kv entry into ranged sstables.
//
// The output is an sstable per range with files in the following locations:
// - <dir>/<unique_int>.sst
// - <dir> is given by the user and may be cloud storage
// - Each file contains data for a key range that doesn't overlap with any other
// file.
func backup(
ctx context.Context,
db *client.DB,
gossip *gossip.Gossip,
exportStore storageccl.ExportStorage,
job *jobs.Job,
backupDesc *BackupDescriptor,
checkpointDesc *BackupDescriptor,
) error {
// TODO(dan): Figure out how permissions should work. #6713 is tracking this
// for grpc.
mu := struct {
syncutil.Mutex
files []BackupDescriptor_File
exported roachpb.BulkOpSummary
lastCheckpoint time.Time
checkpointed bool
}{}
var checkpointMu syncutil.Mutex
var ranges []roachpb.RangeDescriptor
if err := db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
var err error
// TODO(benesch): limit the range descriptors we fetch to the ranges that
// are actually relevant in the backup to speed up small backups on large
// clusters.
ranges, err = allRangeDescriptors(ctx, txn)
return err
}); err != nil {
return errors.Wrap(err, "fetching range descriptors")
}
var completedSpans []roachpb.Span
if checkpointDesc != nil {
// TODO(benesch): verify these files, rather than accepting them as truth
// blindly.
// No concurrency yet, so these assignments are safe.
mu.checkpointed = true
mu.files = checkpointDesc.Files
mu.exported = checkpointDesc.EntryCounts
for _, file := range checkpointDesc.Files {
completedSpans = append(completedSpans, file.Span)
}
}
// Subtract out any completed spans and split the remaining spans into
// range-sized pieces so that we can use the number of completed requests as a
// rough measure of progress.
spans := splitAndFilterSpans(backupDesc.Spans, completedSpans, ranges)
progressLogger := jobProgressLogger{
job: job,
totalChunks: len(spans),
startFraction: job.Payload().FractionCompleted,
}
ctx, cancel := context.WithCancel(ctx)
if err := job.Created(ctx, cancel); err != nil {
return err
}
if err := job.Started(ctx); err != nil {
return err
}
// We're already limiting these on the server-side, but sending all the
// Export requests at once would fill up distsender/grpc/something and cause
// all sorts of badness (node liveness timeouts leading to mass leaseholder
// transfers, poor performance on SQL workloads, etc) as well as log spam
// about slow distsender requests. Rate limit them here, too.
//
// Each node limits the number of running Export & Import requests it serves
// to avoid overloading the network, so multiply that by the number of nodes
// in the cluster and use that as the number of outstanding Export requests
// for the rate limiting. This attempts to strike a balance between
// simplicity, not getting slow distsender log spam, and keeping the server
// side limiter full.
//
// TODO(dan): Make this limiting per node.
//
// TODO(dan): See if there's some better solution than rate-limiting #14798.
maxConcurrentExports := clusterNodeCount(gossip) * storageccl.ExportRequestLimit
exportsSem := make(chan struct{}, maxConcurrentExports)
header := roachpb.Header{Timestamp: backupDesc.EndTime}
g, gCtx := errgroup.WithContext(ctx)
requestFinishedCh := make(chan struct{}, len(spans)) // enough buffer to never block
g.Go(func() error {
return progressLogger.loop(gCtx, requestFinishedCh)
})
for i := range spans {
select {
case exportsSem <- struct{}{}:
case <-ctx.Done():
return ctx.Err()
}
span := spans[i]
g.Go(func() error {
defer func() { <-exportsSem }()
req := &roachpb.ExportRequest{
Span: span,
Storage: exportStore.Conf(),
StartTime: backupDesc.StartTime,
}
res, pErr := client.SendWrappedWith(gCtx, db.GetSender(), header, req)
if pErr != nil {
return pErr.GoError()
}
mu.Lock()
for _, file := range res.(*roachpb.ExportResponse).Files {
mu.files = append(mu.files, BackupDescriptor_File{
Span: file.Span,
Path: file.Path,
Sha512: file.Sha512,
EntryCounts: file.Exported,
})
mu.exported.Add(file.Exported)
}
var checkpointFiles backupFileDescriptors
if timeutil.Since(mu.lastCheckpoint) > BackupCheckpointInterval {
// We optimistically assume the checkpoint will succeed to prevent
// multiple threads from attempting to checkpoint.
mu.lastCheckpoint = timeutil.Now()
checkpointFiles = append(checkpointFiles, mu.files...)
}
mu.Unlock()
requestFinishedCh <- struct{}{}
if checkpointFiles != nil {
checkpointMu.Lock()
backupDesc.Files = checkpointFiles
err := writeBackupDescriptor(
ctx, exportStore, BackupDescriptorCheckpointName, backupDesc,
)
checkpointMu.Unlock()
if err != nil {
log.Errorf(ctx, "unable to checkpoint backup descriptor: %+v", err)
mu.Lock()
mu.checkpointed = true
mu.Unlock()
}
}
return nil
})
}
if err := g.Wait(); err != nil {
return errors.Wrapf(err, "exporting %d ranges", len(spans))
}
// No more concurrency, so no need to acquire locks below.
backupDesc.Files = mu.files
backupDesc.EntryCounts = mu.exported
if err := writeBackupDescriptor(ctx, exportStore, BackupDescriptorName, backupDesc); err != nil {
return err
}
if mu.checkpointed {
if err := exportStore.Delete(ctx, BackupDescriptorCheckpointName); err != nil {
log.Warningf(ctx, "unable to delete checkpointed backup descriptor: %+v", err)
}
}
return nil
}
func backupPlanHook(
stmt parser.Statement, p sql.PlanHookState,
) (func(context.Context, chan<- parser.Datums) error, sqlbase.ResultColumns, error) {
backupStmt, ok := stmt.(*parser.Backup)
if !ok {
return nil, nil, nil
}
if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().ClusterID(), p.ExecCfg().Organization(), "BACKUP",
); err != nil {
return nil, nil, err
}
if err := p.RequireSuperUser("BACKUP"); err != nil {
return nil, nil, err
}
toFn, err := p.TypeAsString(backupStmt.To, "BACKUP")
if err != nil {
return nil, nil, err
}
incrementalFromFn, err := p.TypeAsStringArray(backupStmt.IncrementalFrom, "BACKUP")
if err != nil {
return nil, nil, err
}
header := sqlbase.ResultColumns{
{Name: "job_id", Typ: parser.TypeInt},
{Name: "status", Typ: parser.TypeString},
{Name: "fraction_completed", Typ: parser.TypeFloat},
{Name: "rows", Typ: parser.TypeInt},
{Name: "index_entries", Typ: parser.TypeInt},
{Name: "system_records", Typ: parser.TypeInt},
{Name: "bytes", Typ: parser.TypeInt},
}
fn := func(ctx context.Context, resultsCh chan<- parser.Datums) error {
// TODO(dan): Move this span into sql.
ctx, span := tracing.ChildSpan(ctx, stmt.StatementTag())
defer tracing.FinishSpan(span)
if err := backupStmt.Targets.NormalizeTablesWithDatabase(p.EvalContext().Database); err != nil {
return err
}
to, err := toFn()
if err != nil {
return err
}
incrementalFrom, err := incrementalFromFn()
if err != nil {
return err
}
endTime := p.ExecCfg().Clock.Now()
if backupStmt.AsOf.Expr != nil {
var err error
if endTime, err = sql.EvalAsOfTimestamp(nil, backupStmt.AsOf, endTime); err != nil {
return err
}
}
exportStore, err := exportStorageFromURI(ctx, to)
if err != nil {
return err
}
defer exportStore.Close()
// Ensure there isn't already a readable backup desc.
{
r, err := exportStore.ReadFile(ctx, BackupDescriptorName)
// TODO(dt): If we audit exactly what not-exists error each ExportStorage
// returns (and then wrap/tag them), we could narrow this check.
if err == nil {
r.Close()
return errors.Errorf("a %s file already appears to exist in %s",
BackupDescriptorName, to)
}
}
targetDescs, err := resolveTargetsToDescriptors(ctx, p, endTime, backupStmt.Targets)
if err != nil {
return err
}
var tables []*sqlbase.TableDescriptor
for _, desc := range targetDescs {
if dbDesc := desc.GetDatabase(); dbDesc != nil {
if err := p.CheckPrivilege(dbDesc, privilege.SELECT); err != nil {
return err
}
}
if tableDesc := desc.GetTable(); tableDesc != nil {
if err := p.CheckPrivilege(tableDesc, privilege.SELECT); err != nil {
return err
}
tables = append(tables, tableDesc)
}
}
spans := spansForAllTableIndexes(tables)
var startTime hlc.Timestamp
if backupStmt.IncrementalFrom != nil {
var err error
startTime, err = ValidatePreviousBackups(ctx, incrementalFrom, spans)
if err != nil {
return err
}
}
backupDesc := BackupDescriptor{
StartTime: startTime,
EndTime: endTime,
Descriptors: targetDescs,
Spans: spans,
FormatVersion: BackupFormatInitialVersion,
BuildInfo: build.GetInfo(),
NodeID: p.ExecCfg().NodeID.Get(),
ClusterID: p.ExecCfg().ClusterID(),
}
description, err := backupJobDescription(backupStmt, to, incrementalFrom)
if err != nil {
return err
}
job := p.ExecCfg().JobRegistry.NewJob(jobs.Record{
Description: description,
Username: p.User(),
DescriptorIDs: func() (sqlDescIDs []sqlbase.ID) {
for _, sqlDesc := range backupDesc.Descriptors {
sqlDescIDs = append(sqlDescIDs, sqlDesc.GetID())
}
return sqlDescIDs
}(),
Details: jobs.BackupDetails{
StartTime: startTime,
EndTime: endTime,
URI: to,
},
})
var checkpointDesc *BackupDescriptor
backupErr := backup(ctx,
p.ExecCfg().DB,
p.ExecCfg().Gossip,
exportStore,
job,
&backupDesc,
checkpointDesc,
)
if err := job.FinishedWith(ctx, backupErr); err != nil {
return err
}
if backupErr != nil {
return backupErr
}
// TODO(benesch): emit periodic progress updates.
resultsCh <- parser.Datums{
parser.NewDInt(parser.DInt(*job.ID())),
parser.NewDString(string(jobs.StatusSucceeded)),
parser.NewDFloat(parser.DFloat(1.0)),
parser.NewDInt(parser.DInt(backupDesc.EntryCounts.Rows)),
parser.NewDInt(parser.DInt(backupDesc.EntryCounts.IndexEntries)),
parser.NewDInt(parser.DInt(backupDesc.EntryCounts.SystemRecords)),
parser.NewDInt(parser.DInt(backupDesc.EntryCounts.DataSize)),
}
return nil
}
return fn, header, nil
}
func backupResumeHook(typ jobs.Type) func(context.Context, *jobs.Job) error {
if typ != jobs.TypeBackup {
return nil
}
return func(ctx context.Context, job *jobs.Job) error {
details := job.Record.Details.(jobs.BackupDetails)
var sqlDescs []sqlbase.Descriptor
var tables []*sqlbase.TableDescriptor
{
// TODO(andrei): Plumb a gatewayNodeID in here and also find a way to
// express that whatever this txn does should not count towards lease
// placement stats.
txn := client.NewTxn(job.DB(), 0 /* gatewayNodeID */)
opt := client.TxnExecOptions{AutoRetry: true, AutoCommit: true}
if err := txn.Exec(ctx, opt, func(ctx context.Context, txn *client.Txn, opt *client.TxnExecOptions) error {
txn.SetFixedTimestamp(ctx, details.EndTime)
for _, sqlDescID := range job.Payload().DescriptorIDs {
desc := &sqlbase.Descriptor{}
descKey := sqlbase.MakeDescMetadataKey(sqlDescID)
if err := txn.GetProto(ctx, descKey, desc); err != nil {
return err
}
sqlDescs = append(sqlDescs, *desc)
if tableDesc := desc.GetTable(); tableDesc != nil {
tables = append(tables, tableDesc)
}
}
return nil
}); err != nil {
return err
}
}
backupDesc := BackupDescriptor{
StartTime: details.StartTime,
EndTime: details.EndTime,
Descriptors: sqlDescs,
Spans: spansForAllTableIndexes(tables),
FormatVersion: BackupFormatInitialVersion,
BuildInfo: build.GetInfo(),
NodeID: job.NodeID(),
ClusterID: job.ClusterID(),
}
conf, err := storageccl.ExportStorageConfFromURI(details.URI)
if err != nil {
return err
}
exportStore, err := storageccl.MakeExportStorage(ctx, conf)
if err != nil {
return nil
}
var checkpointDesc *BackupDescriptor
if desc, err := readBackupDescriptor(ctx, exportStore, BackupDescriptorCheckpointName); err == nil {
checkpointDesc = &desc
} else {
// TODO(benesch): distinguish between a missing checkpoint, which simply
// indicates the prior backup attempt made no progress, and a corrupted
// checkpoint, which is more troubling. Sadly, storageccl doesn't provide a
// "not found" error that's consistent across all ExportStorage
// implementations.
log.Warningf(ctx, "unable to load backup checkpoint while resuming job %d: %v", *job.ID(), err)
}
return backup(ctx, job.DB(), job.Gossip(), exportStore, job, &backupDesc, checkpointDesc)
}
}
func showBackupPlanHook(
stmt parser.Statement, p sql.PlanHookState,
) (func(context.Context, chan<- parser.Datums) error, sqlbase.ResultColumns, error) {
backup, ok := stmt.(*parser.ShowBackup)
if !ok {
return nil, nil, nil
}
if err := utilccl.CheckEnterpriseEnabled(
p.ExecCfg().Settings, p.ExecCfg().ClusterID(), p.ExecCfg().Organization(), "SHOW BACKUP",
); err != nil {
return nil, nil, err
}
if err := p.RequireSuperUser("SHOW BACKUP"); err != nil {
return nil, nil, err
}
toFn, err := p.TypeAsString(backup.Path, "SHOW BACKUP")
if err != nil {
return nil, nil, err
}
header := sqlbase.ResultColumns{
{Name: "database", Typ: parser.TypeString},
{Name: "table", Typ: parser.TypeString},
{Name: "start_time", Typ: parser.TypeTimestamp},
{Name: "end_time", Typ: parser.TypeTimestamp},
{Name: "size_bytes", Typ: parser.TypeInt},
{Name: "rows", Typ: parser.TypeInt},
}
fn := func(ctx context.Context, resultsCh chan<- parser.Datums) error {
// TODO(dan): Move this span into sql.
ctx, span := tracing.ChildSpan(ctx, stmt.StatementTag())
defer tracing.FinishSpan(span)
str, err := toFn()
if err != nil {
return err
}
desc, err := readBackupDescriptorFromURI(ctx, str)
if err != nil {
return err
}
descs := make(map[sqlbase.ID]string)
for _, descriptor := range desc.Descriptors {
if database := descriptor.GetDatabase(); database != nil {
if _, ok := descs[database.ID]; !ok {
descs[database.ID] = database.Name
}
}
}
descSizes := make(map[sqlbase.ID]roachpb.BulkOpSummary)
for _, file := range desc.Files {
// TODO(dan): This assumes each file in the backup only contains
// data from a single table, which is usually but not always
// correct. It does not account for interleaved tables or if a
// BACKUP happened to catch a newly created table that hadn't yet
// been split into its own range.
_, tableID, err := encoding.DecodeUvarintAscending(file.Span.Key)
if err != nil {
continue
}
s := descSizes[sqlbase.ID(tableID)]
s.Add(file.EntryCounts)
descSizes[sqlbase.ID(tableID)] = s
}
start := parser.DNull
if desc.StartTime.WallTime != 0 {
start = parser.MakeDTimestamp(timeutil.Unix(0, desc.StartTime.WallTime), time.Nanosecond)
}
for _, descriptor := range desc.Descriptors {
if table := descriptor.GetTable(); table != nil {
dbName := descs[table.ParentID]
resultsCh <- parser.Datums{
parser.NewDString(dbName),
parser.NewDString(table.Name),
start,
parser.MakeDTimestamp(timeutil.Unix(0, desc.EndTime.WallTime), time.Nanosecond),
parser.NewDInt(parser.DInt(descSizes[table.ID].DataSize)),
parser.NewDInt(parser.DInt(descSizes[table.ID].Rows)),
}
}
}
return nil
}
return fn, header, nil
}
func init() {
sql.AddPlanHook(backupPlanHook)
sql.AddPlanHook(showBackupPlanHook)
jobs.AddResumeHook(backupResumeHook)
}
马建仓 AI 助手
尝试更多
代码解读
代码找茬
代码优化
1
https://gitee.com/mirrors_cockroachdb/cockroach.git
git@gitee.com:mirrors_cockroachdb/cockroach.git
mirrors_cockroachdb
cockroach
cockroach
v1.1.8

搜索帮助