From d41203f6729f51ee5d7a08f541ce5f815ccec11c Mon Sep 17 00:00:00 2001 From: Ahsan Barkati Date: Thu, 22 Apr 2021 17:46:52 +0530 Subject: [PATCH] fix(backup): use StreamWriter to write to badger during backup/restore This commit is a major rewrite of backup and online restore code. It used to use KVLoader in badger. Now it instead uses StreamWriter that is much faster for writes. cherry-pick PR https://github.com/dgraph-io/dgraph/pull/7753 following commits are cherry-picked (in reverse order): * opt(restore): Sort the buffer before spinning the writeToDisk goroutine (#7984) (#7996) * fix(backup): Fix full backup request (#7932) (#7933) * fix: fixing graphql schema update when the data is restored + skipping /probe/graphql from audit (#7925) * fix(restore): return nil if there is error (#7899) * Don't ban namespace in export_backup * reset the kv.StreamId before sending to stream writer (#7833) (#7837) * fix(restore): Bump uid and namespace after restore (#7790) (#7800) * fix(ee): GetKeys should return an error (#7713) (#7797) * fix(backup): Free the UidPack after use (#7786) * fix(export-backup): Fix double free in export backup (#7780) (#7783) * fix(lsbackup): Fix profiler in lsBackup (#7729) * Bring back "perf(Backup): Improve backup performance (#7601)" * Opt(Backup): Make backups faster (#7680) * Fix s3 backup copy (#7669) * [BREAKING] Opt(Restore): Optimize Restore's new map-reduce based design (#7666) * Perf(restore): Implement map-reduce based restore (#7664) * feat(backup): Merge backup refactoring * Revert "perf(Backup): Improve backup performance (#7601)" --- dgraph/cmd/bulk/reduce.go | 2 +- dgraph/cmd/decrypt/decrypt.go | 2 +- dgraph/cmd/increment/increment.go | 45 +- dgraph/cmd/root_ee.go | 1 + dgraph/cmd/zero/assign.go | 23 +- dgraph/cmd/zero/zero_test.go | 39 + dgraph/main.go | 2 +- ee/acl/acl_test.go | 5 +- ee/audit/interceptor_ee.go | 5 +- ee/backup/run.go | 204 +++-- ee/keys.go | 1 + ee/keys_ee.go | 4 +- ee/vault/vault.go | 32 + ee/vault_ee.go | 1 + graphql/admin/admin.go | 52 +- graphql/admin/schema.go | 5 +- graphql/e2e/common/common.go | 3 +- posting/list.go | 5 +- protos/pb.proto | 3 + protos/pb/pb.pb.go | 722 ++++++++++-------- systest/backup/common/utils.go | 5 +- systest/backup/encryption/backup_test.go | 17 +- systest/backup/filesystem/backup_test.go | 3 +- systest/backup/minio-large/backup_test.go | 3 +- systest/backup/minio/backup_test.go | 7 +- systest/backup/multi-tenancy/backup_test.go | 274 ++----- systest/online-restore/online_restore_test.go | 2 +- worker/acl_cache.go | 1 + worker/acl_cache_test.go | 2 + worker/aggregator.go | 4 +- worker/backup.go | 111 ++- worker/backup_common.go | 128 ---- worker/backup_ee.go | 479 +++++++++++- worker/backup_handler.go | 459 ++++++----- worker/backup_handler_test.go | 123 --- worker/backup_manifest.go | 259 +++++++ worker/backup_oss.go | 46 ++ worker/backup_processor.go | 473 ------------ worker/cdc_ee.go | 2 + worker/draft.go | 21 +- worker/draft_test.go | 1 + worker/executor.go | 1 + worker/export.go | 459 +++++------ worker/export_test.go | 7 +- worker/file_handler.go | 305 -------- worker/graphql_schema.go | 54 +- worker/groups.go | 1 + worker/online_restore.go | 510 ++++++++++++- worker/online_restore_ee.go | 412 ---------- worker/online_restore_oss.go | 43 ++ worker/restore.go | 317 -------- worker/restore_map.go | 708 +++++++++++++++++ worker/restore_reduce.go | 314 ++++++++ worker/s3_handler.go | 405 ---------- 54 files changed, 3765 insertions(+), 3347 deletions(-) create mode 100644 ee/vault/vault.go delete mode 100644 worker/backup_common.go delete mode 100644 worker/backup_handler_test.go create mode 100644 worker/backup_manifest.go create mode 100644 worker/backup_oss.go delete mode 100644 worker/backup_processor.go delete mode 100644 worker/file_handler.go delete mode 100644 worker/online_restore_ee.go create mode 100644 worker/online_restore_oss.go delete mode 100644 worker/restore.go create mode 100644 worker/restore_map.go create mode 100644 worker/restore_reduce.go delete mode 100644 worker/s3_handler.go diff --git a/dgraph/cmd/bulk/reduce.go b/dgraph/cmd/bulk/reduce.go index 808fb540f6b..b5094dd5b34 100644 --- a/dgraph/cmd/bulk/reduce.go +++ b/dgraph/cmd/bulk/reduce.go @@ -481,8 +481,8 @@ func (r *reducer) reduce(partitionKeys [][]byte, mapItrs []*mapIterator, ci *cou partitionKeys = append(partitionKeys, nil) for i := 0; i < len(partitionKeys); i++ { + pkey := partitionKeys[i] for _, itr := range mapItrs { - pkey := partitionKeys[i] itr.Next(cbuf, pkey) } if cbuf.LenNoPadding() < 256<<20 { diff --git a/dgraph/cmd/decrypt/decrypt.go b/dgraph/cmd/decrypt/decrypt.go index 78ab144dce4..96893767ea3 100644 --- a/dgraph/cmd/decrypt/decrypt.go +++ b/dgraph/cmd/decrypt/decrypt.go @@ -30,7 +30,7 @@ import ( ) type options struct { - // keyfile comes from the encryption or Vault flags + // keyfile comes from the encryption_key_file or Vault flags keyfile x.Sensitive file string output string diff --git a/dgraph/cmd/increment/increment.go b/dgraph/cmd/increment/increment.go index 30e57318a84..e5a89425448 100644 --- a/dgraph/cmd/increment/increment.go +++ b/dgraph/cmd/increment/increment.go @@ -30,6 +30,7 @@ import ( "github.com/dgraph-io/dgo/v210/protos/api" "github.com/dgraph-io/dgraph/x" "github.com/dgraph-io/ristretto/z" + "github.com/pkg/errors" "github.com/spf13/cobra" "github.com/spf13/viper" @@ -199,32 +200,42 @@ func run(conf *viper.Viper) { dg = dgTmp } - // Run things serially first. - for i := 0; i < conc; i++ { - _, err := process(dg, conf) - x.Check(err) - num-- + processOne := func(i int) error { + txnStart := time.Now() // Start time of transaction + cnt, err := process(dg, conf) + now := time.Now().UTC().Format(format) + if err != nil { + return err + } + serverLat := cnt.qLatency + cnt.mLatency + clientLat := time.Since(txnStart).Round(time.Millisecond) + fmt.Printf( + "[w%d] %-17s Counter VAL: %d [ Ts: %d ] Latency: Q %s M %s S %s C %s D %s\n", + i, now, cnt.Val, cnt.startTs, cnt.qLatency, cnt.mLatency, + serverLat, clientLat, clientLat-serverLat) + return nil + } + + // Run things serially first, if conc > 1. + if conc > 1 { + for i := 0; i < conc; i++ { + err := processOne(0) + x.Check(err) + num-- + } } var wg sync.WaitGroup - f := func(i int) { + f := func(worker int) { defer wg.Done() count := 0 for count < num { - txnStart := time.Now() // Start time of transaction - cnt, err := process(dg, conf) - now := time.Now().UTC().Format(format) - if err != nil { + if err := processOne(worker); err != nil { + now := time.Now().UTC().Format(format) fmt.Printf("%-17s While trying to process counter: %v. Retrying...\n", now, err) time.Sleep(time.Second) continue } - serverLat := cnt.qLatency + cnt.mLatency - clientLat := time.Since(txnStart).Round(time.Millisecond) - fmt.Printf( - "[%d] %-17s Counter VAL: %d [ Ts: %d ] Latency: Q %s M %s S %s C %s D %s\n", - i, now, cnt.Val, cnt.startTs, cnt.qLatency, cnt.mLatency, - serverLat, clientLat, clientLat-serverLat) time.Sleep(waitDur) count++ } @@ -232,7 +243,7 @@ func run(conf *viper.Viper) { for i := 0; i < conc; i++ { wg.Add(1) - go f(i) + go f(i + 1) } wg.Wait() } diff --git a/dgraph/cmd/root_ee.go b/dgraph/cmd/root_ee.go index 55b90bf1b13..baf794daae2 100644 --- a/dgraph/cmd/root_ee.go +++ b/dgraph/cmd/root_ee.go @@ -1,3 +1,4 @@ +//go:build !oss // +build !oss /* diff --git a/dgraph/cmd/zero/assign.go b/dgraph/cmd/zero/assign.go index 70ee00bcd15..e2085754169 100644 --- a/dgraph/cmd/zero/assign.go +++ b/dgraph/cmd/zero/assign.go @@ -175,7 +175,9 @@ func (s *Server) lease(ctx context.Context, num *pb.Num) (*pb.AssignedIds, error } // AssignIds is used to assign new ids (UIDs, NsIDs) by communicating with the leader of the -// RAFT group responsible for handing out ids. +// RAFT group responsible for handing out ids. If bump is set to true in the request then the +// lease for the given id type is bumped to num.Val and {startId, endId} of the newly leased ids +// in the process of bump is returned. func (s *Server) AssignIds(ctx context.Context, num *pb.Num) (*pb.AssignedIds, error) { if ctx.Err() != nil { return &emptyAssignedIds, ctx.Err() @@ -246,6 +248,25 @@ func (s *Server) AssignIds(ctx context.Context, num *pb.Num) (*pb.AssignedIds, e return err } + // If this is a bump request and the current node is the leader then we create a normal lease + // request based on the number of required ids to reach the asked bump value. If the current + // node is not the leader then the bump request will be forwarded to the leader by lease(). + if num.GetBump() && s.Node.AmLeader() { + s.leaseLock.Lock() + cur := s.nextLease[num.GetType()] - 1 + s.leaseLock.Unlock() + + // We need to lease more UIDs if bump request is more than current max lease. + req := num.GetVal() + if cur >= req { + return &emptyAssignedIds, errors.Errorf("Nothing to be leased") + } + num.Val = req - cur + + // Set bump to false because we want to lease the required ids in the following request. + num.Bump = false + } + c := make(chan error, 1) go func() { c <- lease() diff --git a/dgraph/cmd/zero/zero_test.go b/dgraph/cmd/zero/zero_test.go index d4f6e568101..71887862f16 100644 --- a/dgraph/cmd/zero/zero_test.go +++ b/dgraph/cmd/zero/zero_test.go @@ -24,6 +24,7 @@ import ( "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/testutil" "github.com/stretchr/testify/require" + "google.golang.org/grpc" ) func TestRemoveNode(t *testing.T) { @@ -44,3 +45,41 @@ func TestIdLeaseOverflow(t *testing.T) { require.Error(t, err) require.Contains(t, err.Error(), "limit has reached") } + +func TestIdBump(t *testing.T) { + dialOpts := []grpc.DialOption{ + grpc.WithBlock(), + grpc.WithInsecure(), + } + ctx := context.Background() + con, err := grpc.DialContext(ctx, testutil.SockAddrZero, dialOpts...) + require.NoError(t, err) + + zc := pb.NewZeroClient(con) + + res, err := zc.AssignIds(ctx, &pb.Num{Val: 10, Type: pb.Num_UID}) + require.NoError(t, err) + require.Equal(t, uint64(10), res.GetEndId()-res.GetStartId()+1) + + // Next assignemnt's startId should be greater than 10. + res, err = zc.AssignIds(ctx, &pb.Num{Val: 50, Type: pb.Num_UID}) + require.NoError(t, err) + require.Greater(t, res.GetStartId(), uint64(10)) + require.Equal(t, uint64(50), res.GetEndId()-res.GetStartId()+1) + + bumpTo := res.GetEndId() + 100000 + + // Bump the lease to (last result + 100000). + res, err = zc.AssignIds(ctx, &pb.Num{Val: bumpTo, Type: pb.Num_UID, Bump: true}) + require.NoError(t, err) + + // Next assignemnt's startId should be greater than bumpTo. + res, err = zc.AssignIds(ctx, &pb.Num{Val: 10, Type: pb.Num_UID}) + require.NoError(t, err) + require.Greater(t, res.GetStartId(), bumpTo) + require.Equal(t, uint64(10), res.GetEndId()-res.GetStartId()+1) + + // If bump request is less than maxLease, then it should result in no-op. + res, err = zc.AssignIds(ctx, &pb.Num{Val: 10, Type: pb.Num_UID, Bump: true}) + require.Contains(t, err.Error(), "Nothing to be leased") +} diff --git a/dgraph/main.go b/dgraph/main.go index ae387d399cc..59fb3540e1f 100644 --- a/dgraph/main.go +++ b/dgraph/main.go @@ -57,7 +57,7 @@ func main() { for range ticker.C { // Read Jemalloc stats first. Print if there's a big difference. z.ReadMemStats(&js) - if diff := absDiff(uint64(z.NumAllocBytes()), lastAlloc); diff > 256<<20 { + if diff := absDiff(uint64(z.NumAllocBytes()), lastAlloc); diff > 1<<30 { glog.V(2).Infof("NumAllocBytes: %s jemalloc: Active %s Allocated: %s"+ " Resident: %s Retained: %s\n", humanize.IBytes(uint64(z.NumAllocBytes())), diff --git a/ee/acl/acl_test.go b/ee/acl/acl_test.go index 50b7920071c..bb408b4035c 100644 --- a/ee/acl/acl_test.go +++ b/ee/acl/acl_test.go @@ -28,6 +28,7 @@ import ( "github.com/dgraph-io/dgo/v210/protos/api" "github.com/dgraph-io/dgraph/testutil" "github.com/dgraph-io/dgraph/x" + "github.com/golang/glog" "github.com/stretchr/testify/require" ) @@ -2858,7 +2859,7 @@ func TestGuardianOnlyAccessForAdminEndpoints(t *testing.T) { queryName: "listBackups", respIsArray: true, testGuardianAccess: true, - guardianErr: "The path \"\" does not exist or it is inaccessible.", + guardianErr: `The uri path: "" doesn't exist`, guardianData: `{"listBackups": []}`, }, { @@ -2939,7 +2940,7 @@ func TestGuardianOnlyAccessForAdminEndpoints(t *testing.T) { }`, queryName: "restore", testGuardianAccess: true, - guardianErr: "The path \"\" does not exist or it is inaccessible.", + guardianErr: `The uri path: "" doesn't exist`, guardianData: `{"restore": {"code": "Failure"}}`, }, { diff --git a/ee/audit/interceptor_ee.go b/ee/audit/interceptor_ee.go index 2e59b705957..e9de9a1b777 100644 --- a/ee/audit/interceptor_ee.go +++ b/ee/audit/interceptor_ee.go @@ -63,8 +63,9 @@ var skipApis = map[string]bool{ var skipEPs = map[string]bool{ // list of endpoints that needs to be skipped - "/health": true, - "/state": true, + "/health": true, + "/state": true, + "/probe/graphql": true, } func AuditRequestGRPC(ctx context.Context, req interface{}, diff --git a/ee/backup/run.go b/ee/backup/run.go index 4a69e0cfb25..d0c0e817ea3 100644 --- a/ee/backup/run.go +++ b/ee/backup/run.go @@ -18,28 +18,24 @@ import ( "encoding/json" "fmt" "io/ioutil" - "math" + "net/url" "os" "path/filepath" - "strconv" - "strings" "time" - "github.com/dgraph-io/badger/v3" - "github.com/dgraph-io/badger/v3/options" - "golang.org/x/sync/errgroup" - - "google.golang.org/grpc/credentials" - + bpb "github.com/dgraph-io/badger/v3/pb" "github.com/dgraph-io/dgraph/ee" + "github.com/dgraph-io/dgraph/posting" "github.com/dgraph-io/dgraph/protos/pb" - "github.com/dgraph-io/dgraph/upgrade" "github.com/dgraph-io/dgraph/worker" "github.com/dgraph-io/dgraph/x" "github.com/dgraph-io/ristretto/z" + + "github.com/golang/glog" "github.com/pkg/errors" "github.com/spf13/cobra" "google.golang.org/grpc" + "google.golang.org/grpc/credentials" ) // Restore is the sub-command used to restore a backup. @@ -76,10 +72,8 @@ func initRestore() { Short: "Restore backup from Dgraph Enterprise Edition", Long: ` Restore loads objects created with the backup feature in Dgraph Enterprise Edition (EE). - Backups are originated from HTTP at /admin/backup, then can be restored using CLI restore command. Restore is intended to be used with new Dgraph clusters in offline state. - The --location flag indicates a source URI with Dgraph backup objects. This URI supports all the schemes used for backup. @@ -95,7 +89,6 @@ Source URI parts: args - specific arguments that are ok to appear in logs. The --posting flag sets the posting list parent dir to store the loaded backup files. - Using the --zero flag will use a Dgraph Zero address to update the start timestamp using the restored version. Otherwise, the timestamp must be manually updated through Zero's HTTP 'assign' command. @@ -105,7 +98,6 @@ a posting directory 'p' matching the backup group ID. Such that a backup file named '.../r32-g2.backup' will be loaded to posting dir 'p2'. Usage examples: - # Restore from local dir or NFS mount: $ dgraph restore -p . -l /var/backups/dgraph @@ -114,7 +106,6 @@ $ dgraph restore -p /var/db/dgraph -l s3://s3.us-west-2.amazonaws.com/srfrog/dgr # Restore from dir and update Ts: $ dgraph restore -p . -l /var/backups/dgraph -z localhost:5080 - `, Args: cobra.NoArgs, Run: func(cmd *cobra.Command, args []string) { @@ -133,11 +124,10 @@ $ dgraph restore -p . -l /var/backups/dgraph -z localhost:5080 z.NewSuperFlagHelp(worker.BadgerDefaults). Head("Badger options"). Flag("compression", - "Specifies the compression algorithm and compression level (if applicable) for the "+ - `postings directory. "none" would disable compression, while "zstd:1" would set `+ - "zstd compression at level 1."). - Flag("goroutines", - "The number of goroutines to use in badger.Stream."). + "Specifies the compression algorithm & compression level (if applicable) for the "+ + `postings directory. "none" would disable compression, while "zstd:1" would `+ + "set zstd compression at level 1."). + Flag("goroutines", "The number of goroutines to use in badger.Stream."). String()) flag.StringVarP(&opt.location, "location", "l", "", @@ -181,11 +171,6 @@ func initBackupLs() { } func runRestoreCmd() error { - var ( - start time.Time - zc pb.ZeroClient - err error - ) keys, err := ee.GetKeys(Restore.Conf) if err != nil { return err @@ -199,6 +184,7 @@ func runRestoreCmd() error { "meant to do this") } + var zc pb.ZeroClient if opt.zero != "" { fmt.Println("Updating Zero timestamp at:", opt.zero) ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) @@ -223,13 +209,14 @@ func runRestoreCmd() error { badger := z.NewSuperFlag(opt.badger).MergeAndCheckDefault(worker.BadgerDefaults) ctype, clevel := x.ParseCompression(badger.GetString("compression")) - start = time.Now() - result := worker.RunRestore(opt.pdir, opt.location, opt.backupId, opt.key, ctype, clevel) + start := time.Now() + result := worker.RunOfflineRestore(opt.pdir, opt.location, + opt.backupId, "", opt.key, ctype, clevel) if result.Err != nil { return result.Err } if result.Version == 0 { - return errors.Errorf("Failed to obtain a restore version") + return errors.Errorf("failed to obtain a restore version") } fmt.Printf("Restore version: %d\n", result.Version) fmt.Printf("Restore max uid: %d\n", result.MaxLeaseUid) @@ -349,6 +336,66 @@ func initExportBackup() { ee.RegisterEncFlag(flag) } +type bufWriter struct { + writers *worker.Writers + req *pb.ExportRequest +} + +func exportSchema(writers *worker.Writers, val []byte, pk x.ParsedKey) error { + var kv *bpb.KV + var err error + if pk.IsSchema() { + kv, err = worker.SchemaExportKv(pk.Attr, val, true) + if err != nil { + return err + } + } else { + kv, err = worker.TypeExportKv(pk.Attr, val) + if err != nil { + return err + } + } + return worker.WriteExport(writers, kv, "rdf") +} + +func (bw *bufWriter) Write(buf *z.Buffer) error { + kv := &bpb.KV{} + err := buf.SliceIterate(func(s []byte) error { + kv.Reset() + if err := kv.Unmarshal(s); err != nil { + return errors.Wrap(err, "processKvBuf failed to unmarshal kv") + } + pk, err := x.Parse(kv.Key) + if err != nil { + return errors.Wrap(err, "processKvBuf failed to parse key") + } + if pk.Attr == "_predicate_" { + return nil + } + if pk.IsSchema() || pk.IsType() { + return exportSchema(bw.writers, kv.Value, pk) + } + if pk.IsData() { + pl := &pb.PostingList{} + if err := pl.Unmarshal(kv.Value); err != nil { + return errors.Wrap(err, "processKvBuf failed to Unmarshal pl") + } + l := posting.NewList(kv.Key, pl, kv.Version) + kvList, err := worker.ToExportKvList(pk, l, bw.req) + if err != nil { + return errors.Wrap(err, "processKvBuf failed to Export") + } + if len(kvList.Kv) == 0 { + return nil + } + exportKv := kvList.Kv[0] + return worker.WriteExport(bw.writers, exportKv, bw.req.Format) + } + return nil + }) + return errors.Wrap(err, "bufWriter failed to write") +} + func runExportBackup() error { keys, err := ee.GetKeys(ExportBackup.Conf) if err != nil { @@ -366,70 +413,63 @@ func runExportBackup() error { if err := os.MkdirAll(exportDir, 0755); err != nil { return errors.Wrapf(err, "cannot create dir %s", exportDir) } - tmpDir, err := ioutil.TempDir("", "export_backup") + + uri, err := url.Parse(opt.location) if err != nil { - return errors.Wrapf(err, "cannot create temp dir") + return errors.Wrapf(err, "runExportBackup") } - - restore := worker.RunRestore(tmpDir, opt.location, "", opt.key, options.None, 0) - if restore.Err != nil { - return restore.Err + handler, err := worker.NewUriHandler(uri, nil) + if err != nil { + return errors.Wrapf(err, "runExportBackup") } - - files, err := ioutil.ReadDir(tmpDir) + latestManifest, err := worker.GetLatestManifest(handler, uri) if err != nil { - return err + return errors.Wrapf(err, "runExportBackup") } - // Export the data from the p directories produced by the last step. - eg, _ := errgroup.WithContext(context.Background()) - for _, f := range files { - if !f.IsDir() { - continue - } - dir := filepath.Join(filepath.Join(tmpDir, f.Name())) - gid, err := strconv.ParseUint(strings.TrimPrefix(f.Name(), "p"), 32, 10) - if err != nil { - fmt.Printf("WARNING WARNING WARNING: unable to get group id from directory "+ - "inside DB at %s: %v", dir, err) - continue + mapDir, err := ioutil.TempDir(x.WorkerConfig.TmpDir, "restore-export") + x.Check(err) + defer os.RemoveAll(mapDir) + glog.Infof("Created temporary map directory: %s\n", mapDir) + + // TODO: Can probably make this procesing concurrent. + for gid := range latestManifest.Groups { + glog.Infof("Exporting group: %d", gid) + req := &pb.RestoreRequest{ + GroupId: gid, + Location: opt.location, + EncryptionKeyFile: ExportBackup.Conf.GetString("encryption_key_file"), + RestoreTs: 1, } - if opt.upgrade && gid == 1 { - // Query the cors in badger db and append it at the end of GraphQL schema. - // This change was introduced in v21.03. Backups with 20.07 <= version < 21.03 - // should apply this. - db, err := badger.OpenManaged(badger.DefaultOptions(dir). - WithNumVersionsToKeep(math.MaxInt32). - WithEncryptionKey(opt.key)) - if err != nil { - return err - } - if err := upgrade.OfflineUpgradeFrom2011To2103(db); err != nil { - return errors.Wrapf(err, "while fixing cors") - } - if err := db.Close(); err != nil { - return err - } + if _, err := worker.RunMapper(req, mapDir); err != nil { + return errors.Wrap(err, "Failed to map the backups") + } + in := &pb.ExportRequest{ + GroupId: uint32(gid), + ReadTs: latestManifest.ValidReadTs(), + UnixTs: time.Now().Unix(), + Format: opt.format, + Destination: exportDir, + } + uts := time.Unix(in.UnixTs, 0) + destPath := fmt.Sprintf("dgraph.r%d.u%s", in.ReadTs, uts.UTC().Format("0102.1504")) + exportStorage, err := worker.NewExportStorage(in, destPath) + if err != nil { + return err } - eg.Go(func() error { - return worker.StoreExport(&pb.ExportRequest{ - GroupId: uint32(gid), - ReadTs: restore.Version, - UnixTs: time.Now().Unix(), - Format: opt.format, - Destination: exportDir, - }, dir, opt.key) - }) - } - if err := eg.Wait(); err != nil { - return errors.Wrapf(err, "error while exporting data") - } + writers, err := worker.InitWriters(exportStorage, in) + if err != nil { + return err + } - // Clean up temporary directory. - if err := os.RemoveAll(tmpDir); err != nil { - return errors.Wrapf(err, "cannot remove temp directory at %s", tmpDir) + w := &bufWriter{req: in, writers: writers} + if err := worker.RunReducer(w, mapDir); err != nil { + return errors.Wrap(err, "Failed to reduce the map") + } + if _, err := exportStorage.FinishWriting(writers); err != nil { + return errors.Wrap(err, "Failed to finish write") + } } - return nil } diff --git a/ee/keys.go b/ee/keys.go index a81eca3c09c..c88686f1feb 100644 --- a/ee/keys.go +++ b/ee/keys.go @@ -1,3 +1,4 @@ +//go:build oss // +build oss /* diff --git a/ee/keys_ee.go b/ee/keys_ee.go index 97f08a997b0..d9e2a5c304f 100644 --- a/ee/keys_ee.go +++ b/ee/keys_ee.go @@ -1,3 +1,4 @@ +//go:build !oss // +build !oss /* @@ -25,7 +26,6 @@ import ( // this function always returns an error. func GetKeys(config *viper.Viper) (*Keys, error) { keys := &Keys{} - var err error aclSuperFlag := z.NewSuperFlag(config.GetString("acl")).MergeAndCheckDefault(AclDefaults) encSuperFlag := z.NewSuperFlag(config.GetString("encryption")).MergeAndCheckDefault(EncDefaults) @@ -37,6 +37,7 @@ func GetKeys(config *viper.Viper) (*Keys, error) { if keys.AclKey != nil { return nil, fmt.Errorf("flags: ACL secret key set in both vault and acl flags") } + var err error if keys.AclKey, err = ioutil.ReadFile(aclKeyFile); err != nil { return nil, fmt.Errorf("error reading ACL secret key from file: %s: %s", aclKeyFile, err) } @@ -50,6 +51,7 @@ func GetKeys(config *viper.Viper) (*Keys, error) { if keys.EncKey != nil { return nil, fmt.Errorf("flags: Encryption key set in both vault and encryption flags") } + var err error if keys.EncKey, err = ioutil.ReadFile(encKeyFile); err != nil { return nil, fmt.Errorf("error reading encryption key from file: %s: %s", encKeyFile, err) } diff --git a/ee/vault/vault.go b/ee/vault/vault.go new file mode 100644 index 00000000000..64ad7a212a9 --- /dev/null +++ b/ee/vault/vault.go @@ -0,0 +1,32 @@ +//go:build oss +// +build oss + +/* + * Copyright 2020-2021 Dgraph Labs, Inc. and Contributors + * + * 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 vault + +import ( + "github.com/dgraph-io/dgraph/ee" + + "github.com/golang/glog" + "github.com/spf13/viper" +) + +func GetKeys(config *viper.Viper) (*ee.Keys, error) { + glog.Exit("flags: vault is an enterprise-only feature") + return nil, nil +} diff --git a/ee/vault_ee.go b/ee/vault_ee.go index 470b262378b..0b05f31ef65 100644 --- a/ee/vault_ee.go +++ b/ee/vault_ee.go @@ -20,6 +20,7 @@ import ( "reflect" "github.com/dgraph-io/dgraph/x" + "github.com/dgraph-io/ristretto/z" "github.com/golang/glog" "github.com/hashicorp/vault/api" diff --git a/graphql/admin/admin.go b/graphql/admin/admin.go index 5297a4d1eb1..53f09508797 100644 --- a/graphql/admin/admin.go +++ b/graphql/admin/admin.go @@ -18,6 +18,7 @@ package admin import ( "context" + "fmt" "sync" "sync/atomic" "time" @@ -582,14 +583,6 @@ func (g *GraphQLHealthStore) updatingSchema() { g.v.Store(GraphQLHealth{Healthy: true, StatusMsg: "updating schema"}) } -type gqlSchema struct { - ID string `json:"id,omitempty"` - Schema string `json:"schema,omitempty"` - Version uint64 - GeneratedSchema string - loaded bool // This indicate whether the schema has been loaded into graphql server or not -} - type adminServer struct { rf resolve.ResolverFactory resolver *resolve.RequestResolver @@ -600,8 +593,7 @@ type adminServer struct { // The GraphQL server that's being admin'd gqlServer IServeGraphQL - schema map[uint64]*gqlSchema - + gqlSchemas *worker.GQLSchemaStore // When the schema changes, we use these to create a new RequestResolver for // the main graphql endpoint (gqlServer) and thus refresh the API. fns *resolve.ResolverFns @@ -659,7 +651,7 @@ func newAdminResolver( fns: fns, withIntrospection: withIntrospection, globalEpoch: epoch, - schema: make(map[uint64]*gqlSchema), + gqlSchemas: worker.NewGQLSchemaStore(), gqlServer: defaultGqlServer, } adminServerVar = server // store the admin server in package variable @@ -695,14 +687,14 @@ func newAdminResolver( } ns, _ := x.ParseNamespaceAttr(pk.Attr) - newSchema := &gqlSchema{ + newSchema := &worker.GqlSchema{ ID: query.UidToHex(pk.Uid), Version: kv.GetVersion(), Schema: string(pl.Postings[0].Value), } server.mux.RLock() - currentSchema, ok := server.schema[ns] + currentSchema, ok := server.gqlSchemas.GetCurrent(ns) if ok { schemaChanged := newSchema.Schema == currentSchema.Schema if newSchema.Version <= currentSchema.Version || schemaChanged { @@ -730,18 +722,18 @@ func newAdminResolver( server.incrementSchemaUpdateCounter(ns) // if the schema hasn't been loaded yet, then we don't need to load it here - currentSchema, ok = server.schema[ns] - if !(ok && currentSchema.loaded) { + currentSchema, ok = server.gqlSchemas.GetCurrent(ns) + if !(ok && currentSchema.Loaded) { // this just set schema in admin server, so that next invalid badger subscription update gets rejected upfront - server.schema[ns] = newSchema + server.gqlSchemas.Set(ns, newSchema) glog.Infof("namespace: %d. Skipping in-memory GraphQL schema update, "+ "it will be lazy-loaded later.", ns) return } // update this schema in both admin and graphql server - newSchema.loaded = true - server.schema[ns] = newSchema + newSchema.Loaded = true + server.gqlSchemas.Set(ns, newSchema) server.resetSchema(ns, gqlSchema) glog.Infof("namespace: %d. Successfully updated GraphQL schema. "+ @@ -815,16 +807,16 @@ func newAdminResolverFactory() resolve.ResolverFactory { return rf.WithSchemaIntrospection() } -func getCurrentGraphQLSchema(namespace uint64) (*gqlSchema, error) { +func getCurrentGraphQLSchema(namespace uint64) (*worker.GqlSchema, error) { uid, graphQLSchema, err := edgraph.GetGQLSchema(namespace) if err != nil { return nil, err } - return &gqlSchema{ID: uid, Schema: graphQLSchema}, nil + return &worker.GqlSchema{ID: uid, Schema: graphQLSchema}, nil } -func generateGQLSchema(sch *gqlSchema, ns uint64) (schema.Schema, error) { +func generateGQLSchema(sch *worker.GqlSchema, ns uint64) (schema.Schema, error) { schHandler, err := schema.NewHandler(sch.Schema, false) if err != nil { return nil, err @@ -862,8 +854,8 @@ func (as *adminServer) initServer() { glog.Errorf("namespace: %d. Error reading GraphQL schema: %s.", x.GalaxyNamespace, err) continue } - sch.loaded = true - as.schema[x.GalaxyNamespace] = sch + sch.Loaded = true + as.gqlSchemas.Set(x.GalaxyNamespace, sch) // adding the actual resolvers for updateGQLSchema and getGQLSchema only after server has // current GraphQL schema, if there was any. as.addConnectedAdminResolvers() @@ -1003,8 +995,12 @@ func (as *adminServer) resetSchema(ns uint64, gqlSchema schema.Schema) { return resolve.QueryResolverFunc(func(ctx context.Context, query schema.Query) *resolve.Resolved { as.mux.RLock() defer as.mux.RUnlock() - sch := as.schema[ns].Schema - handler, err := schema.NewHandler(sch, true) + sch, ok := as.gqlSchemas.GetCurrent(ns) + if !ok { + return resolve.EmptyResult(query, + fmt.Errorf("error while getting the schema for ns %d", ns)) + } + handler, err := schema.NewHandler(sch.Schema, true) if err != nil { return resolve.EmptyResult(query, err) } @@ -1031,7 +1027,7 @@ func (as *adminServer) resetSchema(ns uint64, gqlSchema schema.Schema) { func (as *adminServer) lazyLoadSchema(namespace uint64) error { // if the schema is already in memory, no need to fetch it from disk as.mux.RLock() - if currentSchema, ok := as.schema[namespace]; ok && currentSchema.loaded { + if currentSchema, ok := as.gqlSchemas.GetCurrent(namespace); ok && currentSchema.Loaded { as.mux.RUnlock() return nil } @@ -1061,8 +1057,8 @@ func (as *adminServer) lazyLoadSchema(namespace uint64) error { as.mux.Lock() defer as.mux.Unlock() - sch.loaded = true - as.schema[namespace] = sch + sch.Loaded = true + as.gqlSchemas.Set(namespace, sch) as.resetSchema(namespace, generatedSchema) glog.Infof("namespace: %d. Successfully lazy-loaded GraphQL schema.", namespace) diff --git a/graphql/admin/schema.go b/graphql/admin/schema.go index c4f329f4814..29a1e7360f8 100644 --- a/graphql/admin/schema.go +++ b/graphql/admin/schema.go @@ -19,6 +19,7 @@ package admin import ( "context" "encoding/json" + "github.com/dgraph-io/dgraph/worker" "github.com/dgraph-io/dgraph/edgraph" "github.com/dgraph-io/dgraph/graphql/resolve" @@ -33,7 +34,7 @@ type getSchemaResolver struct { } type updateGQLSchemaInput struct { - Set gqlSchema `json:"set,omitempty"` + Set worker.GqlSchema `json:"set,omitempty"` } type updateSchemaResolver struct { @@ -88,7 +89,7 @@ func (gsr *getSchemaResolver) Resolve(ctx context.Context, q schema.Query) *reso return resolve.EmptyResult(q, err) } - cs := gsr.admin.schema[ns] + cs, _ := gsr.admin.gqlSchemas.GetCurrent(ns) if cs == nil || cs.ID == "" { data = map[string]interface{}{q.Name(): nil} } else { diff --git a/graphql/e2e/common/common.go b/graphql/e2e/common/common.go index 1001508ec2e..8ce7e6ba1ae 100644 --- a/graphql/e2e/common/common.go +++ b/graphql/e2e/common/common.go @@ -938,7 +938,8 @@ func gunzipData(data []byte) ([]byte, error) { func gzipData(data []byte) ([]byte, error) { var b bytes.Buffer - gz := gzip.NewWriter(&b) + gz, err := gzip.NewWriterLevel(&b, gzip.BestSpeed) + x.Check(err) if _, err := gz.Write(data); err != nil { return nil, err diff --git a/posting/list.go b/posting/list.go index 60881dde8e2..d0adc610b7e 100644 --- a/posting/list.go +++ b/posting/list.go @@ -853,7 +853,9 @@ func (l *List) Rollup(alloc *z.Allocator) ([]*bpb.KV, error) { // ToBackupPostingList uses rollup to generate a single list with no splits. // It's used during backup so that each backed up posting list is stored in a single key. -func (l *List) ToBackupPostingList(bl *pb.BackupPostingList, alloc *z.Allocator, buf *z.Buffer) (*bpb.KV, error) { +func (l *List) ToBackupPostingList( + bl *pb.BackupPostingList, alloc *z.Allocator, buf *z.Buffer) (*bpb.KV, error) { + bl.Reset() l.RLock() defer l.RUnlock() @@ -873,7 +875,6 @@ func (l *List) ToBackupPostingList(bl *pb.BackupPostingList, alloc *z.Allocator, buf.Reset() codec.DecodeToBuffer(buf, ol.Pack) bl.UidBytes = buf.Bytes() - bl.Postings = ol.Postings bl.CommitTs = ol.CommitTs bl.Splits = ol.Splits diff --git a/protos/pb.proto b/protos/pb.proto index b41b49ea72a..c537b3e0131 100644 --- a/protos/pb.proto +++ b/protos/pb.proto @@ -618,6 +618,9 @@ message Num { uint64 val = 1; bool read_only = 2; bool forwarded = 3; // True if this request was forwarded by a peer. + // If bump is set to true then we bump the lease to val. If false, we assign new ids with count + // equal to val. + bool bump = 5; enum leaseType { NS_ID = 0; UID = 1; diff --git a/protos/pb/pb.pb.go b/protos/pb/pb.pb.go index e4a7811a9bd..bcf90dbbe1f 100644 --- a/protos/pb/pb.pb.go +++ b/protos/pb/pb.pb.go @@ -4297,10 +4297,13 @@ func (m *SubscriptionResponse) GetKvs() *pb.KVList { } type Num struct { - Val uint64 `protobuf:"varint,1,opt,name=val,proto3" json:"val,omitempty"` - ReadOnly bool `protobuf:"varint,2,opt,name=read_only,json=readOnly,proto3" json:"read_only,omitempty"` - Forwarded bool `protobuf:"varint,3,opt,name=forwarded,proto3" json:"forwarded,omitempty"` - Type NumLeaseType `protobuf:"varint,4,opt,name=type,proto3,enum=pb.NumLeaseType" json:"type,omitempty"` + Val uint64 `protobuf:"varint,1,opt,name=val,proto3" json:"val,omitempty"` + ReadOnly bool `protobuf:"varint,2,opt,name=read_only,json=readOnly,proto3" json:"read_only,omitempty"` + Forwarded bool `protobuf:"varint,3,opt,name=forwarded,proto3" json:"forwarded,omitempty"` + // If bump is set to true then we bump the lease to val. If false, we assign new ids with count + // equal to val. + Bump bool `protobuf:"varint,5,opt,name=bump,proto3" json:"bump,omitempty"` + Type NumLeaseType `protobuf:"varint,4,opt,name=type,proto3,enum=pb.NumLeaseType" json:"type,omitempty"` } func (m *Num) Reset() { *m = Num{} } @@ -4357,6 +4360,13 @@ func (m *Num) GetForwarded() bool { return false } +func (m *Num) GetBump() bool { + if m != nil { + return m.Bump + } + return false +} + func (m *Num) GetType() NumLeaseType { if m != nil { return m.Type @@ -5675,346 +5685,347 @@ func init() { func init() { proto.RegisterFile("pb.proto", fileDescriptor_f80abaa17e25ccc8) } var fileDescriptor_f80abaa17e25ccc8 = []byte{ - // 5418 bytes of a gzipped FileDescriptorProto + // 5426 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x3b, 0x4b, 0x8f, 0x1c, 0x69, 0x52, 0x5d, 0xef, 0xca, 0xa8, 0x87, 0xab, 0x3f, 0x7b, 0x3c, 0xb5, 0xe5, 0x19, 0xdb, 0x93, 0x1e, 0xcf, 0xf4, 0xd8, 0xe3, 0xb6, 0xdd, 0xde, 0x85, 0x9d, 0x59, 0xad, 0x44, 0x3f, 0xaa, 0x3d, 0x3d, 0x6e, 0x77, 0x7b, 0xb3, 0xcb, 0xde, 0x87, 0x04, 0xa5, 0xec, 0xcc, 0xaf, 0xbb, 0x73, 0x3b, 0x2b, 0x33, 0x37, 0x33, 0xab, 0xb7, 0x7b, 0x6e, 0x08, 0x89, 0xbd, 0x70, 0x58, 0x89, 0x0b, 0x27, 0x0e, - 0x1c, 0xb8, 0xc0, 0x05, 0x04, 0x82, 0x0b, 0x37, 0x84, 0x10, 0xa7, 0x3d, 0x82, 0x80, 0x11, 0x9a, - 0x45, 0x1c, 0x7c, 0x40, 0x42, 0xfc, 0x01, 0x14, 0x11, 0x5f, 0xbe, 0xaa, 0xab, 0x6d, 0xcf, 0x22, - 0x0e, 0x9c, 0xea, 0x8b, 0xf8, 0xde, 0xf1, 0xc5, 0x3b, 0xb2, 0xa0, 0x19, 0xec, 0x2f, 0x07, 0xa1, - 0x1f, 0xfb, 0xa2, 0x1c, 0xec, 0x0f, 0x34, 0x33, 0x70, 0x18, 0x1c, 0xdc, 0x39, 0x74, 0xe2, 0xa3, - 0xe9, 0xfe, 0xb2, 0xe5, 0x4f, 0xee, 0xdb, 0x87, 0xa1, 0x19, 0x1c, 0xdd, 0x73, 0xfc, 0xfb, 0xfb, - 0xa6, 0x7d, 0x28, 0xc3, 0xfb, 0x27, 0x8f, 0xee, 0x07, 0xfb, 0xf7, 0x93, 0xa9, 0x83, 0x7b, 0xb9, - 0xb1, 0x87, 0xfe, 0xa1, 0x7f, 0x9f, 0xd0, 0xfb, 0xd3, 0x03, 0x82, 0x08, 0xa0, 0x16, 0x0f, 0xd7, - 0x07, 0x50, 0xdd, 0x76, 0xa2, 0x58, 0x08, 0xa8, 0x4e, 0x1d, 0x3b, 0xea, 0x97, 0x6e, 0x56, 0x96, - 0xea, 0x06, 0xb5, 0xf5, 0xa7, 0xa0, 0x8d, 0xcc, 0xe8, 0xf8, 0x85, 0xe9, 0x4e, 0xa5, 0xe8, 0x41, - 0xe5, 0xc4, 0x74, 0xfb, 0xa5, 0x9b, 0xa5, 0xa5, 0xb6, 0x81, 0x4d, 0xb1, 0x0c, 0xcd, 0x13, 0xd3, - 0x1d, 0xc7, 0x67, 0x81, 0xec, 0x97, 0x6f, 0x96, 0x96, 0xba, 0x2b, 0x97, 0x97, 0x83, 0xfd, 0xe5, - 0x67, 0x7e, 0x14, 0x3b, 0xde, 0xe1, 0xf2, 0x0b, 0xd3, 0x1d, 0x9d, 0x05, 0xd2, 0x68, 0x9c, 0x70, - 0x43, 0xdf, 0x85, 0xd6, 0x5e, 0x68, 0x6d, 0x4e, 0x3d, 0x2b, 0x76, 0x7c, 0x0f, 0x77, 0xf4, 0xcc, - 0x89, 0xa4, 0x15, 0x35, 0x83, 0xda, 0x88, 0x33, 0xc3, 0xc3, 0xa8, 0x5f, 0xb9, 0x59, 0x41, 0x1c, - 0xb6, 0x45, 0x1f, 0x1a, 0x4e, 0xb4, 0xee, 0x4f, 0xbd, 0xb8, 0x5f, 0xbd, 0x59, 0x5a, 0x6a, 0x1a, - 0x09, 0xa8, 0xff, 0x55, 0x05, 0x6a, 0xdf, 0x9b, 0xca, 0xf0, 0x8c, 0xe6, 0xc5, 0x71, 0x98, 0xac, - 0x85, 0x6d, 0x71, 0x05, 0x6a, 0xae, 0xe9, 0x1d, 0x46, 0xfd, 0x32, 0x2d, 0xc6, 0x80, 0xb8, 0x06, - 0x9a, 0x79, 0x10, 0xcb, 0x70, 0x3c, 0x75, 0xec, 0x7e, 0xe5, 0x66, 0x69, 0xa9, 0x6e, 0x34, 0x09, - 0xf1, 0xdc, 0xb1, 0xc5, 0x37, 0xa0, 0x69, 0xfb, 0x63, 0x2b, 0xbf, 0x97, 0xed, 0xd3, 0x5e, 0xe2, - 0x16, 0x34, 0xa7, 0x8e, 0x3d, 0x76, 0x9d, 0x28, 0xee, 0xd7, 0x6e, 0x96, 0x96, 0x5a, 0x2b, 0x4d, - 0xbc, 0x2c, 0xd2, 0xce, 0x68, 0x4c, 0x1d, 0x9b, 0x88, 0x78, 0x07, 0x9a, 0x51, 0x68, 0x8d, 0x0f, - 0xa6, 0x9e, 0xd5, 0xaf, 0xd3, 0xa0, 0x4b, 0x38, 0x28, 0x77, 0x6b, 0xa3, 0x11, 0x31, 0x80, 0xd7, - 0x0a, 0xe5, 0x89, 0x0c, 0x23, 0xd9, 0x6f, 0xf0, 0x56, 0x0a, 0x14, 0x0f, 0xa0, 0x75, 0x60, 0x5a, - 0x32, 0x1e, 0x07, 0x66, 0x68, 0x4e, 0xfa, 0xcd, 0x6c, 0xa1, 0x4d, 0x44, 0x3f, 0x43, 0x6c, 0x64, - 0xc0, 0x41, 0x0a, 0x88, 0x47, 0xd0, 0x21, 0x28, 0x1a, 0x1f, 0x38, 0x6e, 0x2c, 0xc3, 0xbe, 0x46, - 0x73, 0xba, 0x34, 0x87, 0x30, 0xa3, 0x50, 0x4a, 0xa3, 0xcd, 0x83, 0x18, 0x23, 0xde, 0x05, 0x90, - 0xa7, 0x81, 0xe9, 0xd9, 0x63, 0xd3, 0x75, 0xfb, 0x40, 0x67, 0xd0, 0x18, 0xb3, 0xea, 0xba, 0xe2, - 0x6d, 0x3c, 0x9f, 0x69, 0x8f, 0xe3, 0xa8, 0xdf, 0xb9, 0x59, 0x5a, 0xaa, 0x1a, 0x75, 0x04, 0x47, - 0x11, 0xd2, 0xd5, 0x32, 0xad, 0x23, 0xd9, 0xef, 0xde, 0x2c, 0x2d, 0xd5, 0x0c, 0x06, 0x10, 0x7b, - 0xe0, 0x84, 0x51, 0xdc, 0xbf, 0xc4, 0x58, 0x02, 0xc4, 0x55, 0xa8, 0xfb, 0x07, 0x07, 0x91, 0x8c, - 0xfb, 0x3d, 0x42, 0x2b, 0x48, 0x5f, 0x01, 0x8d, 0xb8, 0x8a, 0xa8, 0x76, 0x1b, 0xea, 0x27, 0x08, - 0x30, 0xf3, 0xb5, 0x56, 0x3a, 0x78, 0xec, 0x94, 0xf1, 0x0c, 0xd5, 0xa9, 0x5f, 0x87, 0xe6, 0xb6, - 0xe9, 0x1d, 0x26, 0xdc, 0x8a, 0xcf, 0x49, 0x13, 0x34, 0x83, 0xda, 0xfa, 0x1f, 0x94, 0xa1, 0x6e, - 0xc8, 0x68, 0xea, 0xc6, 0xe2, 0x43, 0x00, 0x7c, 0xac, 0x89, 0x19, 0x87, 0xce, 0xa9, 0x5a, 0x35, - 0x7b, 0x2e, 0x6d, 0xea, 0xd8, 0x4f, 0xa9, 0x4b, 0x3c, 0x80, 0x36, 0xad, 0x9e, 0x0c, 0x2d, 0x67, - 0x07, 0x48, 0xcf, 0x67, 0xb4, 0x68, 0x88, 0x9a, 0x71, 0x15, 0xea, 0xc4, 0x1f, 0xcc, 0xa3, 0x1d, - 0x43, 0x41, 0xe2, 0x36, 0x74, 0x1d, 0x2f, 0xc6, 0xf7, 0xb3, 0xe2, 0xb1, 0x2d, 0xa3, 0x84, 0x81, - 0x3a, 0x29, 0x76, 0x43, 0x46, 0xb1, 0x78, 0x08, 0xfc, 0x08, 0xc9, 0x86, 0x35, 0xda, 0xb0, 0x9b, - 0x3e, 0x6e, 0xc4, 0x3b, 0xd2, 0x18, 0xb5, 0xe3, 0x3d, 0x68, 0xe1, 0xfd, 0x92, 0x19, 0x75, 0x9a, - 0xd1, 0xa6, 0xdb, 0x28, 0x72, 0x18, 0x80, 0x03, 0xd4, 0x70, 0x24, 0x0d, 0x32, 0x29, 0x33, 0x15, - 0xb5, 0xf5, 0x21, 0xd4, 0x76, 0x43, 0x5b, 0x86, 0x73, 0xe5, 0x44, 0x40, 0xd5, 0x96, 0x91, 0x45, - 0x22, 0xdc, 0x34, 0xa8, 0x9d, 0xc9, 0x4e, 0x25, 0x27, 0x3b, 0xfa, 0x1f, 0x96, 0xa0, 0xb5, 0xe7, - 0x87, 0xf1, 0x53, 0x19, 0x45, 0xe6, 0xa1, 0x14, 0x37, 0xa0, 0xe6, 0xe3, 0xb2, 0x8a, 0xc2, 0x1a, - 0x9e, 0x89, 0xf6, 0x31, 0x18, 0x3f, 0xf3, 0x0e, 0xe5, 0x8b, 0xdf, 0x01, 0x79, 0x8a, 0xa4, 0xae, - 0xa2, 0x78, 0x8a, 0x64, 0x2e, 0xe3, 0x9e, 0x6a, 0x9e, 0x7b, 0x2e, 0x64, 0x4d, 0xfd, 0x5b, 0x00, - 0x78, 0xbe, 0xaf, 0xc9, 0x05, 0xfa, 0xcf, 0x4a, 0xd0, 0x32, 0xcc, 0x83, 0x78, 0xdd, 0xf7, 0x62, - 0x79, 0x1a, 0x8b, 0x2e, 0x94, 0x1d, 0x9b, 0x68, 0x54, 0x37, 0xca, 0x8e, 0x8d, 0xa7, 0x3b, 0x0c, - 0xfd, 0x69, 0x40, 0x24, 0xea, 0x18, 0x0c, 0x10, 0x2d, 0x6d, 0x3b, 0xa4, 0x23, 0x23, 0x2d, 0x6d, - 0x3b, 0x14, 0x37, 0xa0, 0x15, 0x79, 0x66, 0x10, 0x1d, 0xf9, 0x31, 0x9e, 0xae, 0x4a, 0xa7, 0x83, - 0x04, 0x35, 0x8a, 0x50, 0xe8, 0x9c, 0x68, 0xec, 0x4a, 0x33, 0xf4, 0x64, 0x48, 0x8a, 0xa4, 0x69, - 0x68, 0x4e, 0xb4, 0xcd, 0x08, 0xfd, 0x67, 0x15, 0xa8, 0x3f, 0x95, 0x93, 0x7d, 0x19, 0x9e, 0x3b, - 0xc4, 0x03, 0x68, 0xd2, 0xbe, 0x63, 0xc7, 0xe6, 0x73, 0xac, 0xbd, 0xf5, 0xf2, 0xcb, 0x1b, 0x8b, - 0x84, 0xdb, 0xb2, 0x3f, 0xf6, 0x27, 0x4e, 0x2c, 0x27, 0x41, 0x7c, 0x66, 0x34, 0x14, 0x6a, 0xee, - 0x01, 0xaf, 0x42, 0xdd, 0x95, 0x26, 0xbe, 0x19, 0xb3, 0xa7, 0x82, 0xc4, 0x3d, 0x68, 0x98, 0x93, - 0xb1, 0x2d, 0x4d, 0x9b, 0x0f, 0xb5, 0x76, 0xe5, 0xe5, 0x97, 0x37, 0x7a, 0xe6, 0x64, 0x43, 0x9a, - 0xf9, 0xb5, 0xeb, 0x8c, 0x11, 0x9f, 0x20, 0x4f, 0x46, 0xf1, 0x78, 0x1a, 0xd8, 0x66, 0x2c, 0x49, - 0xd7, 0x55, 0xd7, 0xfa, 0x2f, 0xbf, 0xbc, 0x71, 0x05, 0xd1, 0xcf, 0x09, 0x9b, 0x9b, 0x06, 0x19, - 0x16, 0xf5, 0x5e, 0x72, 0x7d, 0xa5, 0xf7, 0x14, 0x28, 0xb6, 0x60, 0xd1, 0x72, 0xa7, 0x11, 0x2a, - 0x67, 0xc7, 0x3b, 0xf0, 0xc7, 0xbe, 0xe7, 0x9e, 0xd1, 0x03, 0x37, 0xd7, 0xde, 0x7d, 0xf9, 0xe5, - 0x8d, 0x6f, 0xa8, 0xce, 0x2d, 0xef, 0xc0, 0xdf, 0xf5, 0xdc, 0xb3, 0xdc, 0xfa, 0x97, 0x66, 0xba, - 0xc4, 0x6f, 0x40, 0xf7, 0xc0, 0x0f, 0x2d, 0x39, 0x4e, 0x49, 0xd6, 0xa5, 0x75, 0x06, 0x2f, 0xbf, - 0xbc, 0x71, 0x95, 0x7a, 0x1e, 0x9f, 0xa3, 0x5b, 0x3b, 0x8f, 0xd7, 0xff, 0xb5, 0x0c, 0x35, 0x6a, - 0x8b, 0x07, 0xd0, 0x98, 0xd0, 0x93, 0x24, 0xfa, 0xe9, 0x2a, 0xf2, 0x10, 0xf5, 0x2d, 0xf3, 0x5b, - 0x45, 0x43, 0x2f, 0x0e, 0xcf, 0x8c, 0x64, 0x18, 0xce, 0x88, 0xcd, 0x7d, 0x57, 0xc6, 0x91, 0xe2, - 0xf9, 0xdc, 0x8c, 0x11, 0x77, 0xa8, 0x19, 0x6a, 0xd8, 0x2c, 0xdf, 0x54, 0xce, 0xf1, 0xcd, 0x00, - 0x9a, 0xd6, 0x91, 0xb4, 0x8e, 0xa3, 0xe9, 0x44, 0x71, 0x55, 0x0a, 0x8b, 0x5b, 0xd0, 0xa1, 0x76, - 0xe0, 0x3b, 0x1e, 0x4d, 0xaf, 0xd1, 0x80, 0x76, 0x86, 0x1c, 0x45, 0x83, 0x4d, 0x68, 0xe7, 0x0f, - 0x8b, 0xe6, 0xfc, 0x58, 0x9e, 0x11, 0x7f, 0x55, 0x0d, 0x6c, 0x8a, 0x9b, 0x50, 0x23, 0x45, 0x47, - 0xdc, 0xd5, 0x5a, 0x01, 0x3c, 0x33, 0x4f, 0x31, 0xb8, 0xe3, 0xd3, 0xf2, 0xb7, 0x4b, 0xb8, 0x4e, - 0xfe, 0x0a, 0xf9, 0x75, 0xb4, 0x8b, 0xd7, 0xe1, 0x29, 0xb9, 0x75, 0x74, 0x1f, 0x1a, 0xdb, 0x8e, - 0x25, 0xbd, 0x88, 0x8c, 0xfe, 0x34, 0x92, 0xa9, 0x52, 0xc2, 0x36, 0xde, 0x77, 0x62, 0x9e, 0xee, - 0xf8, 0xb6, 0x8c, 0x68, 0x9d, 0xaa, 0x91, 0xc2, 0xd8, 0x27, 0x4f, 0x03, 0x27, 0x3c, 0x1b, 0x31, - 0xa5, 0x2a, 0x46, 0x0a, 0x23, 0x77, 0x49, 0x0f, 0x37, 0xb3, 0x13, 0x03, 0xae, 0x40, 0xfd, 0x4f, - 0xab, 0xd0, 0xfe, 0x91, 0x0c, 0xfd, 0x67, 0xa1, 0x1f, 0xf8, 0x91, 0xe9, 0x8a, 0xd5, 0x22, 0xcd, - 0xf9, 0x6d, 0x6f, 0xe2, 0x69, 0xf3, 0xc3, 0x96, 0xf7, 0xd2, 0x47, 0xe0, 0x37, 0xcb, 0xbf, 0x8a, - 0x0e, 0x75, 0x7e, 0xf3, 0x39, 0x34, 0x53, 0x3d, 0x38, 0x86, 0x5f, 0x99, 0xce, 0x5a, 0xa4, 0x87, - 0xea, 0x41, 0xa9, 0x9c, 0x98, 0xa7, 0xcf, 0xb7, 0x36, 0xd4, 0xdb, 0x2a, 0x48, 0x51, 0x61, 0x74, - 0xea, 0x8d, 0x92, 0x47, 0x4d, 0x61, 0xbc, 0x29, 0x52, 0x24, 0xda, 0xda, 0xe8, 0xb7, 0xa9, 0x2b, - 0x01, 0xc5, 0x3b, 0xa0, 0x4d, 0xcc, 0x53, 0x54, 0x68, 0x5b, 0x36, 0x8b, 0xa6, 0x91, 0x21, 0xc4, - 0x7b, 0x50, 0x89, 0x4f, 0x3d, 0x92, 0x3d, 0xf4, 0x2a, 0xd0, 0xc9, 0x1c, 0x9d, 0x7a, 0x4a, 0xf5, - 0x19, 0xd8, 0x87, 0x6f, 0x6a, 0x39, 0x36, 0x39, 0x11, 0x9a, 0x81, 0x4d, 0x71, 0x1b, 0x1a, 0x2e, - 0xbf, 0x16, 0x39, 0x0a, 0xad, 0x95, 0x16, 0xeb, 0x51, 0x42, 0x19, 0x49, 0x9f, 0xf8, 0x18, 0x9a, - 0x09, 0x75, 0xfa, 0x2d, 0x1a, 0xd7, 0x4b, 0xe8, 0x99, 0x90, 0xd1, 0x48, 0x47, 0x88, 0x07, 0xa0, - 0xd9, 0xd2, 0x95, 0xb1, 0x1c, 0x7b, 0xac, 0xc8, 0x5b, 0xec, 0x40, 0x6e, 0x10, 0x72, 0x27, 0x32, - 0xe4, 0x4f, 0xa6, 0x32, 0x8a, 0x8d, 0xa6, 0xad, 0x10, 0xe2, 0xfd, 0x4c, 0xb0, 0xba, 0xf4, 0x5c, - 0x79, 0x62, 0x26, 0x5d, 0x83, 0xef, 0xc2, 0xa5, 0x99, 0x47, 0xcb, 0x73, 0x69, 0x87, 0xb9, 0xf4, - 0x4a, 0x9e, 0x4b, 0xab, 0x39, 0xce, 0xfc, 0xbc, 0xda, 0x6c, 0xf6, 0x34, 0xfd, 0xbf, 0x2a, 0x70, - 0x49, 0x09, 0xcc, 0x91, 0x13, 0xec, 0xc5, 0x4a, 0x75, 0x91, 0x61, 0x52, 0xbc, 0x5a, 0x35, 0x12, - 0x50, 0xfc, 0x3a, 0xd4, 0x49, 0xd3, 0x24, 0x02, 0x7f, 0x23, 0x63, 0x84, 0x74, 0x3a, 0x2b, 0x00, - 0xc5, 0x45, 0x6a, 0xb8, 0xf8, 0x26, 0xd4, 0xbe, 0x90, 0xa1, 0xcf, 0x86, 0xb6, 0xb5, 0x72, 0x7d, - 0xde, 0x3c, 0x24, 0x9f, 0x9a, 0xc6, 0x83, 0xff, 0xb7, 0xfc, 0x02, 0x5f, 0x87, 0x5f, 0xde, 0x47, - 0x63, 0x3b, 0xf1, 0x4f, 0xa4, 0xdd, 0x6f, 0x64, 0x34, 0x57, 0x4c, 0x9e, 0x74, 0x25, 0x2c, 0xd3, - 0x9c, 0xcb, 0x32, 0xda, 0xc5, 0x2c, 0x33, 0xd8, 0x80, 0x56, 0x8e, 0x2e, 0x73, 0x1e, 0xea, 0x46, - 0x51, 0x9d, 0x68, 0xa9, 0x2a, 0xcd, 0x6b, 0xa5, 0x0d, 0x80, 0x8c, 0x4a, 0xbf, 0xaa, 0x6e, 0xd3, - 0x7f, 0xbb, 0x04, 0x97, 0xd6, 0x7d, 0xcf, 0x93, 0xe4, 0xaa, 0xf3, 0x9b, 0x67, 0x22, 0x5e, 0xba, - 0x50, 0xc4, 0x3f, 0x82, 0x5a, 0x84, 0x83, 0xd5, 0xea, 0x97, 0xe7, 0x3c, 0xa2, 0xc1, 0x23, 0x50, - 0xd1, 0x4f, 0xcc, 0xd3, 0x71, 0x20, 0x3d, 0xdb, 0xf1, 0x0e, 0x13, 0x45, 0x3f, 0x31, 0x4f, 0x9f, - 0x31, 0x46, 0xff, 0xeb, 0x32, 0xc0, 0x67, 0xd2, 0x74, 0xe3, 0x23, 0x34, 0x66, 0xf8, 0xa2, 0x8e, - 0x17, 0xc5, 0xa6, 0x67, 0x25, 0x81, 0x52, 0x0a, 0xe3, 0x8b, 0xa2, 0x4d, 0x97, 0x11, 0xab, 0x48, - 0xcd, 0x48, 0x40, 0xe4, 0x0f, 0xdc, 0x6e, 0x1a, 0x29, 0xdb, 0xaf, 0xa0, 0xcc, 0x91, 0xa9, 0x12, - 0x5a, 0x39, 0x32, 0x7d, 0x68, 0x60, 0xe0, 0xe1, 0xf8, 0x1e, 0x31, 0x8d, 0x66, 0x24, 0x20, 0xae, - 0x33, 0x0d, 0x62, 0x67, 0xc2, 0x16, 0xbe, 0x62, 0x28, 0x08, 0x4f, 0x85, 0x16, 0x7d, 0x68, 0x1d, - 0xf9, 0xa4, 0x48, 0x2a, 0x46, 0x0a, 0xe3, 0x6a, 0xbe, 0x77, 0xe8, 0xe3, 0xed, 0x9a, 0xe4, 0x3c, - 0x26, 0x20, 0xdf, 0xc5, 0x96, 0xa7, 0xd8, 0xa5, 0x51, 0x57, 0x0a, 0x23, 0x5d, 0xa4, 0x1c, 0x1f, - 0x48, 0x33, 0x9e, 0x86, 0x32, 0xea, 0x03, 0x75, 0x83, 0x94, 0x9b, 0x0a, 0x23, 0xde, 0x83, 0x36, - 0x12, 0xce, 0x8c, 0x22, 0xe7, 0xd0, 0x93, 0x36, 0xa9, 0x97, 0xaa, 0x81, 0xc4, 0x5c, 0x55, 0x28, - 0xfd, 0x6f, 0xca, 0x50, 0x67, 0x5d, 0x50, 0x70, 0x96, 0x4a, 0x6f, 0xe4, 0x2c, 0xbd, 0x03, 0x5a, - 0x10, 0x4a, 0xdb, 0xb1, 0x92, 0x77, 0xd4, 0x8c, 0x0c, 0x41, 0xd1, 0x0d, 0x7a, 0x07, 0x44, 0xcf, - 0xa6, 0xc1, 0x80, 0xd0, 0xa1, 0xe3, 0x7b, 0x63, 0xdb, 0x89, 0x8e, 0xc7, 0xfb, 0x67, 0xb1, 0x8c, - 0x14, 0x2d, 0x5a, 0xbe, 0xb7, 0xe1, 0x44, 0xc7, 0x6b, 0x88, 0x42, 0x12, 0xb2, 0x8c, 0x90, 0x6c, - 0x34, 0x0d, 0x05, 0x89, 0x47, 0xa0, 0x91, 0x0f, 0x4b, 0x4e, 0x8e, 0x46, 0xce, 0xc9, 0xd5, 0x97, - 0x5f, 0xde, 0x10, 0x88, 0x9c, 0xf1, 0x6e, 0x9a, 0x09, 0x0e, 0xbd, 0x34, 0x9c, 0x8c, 0xe6, 0x8a, - 0x64, 0x98, 0xbd, 0x34, 0x44, 0x8d, 0xa2, 0xbc, 0x97, 0xc6, 0x18, 0x71, 0x0f, 0xc4, 0xd4, 0xb3, - 0xfc, 0x49, 0x80, 0x4c, 0x21, 0x6d, 0x75, 0xc8, 0x16, 0x1d, 0x72, 0x31, 0xdf, 0x43, 0x47, 0xd5, - 0xff, 0xa5, 0x0c, 0xed, 0x0d, 0x27, 0x94, 0x56, 0x2c, 0xed, 0xa1, 0x7d, 0x28, 0xf1, 0xec, 0xd2, - 0x8b, 0x9d, 0xf8, 0x4c, 0xb9, 0xa1, 0x0a, 0x4a, 0xa3, 0x88, 0x72, 0x31, 0xda, 0x66, 0x09, 0xab, - 0x50, 0x82, 0x80, 0x01, 0xb1, 0x02, 0xc0, 0xf1, 0x15, 0x25, 0x09, 0xaa, 0x17, 0x27, 0x09, 0x34, - 0x1a, 0x86, 0x4d, 0x0c, 0xc2, 0x79, 0x8e, 0xc3, 0xbe, 0x68, 0x9d, 0x32, 0x08, 0x53, 0xc9, 0x1e, - 0x2d, 0x85, 0x7d, 0x0d, 0xde, 0x18, 0xdb, 0xe2, 0x16, 0x94, 0xfd, 0x80, 0x88, 0xab, 0x96, 0xce, - 0x5f, 0x61, 0x79, 0x37, 0x30, 0xca, 0x7e, 0x80, 0x52, 0xcc, 0xb1, 0x2f, 0x31, 0x1e, 0x4a, 0x31, - 0xda, 0x3d, 0x8a, 0xb8, 0x0c, 0xd5, 0x23, 0x74, 0x68, 0x9b, 0xae, 0xeb, 0xff, 0x54, 0xda, 0xcf, - 0x42, 0x69, 0x27, 0x3c, 0x58, 0xc0, 0x21, 0x97, 0x78, 0xe6, 0x44, 0x46, 0x81, 0x69, 0x49, 0xc5, - 0x82, 0x19, 0x42, 0xbf, 0x0a, 0xe5, 0xdd, 0x40, 0x34, 0xa0, 0xb2, 0x37, 0x1c, 0xf5, 0x16, 0xb0, - 0xb1, 0x31, 0xdc, 0xee, 0xa1, 0x45, 0xa9, 0xf7, 0x1a, 0xfa, 0x57, 0x65, 0xd0, 0x9e, 0x4e, 0x63, - 0x13, 0x75, 0x4b, 0x84, 0xb7, 0x2c, 0x72, 0x68, 0xc6, 0x8a, 0xdf, 0x80, 0x66, 0x14, 0x9b, 0x21, - 0x79, 0x25, 0x6c, 0x9d, 0x1a, 0x04, 0x8f, 0x22, 0xf1, 0x01, 0xd4, 0xa4, 0x7d, 0x28, 0x13, 0x73, - 0xd1, 0x9b, 0xbd, 0xaf, 0xc1, 0xdd, 0x62, 0x09, 0xea, 0x91, 0x75, 0x24, 0x27, 0x66, 0xbf, 0x9a, - 0x0d, 0xdc, 0x23, 0x0c, 0xbb, 0xe1, 0x86, 0xea, 0x17, 0xef, 0x43, 0x0d, 0xdf, 0x26, 0x52, 0x71, - 0x25, 0x45, 0xa2, 0xf8, 0x0c, 0x6a, 0x18, 0x77, 0x22, 0xe3, 0xd9, 0xa1, 0x1f, 0x8c, 0xfd, 0x80, - 0x68, 0xdf, 0x5d, 0xb9, 0x42, 0x3a, 0x2e, 0xb9, 0xcd, 0xf2, 0x46, 0xe8, 0x07, 0xbb, 0x81, 0x51, - 0xb7, 0xe9, 0x17, 0xa3, 0x1c, 0x1a, 0xce, 0x1c, 0xc1, 0x46, 0x41, 0x43, 0x0c, 0xa7, 0x92, 0x96, - 0xa0, 0x39, 0x91, 0xb1, 0x69, 0x9b, 0xb1, 0xa9, 0x6c, 0x43, 0x9b, 0x55, 0x26, 0xe3, 0x8c, 0xb4, - 0x57, 0xbf, 0x0f, 0x75, 0x5e, 0x5a, 0x34, 0xa1, 0xba, 0xb3, 0xbb, 0x33, 0x64, 0xb2, 0xae, 0x6e, - 0x6f, 0xf7, 0x4a, 0x88, 0xda, 0x58, 0x1d, 0xad, 0xf6, 0xca, 0xd8, 0x1a, 0xfd, 0xf0, 0xd9, 0xb0, - 0x57, 0xd1, 0xff, 0xa1, 0x04, 0xcd, 0x64, 0x1d, 0xf1, 0x29, 0x00, 0x8a, 0xf0, 0xf8, 0xc8, 0xf1, - 0x52, 0x07, 0xef, 0x5a, 0x7e, 0xa7, 0x65, 0x7c, 0xd5, 0xcf, 0xb0, 0x97, 0xcd, 0x2b, 0x49, 0x3c, - 0xc1, 0x83, 0x3d, 0xe8, 0x16, 0x3b, 0xe7, 0x78, 0xba, 0x77, 0xf3, 0x56, 0xa5, 0xbb, 0xf2, 0x56, - 0x61, 0x69, 0x9c, 0x49, 0xac, 0x9d, 0x33, 0x30, 0xf7, 0xa0, 0x99, 0xa0, 0x45, 0x0b, 0x1a, 0x1b, - 0xc3, 0xcd, 0xd5, 0xe7, 0xdb, 0xc8, 0x2a, 0x00, 0xf5, 0xbd, 0xad, 0x9d, 0xc7, 0xdb, 0x43, 0xbe, - 0xd6, 0xf6, 0xd6, 0xde, 0xa8, 0x57, 0xd6, 0x7f, 0xbf, 0x04, 0xcd, 0xc4, 0x93, 0x11, 0x1f, 0xa1, - 0xf3, 0x41, 0x4e, 0x9a, 0xb2, 0x44, 0x94, 0x11, 0xca, 0x85, 0xad, 0x46, 0xd2, 0x8f, 0xb2, 0x48, - 0x8a, 0x35, 0xf1, 0x6d, 0x08, 0xc8, 0x47, 0xcd, 0x95, 0x42, 0x42, 0x47, 0x40, 0xd5, 0xf6, 0x3d, - 0xa9, 0x1c, 0x66, 0x6a, 0x13, 0x0f, 0x3a, 0x9e, 0x25, 0xb3, 0x70, 0xa2, 0x41, 0xf0, 0x28, 0xd2, - 0x63, 0xf6, 0xa3, 0xd3, 0x83, 0xa5, 0xbb, 0x95, 0xf2, 0xbb, 0x9d, 0x0b, 0x4a, 0xca, 0xe7, 0x83, - 0x92, 0xcc, 0x70, 0xd6, 0x5e, 0x67, 0x38, 0xf5, 0x3f, 0xab, 0x42, 0xd7, 0x90, 0x51, 0xec, 0x87, - 0x52, 0xf9, 0x85, 0xaf, 0x12, 0xa1, 0x77, 0x01, 0x42, 0x1e, 0x9c, 0x6d, 0xad, 0x29, 0x0c, 0x47, - 0x53, 0xae, 0x6f, 0x11, 0xef, 0x2a, 0x0b, 0x99, 0xc2, 0xe2, 0x1a, 0x68, 0xfb, 0xa6, 0x75, 0xcc, - 0xcb, 0xb2, 0x9d, 0x6c, 0x32, 0x82, 0xd7, 0x35, 0x2d, 0x4b, 0x46, 0xd1, 0x18, 0x59, 0x81, 0xad, - 0xa5, 0xc6, 0x98, 0x27, 0xf2, 0x0c, 0xbb, 0x23, 0x69, 0x85, 0x32, 0xa6, 0xee, 0x3a, 0x77, 0x33, - 0x06, 0xbb, 0x6f, 0x41, 0x27, 0x92, 0x11, 0x5a, 0xd6, 0x71, 0xec, 0x1f, 0x4b, 0x4f, 0xe9, 0xb1, - 0xb6, 0x42, 0x8e, 0x10, 0x87, 0x2a, 0xc6, 0xf4, 0x7c, 0xef, 0x6c, 0xe2, 0x4f, 0x23, 0x65, 0x33, - 0x32, 0x84, 0x58, 0x86, 0xcb, 0xd2, 0xb3, 0xc2, 0xb3, 0x00, 0xcf, 0x8a, 0xbb, 0x8c, 0x0f, 0x1c, - 0x57, 0x2a, 0x57, 0x7d, 0x31, 0xeb, 0x7a, 0x22, 0xcf, 0x36, 0x1d, 0x57, 0xe2, 0x89, 0x4e, 0xcc, - 0xa9, 0x1b, 0x8f, 0x29, 0x13, 0x00, 0x7c, 0x22, 0xc2, 0xac, 0xda, 0x76, 0x28, 0xee, 0xc0, 0x22, - 0x77, 0x87, 0xbe, 0x2b, 0x1d, 0x9b, 0x17, 0x6b, 0xd1, 0xa8, 0x4b, 0xd4, 0x61, 0x10, 0x9e, 0x96, - 0x5a, 0x86, 0xcb, 0x3c, 0x96, 0x2f, 0x94, 0x8c, 0x6e, 0xf3, 0xd6, 0xd4, 0xb5, 0xa7, 0x7a, 0x8a, - 0x5b, 0x07, 0x66, 0x7c, 0x44, 0xfe, 0x7d, 0xb2, 0xf5, 0x33, 0x33, 0x3e, 0x42, 0x8b, 0xcf, 0xdd, - 0x07, 0x8e, 0x74, 0x39, 0x3e, 0xd7, 0x0c, 0x9e, 0xb1, 0x89, 0x18, 0xb4, 0xf8, 0x6a, 0x80, 0x1f, - 0x4e, 0x4c, 0x4e, 0x2c, 0x6a, 0x06, 0x4f, 0xda, 0x24, 0x14, 0x6e, 0xa1, 0xde, 0xca, 0x9b, 0x4e, - 0x28, 0xc5, 0x58, 0x35, 0xd4, 0xeb, 0xed, 0x4c, 0x27, 0xfa, 0xcb, 0x0a, 0x34, 0xd3, 0x70, 0xef, - 0x2e, 0x68, 0x93, 0x44, 0x5f, 0x29, 0x47, 0xad, 0x53, 0x50, 0x62, 0x46, 0xd6, 0x2f, 0xde, 0x85, - 0xf2, 0xf1, 0x89, 0xd2, 0x9d, 0x9d, 0x65, 0x4e, 0xb4, 0x07, 0xfb, 0x8f, 0x96, 0x9f, 0xbc, 0x30, - 0xca, 0xc7, 0x27, 0x5f, 0x83, 0x6f, 0xc5, 0x87, 0x70, 0xc9, 0x72, 0xa5, 0xe9, 0x8d, 0x33, 0xef, - 0x82, 0xf9, 0xa2, 0x4b, 0xe8, 0x67, 0xa9, 0x8b, 0x71, 0x1b, 0x6a, 0xb6, 0x74, 0x63, 0x33, 0x9f, - 0xef, 0xdd, 0x0d, 0x4d, 0xcb, 0x95, 0x1b, 0x88, 0x36, 0xb8, 0x17, 0x75, 0x67, 0x1a, 0x62, 0xe5, - 0x74, 0xe7, 0x9c, 0xf0, 0x2a, 0x95, 0x4b, 0xc8, 0xcb, 0xe5, 0x5d, 0x58, 0x94, 0xa7, 0x01, 0x19, - 0x8c, 0x71, 0x9a, 0x51, 0x60, 0x4b, 0xd6, 0x4b, 0x3a, 0xd6, 0x93, 0xcc, 0xc2, 0xc7, 0xa8, 0x32, - 0x48, 0x68, 0xe8, 0x99, 0x5b, 0x2b, 0x82, 0x74, 0x4e, 0x41, 0x0c, 0x8d, 0x64, 0x88, 0xf8, 0x08, - 0x34, 0xcb, 0xb6, 0xc6, 0x4c, 0x99, 0x4e, 0x76, 0xb6, 0xf5, 0x8d, 0x75, 0x26, 0x49, 0xd3, 0xb2, - 0x2d, 0xf6, 0xaa, 0x0b, 0xa1, 0x5f, 0xf7, 0x4d, 0x42, 0xbf, 0xbc, 0x51, 0xec, 0x15, 0x8c, 0xe2, - 0xe7, 0xd5, 0x66, 0xa3, 0xd7, 0xd4, 0x6f, 0x41, 0x33, 0xd9, 0x08, 0x55, 0x5d, 0x24, 0x3d, 0x15, - 0xd6, 0x93, 0xaa, 0x43, 0x70, 0x14, 0xe9, 0x16, 0x54, 0x9e, 0xbc, 0xd8, 0x23, 0x8d, 0x87, 0xc6, - 0xa7, 0x46, 0xbe, 0x0a, 0xb5, 0x53, 0x2d, 0x58, 0xce, 0x69, 0xc1, 0xeb, 0x6c, 0x40, 0xe8, 0x81, - 0x92, 0x5c, 0x68, 0x0e, 0x83, 0x24, 0x66, 0xe3, 0x59, 0xe5, 0x34, 0x29, 0x01, 0xfa, 0x7f, 0x54, - 0xa0, 0xa1, 0xfc, 0x1b, 0x34, 0x1a, 0xd3, 0x34, 0x8d, 0x87, 0xcd, 0x62, 0xe0, 0x99, 0x3a, 0x4a, - 0xf9, 0x5a, 0x4a, 0xe5, 0xf5, 0xb5, 0x14, 0xf1, 0x29, 0xb4, 0x03, 0xee, 0xcb, 0xbb, 0x56, 0x6f, - 0xe7, 0xe7, 0xa8, 0x5f, 0x9a, 0xd7, 0x0a, 0x32, 0x00, 0x49, 0x49, 0x09, 0xe5, 0xd8, 0x3c, 0x54, - 0x14, 0x68, 0x20, 0x3c, 0x32, 0x0f, 0xdf, 0xc8, 0x4f, 0xea, 0x92, 0xc3, 0xd5, 0x26, 0x85, 0x8b, - 0xbe, 0x55, 0xfe, 0x65, 0x3a, 0x45, 0x77, 0xe5, 0x1a, 0x68, 0x96, 0x3f, 0x99, 0x38, 0xd4, 0xd7, - 0x55, 0x69, 0x2b, 0x42, 0x8c, 0x22, 0xfd, 0x77, 0x4b, 0xd0, 0x50, 0xf7, 0x3a, 0x67, 0x0c, 0xd7, - 0xb6, 0x76, 0x56, 0x8d, 0x1f, 0xf6, 0x4a, 0x68, 0xec, 0xb7, 0x76, 0x46, 0xbd, 0xb2, 0xd0, 0xa0, - 0xb6, 0xb9, 0xbd, 0xbb, 0x3a, 0xea, 0x55, 0xd0, 0x40, 0xae, 0xed, 0xee, 0x6e, 0xf7, 0xaa, 0xa2, - 0x0d, 0xcd, 0x8d, 0xd5, 0xd1, 0x70, 0xb4, 0xf5, 0x74, 0xd8, 0xab, 0xe1, 0xd8, 0xc7, 0xc3, 0xdd, - 0x5e, 0x1d, 0x1b, 0xcf, 0xb7, 0x36, 0x7a, 0x0d, 0xec, 0x7f, 0xb6, 0xba, 0xb7, 0xf7, 0xfd, 0x5d, - 0x63, 0xa3, 0xd7, 0x24, 0x23, 0x3b, 0x32, 0xb6, 0x76, 0x1e, 0xf7, 0x34, 0x6c, 0xef, 0xae, 0x7d, - 0x3e, 0x5c, 0x1f, 0xf5, 0x40, 0x7f, 0x08, 0xad, 0x1c, 0xad, 0x70, 0xb6, 0x31, 0xdc, 0xec, 0x2d, - 0xe0, 0x96, 0x2f, 0x56, 0xb7, 0x9f, 0xa3, 0x4d, 0xee, 0x02, 0x50, 0x73, 0xbc, 0xbd, 0xba, 0xf3, - 0xb8, 0x57, 0x56, 0x1e, 0xdd, 0xf7, 0xa0, 0xf9, 0xdc, 0xb1, 0xd7, 0x5c, 0xdf, 0x3a, 0x46, 0xf6, - 0xd9, 0x37, 0x23, 0xa9, 0xf8, 0x8d, 0xda, 0xe8, 0x3f, 0x93, 0xd0, 0x46, 0xea, 0xad, 0x15, 0x84, - 0x14, 0xf3, 0xa6, 0x93, 0x31, 0xd5, 0xdb, 0x2a, 0x6c, 0xb8, 0xbc, 0xe9, 0xe4, 0xb9, 0x63, 0x47, - 0xfa, 0x31, 0x34, 0x9e, 0x3b, 0xf6, 0x33, 0xd3, 0x3a, 0x26, 0xe5, 0x86, 0x4b, 0x8f, 0x23, 0xe7, - 0x0b, 0xa9, 0x0c, 0x9c, 0x46, 0x98, 0x3d, 0xe7, 0x0b, 0x29, 0xde, 0x87, 0x3a, 0x01, 0x49, 0xca, - 0x81, 0x44, 0x2d, 0x39, 0x8e, 0xa1, 0xfa, 0xa8, 0xdc, 0xe5, 0xba, 0xbe, 0x35, 0x0e, 0xe5, 0x41, - 0xff, 0x6d, 0x7e, 0x01, 0x42, 0x18, 0xf2, 0x40, 0xff, 0xbd, 0x52, 0x7a, 0x73, 0xaa, 0xaa, 0xdc, - 0x80, 0x6a, 0x60, 0x5a, 0xc7, 0xca, 0xbf, 0x68, 0xa9, 0x05, 0xf1, 0x30, 0x06, 0x75, 0x88, 0x0f, - 0xa1, 0xa9, 0x18, 0x29, 0xd9, 0xb5, 0x95, 0xe3, 0x38, 0x23, 0xed, 0x2c, 0x3e, 0x7c, 0xa5, 0xf8, - 0xf0, 0x14, 0x9d, 0x06, 0xae, 0x13, 0xb3, 0xd8, 0xa0, 0x70, 0x12, 0xa4, 0x7f, 0x13, 0x20, 0x2b, - 0x70, 0xcd, 0x71, 0xb7, 0xae, 0x40, 0xcd, 0x74, 0x1d, 0x33, 0x89, 0x76, 0x19, 0xd0, 0x77, 0xa0, - 0x95, 0x2b, 0x8b, 0x21, 0x6d, 0x4d, 0xd7, 0x45, 0xcb, 0xc8, 0xb2, 0xdf, 0x34, 0x1a, 0xa6, 0xeb, - 0x3e, 0x91, 0x67, 0x11, 0xba, 0xba, 0x5c, 0x51, 0x2b, 0xcf, 0x14, 0x5d, 0x68, 0xaa, 0xc1, 0x9d, - 0xfa, 0xc7, 0x50, 0xdf, 0x4c, 0x02, 0x82, 0x44, 0x18, 0x4a, 0x17, 0x09, 0x83, 0xfe, 0x89, 0x3a, - 0x33, 0xd5, 0x6d, 0xc4, 0x5d, 0x55, 0xb9, 0x8b, 0xb8, 0x4e, 0x58, 0xca, 0xf2, 0x25, 0x3c, 0x48, - 0x15, 0xed, 0x68, 0xb0, 0xbe, 0x01, 0xcd, 0x57, 0xd6, 0x42, 0x15, 0x01, 0xca, 0x19, 0x01, 0xe6, - 0x54, 0x47, 0xf5, 0x1f, 0x03, 0x64, 0x15, 0x3e, 0x25, 0x9b, 0xbc, 0x0a, 0xca, 0xe6, 0x1d, 0x68, - 0x5a, 0x47, 0x8e, 0x6b, 0x87, 0xd2, 0x2b, 0xdc, 0x3a, 0xab, 0x09, 0xa6, 0xfd, 0xe2, 0x26, 0x54, - 0xa9, 0x70, 0x59, 0xc9, 0x34, 0x77, 0x5a, 0xb5, 0xa4, 0x1e, 0xfd, 0x14, 0x3a, 0x1c, 0x43, 0xbc, - 0x81, 0x07, 0x56, 0x54, 0x9d, 0xe5, 0x73, 0xaa, 0xf3, 0x2a, 0xd4, 0xc9, 0xf0, 0x27, 0xb7, 0x51, - 0xd0, 0x05, 0x2a, 0xf5, 0x77, 0xca, 0x00, 0xbc, 0xf5, 0x8e, 0x6f, 0xcb, 0x62, 0xb0, 0x5e, 0x9a, - 0x0d, 0xd6, 0x05, 0x54, 0xd3, 0x9a, 0xb4, 0x66, 0x50, 0x3b, 0x33, 0x86, 0x2a, 0x80, 0x67, 0x63, - 0xf8, 0x0e, 0x68, 0xe4, 0x88, 0x39, 0x5f, 0x50, 0x41, 0x04, 0x37, 0xcc, 0x10, 0xf9, 0x0a, 0x6d, - 0xad, 0x58, 0xa1, 0x4d, 0xcb, 0x55, 0x75, 0x5e, 0x8d, 0xcb, 0x55, 0x73, 0x2a, 0x6f, 0x9c, 0x41, - 0x89, 0x64, 0x18, 0x27, 0xe1, 0x3f, 0x43, 0x69, 0x24, 0xab, 0xa9, 0xb1, 0x26, 0xe7, 0x40, 0x3c, - 0x7f, 0x6c, 0xf9, 0xde, 0x81, 0xeb, 0x58, 0xb1, 0xaa, 0xc8, 0x82, 0xe7, 0xaf, 0x2b, 0x8c, 0xfe, - 0x29, 0xb4, 0x13, 0xfa, 0x53, 0x81, 0xeb, 0x4e, 0x1a, 0xe5, 0x95, 0xb2, 0xb7, 0xcd, 0xc8, 0xb4, - 0x56, 0xee, 0x97, 0x92, 0x38, 0x4f, 0xff, 0xef, 0x4a, 0x32, 0x59, 0xd5, 0x61, 0x5e, 0x4d, 0xc3, - 0x62, 0xe0, 0x5e, 0x7e, 0xa3, 0xc0, 0xfd, 0xdb, 0xa0, 0xd9, 0x14, 0x8b, 0x3a, 0x27, 0x89, 0x11, - 0x1b, 0xcc, 0xc6, 0x9d, 0x2a, 0x5a, 0x75, 0x4e, 0xa4, 0x91, 0x0d, 0x7e, 0xcd, 0x3b, 0xa4, 0xd4, - 0xae, 0xcd, 0xa3, 0x76, 0xfd, 0x57, 0xa4, 0xf6, 0x7b, 0xd0, 0xf6, 0x7c, 0x6f, 0xec, 0x4d, 0x5d, - 0xd7, 0xdc, 0x77, 0xa5, 0x22, 0x77, 0xcb, 0xf3, 0xbd, 0x1d, 0x85, 0x42, 0xef, 0x38, 0x3f, 0x84, - 0x85, 0xba, 0x45, 0xe3, 0x2e, 0xe5, 0xc6, 0x91, 0xe8, 0x2f, 0x41, 0xcf, 0xdf, 0xff, 0xb1, 0xb4, - 0x62, 0xa2, 0xd8, 0x98, 0xa4, 0x99, 0x5d, 0xe3, 0x2e, 0xe3, 0x91, 0x44, 0x3b, 0x28, 0xd7, 0x33, - 0xcf, 0xdc, 0x39, 0xf7, 0xcc, 0x9f, 0x80, 0x96, 0x52, 0x29, 0x17, 0xf7, 0x6a, 0x50, 0xdb, 0xda, - 0xd9, 0x18, 0xfe, 0xa0, 0x57, 0x42, 0x73, 0x69, 0x0c, 0x5f, 0x0c, 0x8d, 0xbd, 0x61, 0xaf, 0x8c, - 0xa6, 0x6c, 0x63, 0xb8, 0x3d, 0x1c, 0x0d, 0x7b, 0x15, 0x76, 0x85, 0xa8, 0x1c, 0xe2, 0x3a, 0x96, - 0x13, 0xeb, 0x7b, 0x00, 0x59, 0x30, 0x8f, 0x5a, 0x39, 0x3b, 0x9c, 0xca, 0x26, 0xc6, 0xc9, 0xb1, - 0x96, 0x52, 0x81, 0x2c, 0x5f, 0x94, 0x32, 0xe0, 0x7e, 0x7d, 0x05, 0xb4, 0xa7, 0x66, 0xf0, 0x19, - 0x17, 0x0e, 0x6f, 0x43, 0x37, 0x30, 0xc3, 0xd8, 0x49, 0xe2, 0x11, 0x56, 0x96, 0x6d, 0xa3, 0x93, - 0x62, 0x51, 0xf7, 0xea, 0x7f, 0x5e, 0x82, 0x2b, 0x4f, 0xfd, 0x13, 0x99, 0xfa, 0xbb, 0xcf, 0xcc, - 0x33, 0xd7, 0x37, 0xed, 0xd7, 0xb0, 0x21, 0x06, 0x54, 0xfe, 0x94, 0x0a, 0x79, 0x49, 0xd9, 0xd3, - 0xd0, 0x18, 0xf3, 0x58, 0x7d, 0xaf, 0x21, 0xa3, 0x98, 0x3a, 0x95, 0x21, 0x45, 0x18, 0xbb, 0xde, - 0x82, 0x7a, 0x7c, 0xea, 0x65, 0x45, 0xd8, 0x5a, 0x4c, 0x59, 0xf0, 0xb9, 0xee, 0x6f, 0x6d, 0xbe, - 0xfb, 0xab, 0xaf, 0x83, 0x36, 0x3a, 0xa5, 0x3c, 0xf0, 0xb4, 0xe8, 0x80, 0x96, 0x5e, 0xe1, 0xe6, - 0x94, 0x67, 0xdc, 0x9c, 0x7f, 0x2f, 0x41, 0x2b, 0xe7, 0xc7, 0x8b, 0xf7, 0xa0, 0x1a, 0x9f, 0x7a, - 0xc5, 0x6f, 0x1d, 0x92, 0x4d, 0x0c, 0xea, 0x3a, 0x97, 0xeb, 0x2c, 0x9f, 0xcb, 0x75, 0x8a, 0x6d, - 0xb8, 0xc4, 0x9a, 0x37, 0xb9, 0x44, 0x92, 0x12, 0xba, 0x35, 0x13, 0x37, 0x70, 0xae, 0x3c, 0xb9, - 0x92, 0xca, 0x73, 0x74, 0x0f, 0x0b, 0xc8, 0xc1, 0x2a, 0x5c, 0x9e, 0x33, 0xec, 0xeb, 0x54, 0x4d, - 0xf4, 0x1b, 0xd0, 0x19, 0x9d, 0x7a, 0x23, 0x67, 0x22, 0xa3, 0xd8, 0x9c, 0x04, 0xe4, 0x26, 0x2a, - 0xcb, 0x59, 0x35, 0xca, 0x71, 0xa4, 0x7f, 0x00, 0xed, 0x67, 0x52, 0x86, 0x86, 0x8c, 0x02, 0xdf, - 0x63, 0xe7, 0x48, 0xe5, 0xa8, 0xd9, 0x4c, 0x2b, 0x48, 0xff, 0x2d, 0xd0, 0x0c, 0xf3, 0x20, 0x5e, - 0x33, 0x63, 0xeb, 0xe8, 0xeb, 0x24, 0x3d, 0x3e, 0x80, 0x46, 0xc0, 0x3c, 0xa5, 0xa2, 0xbb, 0x36, - 0x99, 0x6b, 0xc5, 0x67, 0x46, 0xd2, 0xa9, 0xff, 0x1a, 0x74, 0x55, 0xc1, 0x28, 0x39, 0x49, 0xae, - 0xaa, 0x54, 0xba, 0xb0, 0xaa, 0xa4, 0x1f, 0x42, 0x27, 0x99, 0xc7, 0xc6, 0xef, 0x8d, 0xa6, 0x7d, - 0xfd, 0xb2, 0xbd, 0xfe, 0x9b, 0x70, 0x79, 0x6f, 0xba, 0x1f, 0x59, 0xa1, 0x43, 0x91, 0x7c, 0xb2, - 0xdd, 0x00, 0x9a, 0x41, 0x28, 0x0f, 0x9c, 0x53, 0x99, 0x88, 0x58, 0x0a, 0x8b, 0x3b, 0xd0, 0x98, - 0x20, 0xbd, 0x64, 0x26, 0xbc, 0x59, 0xcc, 0xfa, 0x14, 0x7b, 0x8c, 0x64, 0x80, 0xfe, 0x1d, 0xb8, - 0x52, 0x5c, 0x5e, 0x51, 0xe1, 0x16, 0x54, 0x8e, 0x4f, 0x22, 0x45, 0xe6, 0xc5, 0x42, 0xcc, 0x4b, - 0xdf, 0x4b, 0x60, 0xaf, 0xfe, 0xc7, 0x25, 0xa8, 0xec, 0x4c, 0x27, 0xf9, 0x8f, 0xc1, 0xaa, 0xfc, - 0x31, 0xd8, 0xb5, 0x7c, 0x3e, 0x9b, 0x63, 0xa8, 0x2c, 0x6f, 0xfd, 0x0e, 0x68, 0x07, 0x7e, 0xf8, - 0x53, 0x33, 0xb4, 0xa5, 0xad, 0x2c, 0x70, 0x86, 0x10, 0xb7, 0x95, 0xbd, 0xe6, 0x18, 0x66, 0x11, - 0xa9, 0xb8, 0x33, 0x9d, 0x2c, 0xbb, 0xd2, 0x8c, 0xc8, 0xb0, 0xb0, 0x09, 0xd7, 0xef, 0x82, 0x96, - 0xa2, 0x50, 0x19, 0xee, 0xec, 0x8d, 0xb7, 0x36, 0x38, 0x1f, 0x88, 0xde, 0x7e, 0x09, 0x15, 0xe1, - 0xe8, 0x07, 0x3b, 0xe3, 0xd1, 0x5e, 0xaf, 0xac, 0xff, 0x08, 0x5a, 0x89, 0xac, 0x6c, 0xd9, 0x54, - 0xfc, 0x22, 0x61, 0xdd, 0xb2, 0x0b, 0xb2, 0xbb, 0x45, 0xe1, 0x98, 0xf4, 0xec, 0xad, 0x44, 0xc8, - 0x18, 0x28, 0xde, 0x46, 0x55, 0xd2, 0x92, 0xdb, 0xe8, 0x43, 0x58, 0x34, 0x28, 0x89, 0x8f, 0x46, - 0x36, 0x79, 0x9e, 0xab, 0x50, 0xf7, 0x7c, 0x5b, 0xa6, 0x1b, 0x28, 0x08, 0x77, 0x56, 0x0f, 0xab, - 0xd4, 0x57, 0xfa, 0xce, 0x12, 0x16, 0x51, 0x23, 0x16, 0x99, 0xaa, 0x90, 0x60, 0x2e, 0xcd, 0x24, - 0x98, 0x71, 0x13, 0x55, 0x4b, 0x66, 0xdf, 0x26, 0xa9, 0x1f, 0x0f, 0xa0, 0x69, 0x47, 0x31, 0x89, - 0xb0, 0xd2, 0x83, 0x29, 0xac, 0xdf, 0x87, 0xcb, 0xab, 0x41, 0xe0, 0x9e, 0x25, 0x95, 0x37, 0xb5, - 0x51, 0x3f, 0x2b, 0xcf, 0x95, 0x54, 0x0c, 0xc8, 0xa0, 0xbe, 0x09, 0xed, 0x24, 0x9b, 0xf0, 0x54, - 0xc6, 0x26, 0x69, 0x37, 0xd7, 0x29, 0x84, 0xd3, 0x4d, 0x46, 0x8c, 0x8a, 0x69, 0xec, 0x99, 0xfb, - 0x2d, 0x43, 0x5d, 0xa9, 0x4e, 0x01, 0x55, 0xcb, 0xb7, 0x79, 0xa3, 0x9a, 0x41, 0x6d, 0xe4, 0xa0, - 0x49, 0x74, 0x98, 0x78, 0xb7, 0x93, 0xe8, 0x50, 0xff, 0xa7, 0x32, 0x74, 0xd6, 0x28, 0x77, 0x93, - 0x9c, 0x31, 0x97, 0xb1, 0x2c, 0x15, 0x32, 0x96, 0xf9, 0xec, 0x64, 0xb9, 0x90, 0x9d, 0x2c, 0x1c, - 0xa8, 0x52, 0x74, 0x49, 0xdf, 0x86, 0xc6, 0xd4, 0x73, 0x4e, 0x13, 0x9b, 0xa0, 0x19, 0x75, 0x04, - 0x47, 0x91, 0xb8, 0x09, 0x2d, 0x34, 0x1b, 0x8e, 0xc7, 0x19, 0x41, 0x4e, 0xeb, 0xe5, 0x51, 0x33, - 0x79, 0xbf, 0xfa, 0xab, 0xf3, 0x7e, 0x8d, 0xd7, 0xe6, 0xfd, 0x9a, 0xaf, 0xcb, 0xfb, 0x69, 0xb3, - 0x79, 0xbf, 0xa2, 0x3b, 0x0d, 0xe7, 0xdc, 0xe9, 0x77, 0x01, 0xf8, 0x83, 0x97, 0x83, 0xa9, 0xeb, - 0x2a, 0x1f, 0x45, 0x23, 0xcc, 0xe6, 0xd4, 0x75, 0xf5, 0x6d, 0xe8, 0x26, 0xa4, 0x55, 0xe2, 0xfe, - 0x29, 0x5c, 0x52, 0x19, 0x7d, 0x19, 0xaa, 0xa4, 0x18, 0x6b, 0x31, 0x92, 0x3f, 0x4e, 0xba, 0xab, - 0x1e, 0xa3, 0x6b, 0xe7, 0xc1, 0x48, 0xff, 0x79, 0x09, 0x3a, 0x85, 0x11, 0xe2, 0x61, 0x56, 0x1f, - 0x28, 0x91, 0x14, 0xf7, 0xcf, 0xad, 0xf2, 0xea, 0x1a, 0x41, 0x79, 0xa6, 0x46, 0xa0, 0xdf, 0x4b, - 0x33, 0xff, 0x2a, 0xdf, 0xbf, 0x90, 0xe6, 0xfb, 0x29, 0x45, 0xbe, 0x3a, 0x1a, 0x19, 0xbd, 0xb2, - 0xa8, 0x43, 0x79, 0x67, 0xaf, 0x57, 0xd1, 0xff, 0xb2, 0x0c, 0x9d, 0xe1, 0x69, 0x40, 0x1f, 0x7f, - 0xbd, 0x36, 0x36, 0xc9, 0xf1, 0x55, 0xb9, 0xc0, 0x57, 0x39, 0x0e, 0xa9, 0xa8, 0x82, 0x27, 0x73, - 0x08, 0x46, 0x2b, 0x9c, 0x85, 0x54, 0x9c, 0xc3, 0xd0, 0xff, 0x07, 0xce, 0x29, 0x68, 0x14, 0x98, - 0x2d, 0x59, 0x6d, 0x43, 0x37, 0x21, 0x9b, 0x62, 0x8c, 0x37, 0x12, 0x56, 0xfe, 0xdc, 0xd3, 0x4d, - 0x93, 0x62, 0x0c, 0xe8, 0x7f, 0x52, 0x06, 0x8d, 0xf9, 0x0c, 0x0f, 0xff, 0x91, 0xd2, 0xeb, 0xa5, - 0xac, 0x3a, 0x92, 0x76, 0x2e, 0x3f, 0x91, 0x67, 0x99, 0x6e, 0x9f, 0x5b, 0x51, 0x54, 0xa9, 0x33, - 0xce, 0x1e, 0x50, 0xea, 0xec, 0x1a, 0x68, 0xec, 0x82, 0x4d, 0x55, 0x6a, 0xbe, 0x6a, 0xb0, 0x4f, - 0xf6, 0xdc, 0xa1, 0xda, 0x60, 0x2c, 0xc3, 0x89, 0x7a, 0x03, 0x6a, 0x17, 0xe3, 0xb4, 0x4e, 0x12, - 0x39, 0x14, 0x28, 0xd2, 0x98, 0xa5, 0xc8, 0x11, 0x34, 0xd4, 0xd9, 0xd0, 0xcd, 0x7e, 0xbe, 0xf3, - 0x64, 0x67, 0xf7, 0xfb, 0x3b, 0x05, 0xee, 0x4b, 0x1d, 0xf1, 0x72, 0xde, 0x11, 0xaf, 0x20, 0x7e, - 0x7d, 0xf7, 0xf9, 0xce, 0xa8, 0x57, 0x15, 0x1d, 0xd0, 0xa8, 0x39, 0x36, 0x86, 0x2f, 0x7a, 0x35, - 0xca, 0x3c, 0xad, 0x7f, 0x36, 0x7c, 0xba, 0xda, 0xab, 0xa7, 0xb5, 0xaa, 0x86, 0xfe, 0x47, 0x25, - 0x58, 0x64, 0x82, 0xe4, 0x93, 0x30, 0xf9, 0x0f, 0xb1, 0xab, 0xfc, 0x21, 0xf6, 0xff, 0x6d, 0xde, - 0x05, 0x27, 0x4d, 0x9d, 0xa4, 0x3a, 0xcc, 0x09, 0xc1, 0xe6, 0xd4, 0x51, 0x45, 0xe1, 0xbf, 0x2b, - 0xc1, 0x80, 0xfd, 0xff, 0xc7, 0xa1, 0x19, 0x1c, 0x7d, 0x6f, 0xfb, 0x5c, 0x06, 0xe0, 0x22, 0xaf, - 0xf8, 0x36, 0x74, 0xe9, 0x53, 0xf5, 0x9f, 0xb8, 0x63, 0x15, 0xa5, 0xf2, 0xeb, 0x76, 0x14, 0x96, - 0x17, 0x12, 0x8f, 0xa0, 0xcd, 0x9f, 0xb4, 0x53, 0x86, 0xbc, 0x50, 0xd9, 0x2c, 0x44, 0x1f, 0x2d, - 0x1e, 0xc5, 0x75, 0xd8, 0x87, 0xe9, 0xa4, 0x2c, 0x59, 0x70, 0xbe, 0x78, 0xa9, 0xa6, 0x8c, 0x28, - 0x85, 0x70, 0x1f, 0xae, 0xcd, 0xbd, 0x87, 0x62, 0xfb, 0x5c, 0xa2, 0x96, 0xb9, 0x4d, 0xff, 0xe7, - 0x12, 0x34, 0xd7, 0xa6, 0xee, 0x31, 0x19, 0xc1, 0x77, 0x01, 0xa4, 0x7d, 0x28, 0xd5, 0xb7, 0xe1, - 0x25, 0x52, 0x0e, 0x1a, 0x62, 0xf8, 0xeb, 0xf0, 0x4f, 0x01, 0xf8, 0x8e, 0xe3, 0x89, 0x19, 0xa8, - 0x27, 0xa2, 0x4a, 0x63, 0xb2, 0x80, 0xba, 0xcb, 0x53, 0x33, 0x50, 0x95, 0xc6, 0x28, 0x81, 0xb3, - 0x0a, 0x6c, 0xe5, 0x15, 0x15, 0xd8, 0xc1, 0x0e, 0x74, 0x8b, 0x4b, 0xcc, 0x49, 0x90, 0x7d, 0x50, - 0xfc, 0xca, 0xe5, 0x3c, 0x0d, 0x73, 0xfe, 0xfa, 0xe7, 0x70, 0x69, 0x26, 0xd9, 0xfe, 0x2a, 0x8d, - 0x59, 0x10, 0x99, 0xf2, 0xac, 0xc8, 0x7c, 0x0c, 0x8b, 0x23, 0x33, 0x3a, 0x56, 0x31, 0x4c, 0x66, - 0xbc, 0x63, 0x33, 0x3a, 0x1e, 0xa7, 0x44, 0xad, 0x23, 0xb8, 0x65, 0xeb, 0x0f, 0x41, 0xe4, 0x47, - 0x2b, 0xfa, 0x63, 0x6c, 0x8a, 0xc3, 0x27, 0x32, 0x36, 0x13, 0x2f, 0x03, 0x11, 0x48, 0xbc, 0x95, - 0xbf, 0x2d, 0x41, 0x15, 0x9d, 0x7e, 0x71, 0x0f, 0xb4, 0xcf, 0xa4, 0x19, 0xc6, 0xfb, 0xd2, 0x8c, - 0x45, 0xc1, 0xc1, 0x1f, 0x10, 0xdd, 0xb2, 0x2f, 0x67, 0xf4, 0x85, 0x07, 0x25, 0xb1, 0xcc, 0xdf, - 0xf5, 0x26, 0xdf, 0x2b, 0x77, 0x92, 0xe0, 0x81, 0x82, 0x8b, 0x41, 0x61, 0xbe, 0xbe, 0xb0, 0x44, - 0xe3, 0x3f, 0xf7, 0x1d, 0x6f, 0x9d, 0xbf, 0x26, 0x15, 0xb3, 0xc1, 0xc6, 0xec, 0x0c, 0x71, 0x0f, - 0xea, 0x5b, 0x11, 0x46, 0x35, 0xe7, 0x87, 0x12, 0xf1, 0xf3, 0x01, 0x8f, 0xbe, 0xb0, 0xf2, 0x17, - 0x35, 0xa8, 0xfe, 0x48, 0x86, 0xbe, 0xf8, 0x18, 0x1a, 0xea, 0x3b, 0x23, 0x91, 0xfb, 0x9e, 0x68, - 0x40, 0x09, 0x96, 0x99, 0x0f, 0x90, 0x68, 0x97, 0x1e, 0xbf, 0x5f, 0x56, 0x62, 0x12, 0xd9, 0x67, - 0x50, 0xe7, 0x0e, 0xf5, 0x09, 0xf4, 0xf6, 0xe2, 0x50, 0x9a, 0x93, 0xdc, 0xf0, 0x22, 0xa9, 0xe6, - 0xd5, 0xab, 0x88, 0x5e, 0x77, 0xa1, 0xce, 0xa1, 0xe3, 0xcc, 0x84, 0xd9, 0x62, 0x14, 0x0d, 0xfe, - 0x10, 0x5a, 0x7b, 0x47, 0xfe, 0xd4, 0xb5, 0xf7, 0x64, 0x78, 0x22, 0x45, 0x2e, 0xfa, 0x19, 0xe4, - 0xda, 0xfa, 0x82, 0x78, 0x08, 0x75, 0x7c, 0x91, 0x70, 0x22, 0x16, 0x73, 0x11, 0x12, 0xb3, 0xc9, - 0x40, 0xe4, 0x51, 0x09, 0xa5, 0xc4, 0x87, 0xa0, 0xb1, 0xfb, 0x8e, 0xce, 0x7b, 0x43, 0x45, 0x04, - 0x7c, 0x8c, 0x9c, 0x5b, 0xaf, 0x2f, 0x88, 0x25, 0x80, 0x5c, 0xcc, 0xf9, 0xaa, 0x91, 0x8f, 0xa0, - 0xb3, 0x4e, 0x9a, 0x70, 0x37, 0x5c, 0xdd, 0xf7, 0xc3, 0x58, 0xcc, 0x7e, 0xfb, 0x38, 0x98, 0x45, - 0xe8, 0x0b, 0x18, 0xbd, 0x8d, 0xc2, 0x33, 0x1e, 0xbf, 0xa8, 0x42, 0xf5, 0x6c, 0xbf, 0x39, 0x74, - 0x11, 0xdf, 0x4c, 0xe5, 0x2a, 0xf5, 0xda, 0xe7, 0x55, 0xb6, 0x98, 0x44, 0x2c, 0x03, 0x44, 0x22, - 0xc8, 0x42, 0x0a, 0xf1, 0x16, 0x57, 0xd9, 0x66, 0x42, 0x8c, 0xf3, 0x53, 0xb2, 0xf0, 0x81, 0xa7, - 0x9c, 0x0b, 0x27, 0x66, 0xa6, 0x7c, 0x0b, 0xda, 0xf9, 0x50, 0x40, 0x50, 0xb9, 0x68, 0x4e, 0x70, - 0x50, 0x9c, 0xb6, 0xf2, 0x9f, 0x35, 0xa8, 0x7f, 0xdf, 0x0f, 0x8f, 0x65, 0x28, 0xee, 0x40, 0x9d, - 0xea, 0xa5, 0x4a, 0x96, 0xd2, 0xda, 0xe9, 0x3c, 0xda, 0xbd, 0x0f, 0x1a, 0x71, 0x06, 0x0a, 0x3b, - 0xf3, 0x2b, 0xfd, 0x57, 0x87, 0x17, 0xe7, 0x0c, 0x26, 0x31, 0x77, 0x97, 0xb9, 0x35, 0xfd, 0x9e, - 0xa0, 0x50, 0xcf, 0x1c, 0xd0, 0x93, 0x3e, 0x79, 0xb1, 0x87, 0xf2, 0xf9, 0xa0, 0x84, 0x3e, 0xc5, - 0x1e, 0x3f, 0x1e, 0x0e, 0xca, 0xfe, 0x8b, 0xc0, 0xe2, 0x9f, 0x7d, 0xfc, 0xaf, 0x2f, 0x88, 0xfb, - 0x50, 0x57, 0x26, 0x66, 0x31, 0x53, 0x84, 0xc9, 0x0d, 0x7b, 0x79, 0x94, 0x9a, 0xf0, 0x10, 0xea, - 0x6c, 0x8e, 0x79, 0x42, 0x21, 0x16, 0x61, 0x3e, 0x2d, 0xfa, 0xd0, 0xfa, 0x82, 0xb8, 0x0b, 0x0d, - 0x55, 0x0d, 0x15, 0x73, 0x4a, 0xa3, 0xe7, 0x5e, 0xac, 0xce, 0xbe, 0x16, 0xaf, 0x5f, 0x70, 0x57, - 0x79, 0xfd, 0xa2, 0x2b, 0xc6, 0xa2, 0x6f, 0x48, 0x4b, 0x3a, 0xb9, 0xc4, 0x99, 0x48, 0x28, 0x32, - 0x47, 0x7f, 0x7d, 0x02, 0x9d, 0x42, 0x92, 0x4d, 0xf4, 0x13, 0xb6, 0x98, 0xcd, 0xbb, 0x9d, 0xd3, - 0x1a, 0xdf, 0x01, 0x4d, 0xa5, 0x05, 0xf6, 0x15, 0x63, 0xcc, 0x49, 0x42, 0x0c, 0xce, 0xe7, 0x05, - 0x48, 0x15, 0xfc, 0x00, 0x2e, 0xcf, 0xb1, 0xad, 0x82, 0xbe, 0x66, 0xbd, 0xd8, 0x79, 0x18, 0xdc, - 0xb8, 0xb0, 0x3f, 0x25, 0xc0, 0xaf, 0x26, 0x4e, 0xdf, 0x05, 0xc8, 0x4c, 0x0c, 0xcb, 0xc6, 0x39, - 0x03, 0x35, 0xb8, 0x3a, 0x8b, 0x4e, 0x36, 0x5d, 0xeb, 0xff, 0xfd, 0x57, 0xd7, 0x4b, 0xbf, 0xf8, - 0xea, 0x7a, 0xe9, 0xdf, 0xbe, 0xba, 0x5e, 0xfa, 0xf9, 0x2f, 0xaf, 0x2f, 0xfc, 0xe2, 0x97, 0xd7, - 0x17, 0xfe, 0xf1, 0x97, 0xd7, 0x17, 0xf6, 0xeb, 0xf4, 0xa7, 0xb9, 0x47, 0xff, 0x13, 0x00, 0x00, - 0xff, 0xff, 0x8f, 0xfa, 0xf4, 0xef, 0xaa, 0x37, 0x00, 0x00, + 0x5c, 0xe1, 0x02, 0x02, 0x81, 0x90, 0xb8, 0x21, 0x84, 0x38, 0xed, 0x11, 0x04, 0x8c, 0xd0, 0x2c, + 0xe2, 0xe0, 0x03, 0x12, 0xe2, 0x0f, 0xa0, 0x88, 0xf8, 0xf2, 0x55, 0x5d, 0x6d, 0x7b, 0x16, 0x71, + 0xe0, 0x54, 0x5f, 0xc4, 0xf7, 0x8e, 0x2f, 0xde, 0x91, 0x05, 0xcd, 0x60, 0x7f, 0x39, 0x08, 0xfd, + 0xd8, 0x17, 0xe5, 0x60, 0x7f, 0xa0, 0x99, 0x81, 0xc3, 0xe0, 0xe0, 0xce, 0xa1, 0x13, 0x1f, 0x4d, + 0xf7, 0x97, 0x2d, 0x7f, 0x72, 0xdf, 0x3e, 0x0c, 0xcd, 0xe0, 0xe8, 0x9e, 0xe3, 0xdf, 0xdf, 0x37, + 0xed, 0x43, 0x19, 0xde, 0x3f, 0x79, 0x74, 0x3f, 0xd8, 0xbf, 0x9f, 0x4c, 0x1d, 0xdc, 0xcb, 0x8d, + 0x3d, 0xf4, 0x0f, 0xfd, 0xfb, 0x84, 0xde, 0x9f, 0x1e, 0x10, 0x44, 0x00, 0xb5, 0x78, 0xb8, 0x3e, + 0x80, 0xea, 0xb6, 0x13, 0xc5, 0x42, 0x40, 0x75, 0xea, 0xd8, 0x51, 0xbf, 0x74, 0xb3, 0xb2, 0x54, + 0x37, 0xa8, 0xad, 0x3f, 0x05, 0x6d, 0x64, 0x46, 0xc7, 0x2f, 0x4c, 0x77, 0x2a, 0x45, 0x0f, 0x2a, + 0x27, 0xa6, 0xdb, 0x2f, 0xdd, 0x2c, 0x2d, 0xb5, 0x0d, 0x6c, 0x8a, 0x65, 0x68, 0x9e, 0x98, 0xee, + 0x38, 0x3e, 0x0b, 0x64, 0xbf, 0x7c, 0xb3, 0xb4, 0xd4, 0x5d, 0xb9, 0xbc, 0x1c, 0xec, 0x2f, 0x3f, + 0xf3, 0xa3, 0xd8, 0xf1, 0x0e, 0x97, 0x5f, 0x98, 0xee, 0xe8, 0x2c, 0x90, 0x46, 0xe3, 0x84, 0x1b, + 0xfa, 0x2e, 0xb4, 0xf6, 0x42, 0x6b, 0x73, 0xea, 0x59, 0xb1, 0xe3, 0x7b, 0xb8, 0xa3, 0x67, 0x4e, + 0x24, 0xad, 0xa8, 0x19, 0xd4, 0x46, 0x9c, 0x19, 0x1e, 0x46, 0xfd, 0xca, 0xcd, 0x0a, 0xe2, 0xb0, + 0x2d, 0xfa, 0xd0, 0x70, 0xa2, 0x75, 0x7f, 0xea, 0xc5, 0xfd, 0xea, 0xcd, 0xd2, 0x52, 0xd3, 0x48, + 0x40, 0xfd, 0x2f, 0x2b, 0x50, 0xfb, 0xde, 0x54, 0x86, 0x67, 0x34, 0x2f, 0x8e, 0xc3, 0x64, 0x2d, + 0x6c, 0x8b, 0x2b, 0x50, 0x73, 0x4d, 0xef, 0x30, 0xea, 0x97, 0x69, 0x31, 0x06, 0xc4, 0x35, 0xd0, + 0xcc, 0x83, 0x58, 0x86, 0xe3, 0xa9, 0x63, 0xf7, 0x2b, 0x37, 0x4b, 0x4b, 0x75, 0xa3, 0x49, 0x88, + 0xe7, 0x8e, 0x2d, 0xbe, 0x01, 0x4d, 0xdb, 0x1f, 0x5b, 0xf9, 0xbd, 0x6c, 0x9f, 0xf6, 0x12, 0xb7, + 0xa0, 0x39, 0x75, 0xec, 0xb1, 0xeb, 0x44, 0x71, 0xbf, 0x76, 0xb3, 0xb4, 0xd4, 0x5a, 0x69, 0xe2, + 0x65, 0x91, 0x76, 0x46, 0x63, 0xea, 0xd8, 0x44, 0xc4, 0x3b, 0xd0, 0x8c, 0x42, 0x6b, 0x7c, 0x30, + 0xf5, 0xac, 0x7e, 0x9d, 0x06, 0x5d, 0xc2, 0x41, 0xb9, 0x5b, 0x1b, 0x8d, 0x88, 0x01, 0xbc, 0x56, + 0x28, 0x4f, 0x64, 0x18, 0xc9, 0x7e, 0x83, 0xb7, 0x52, 0xa0, 0x78, 0x00, 0xad, 0x03, 0xd3, 0x92, + 0xf1, 0x38, 0x30, 0x43, 0x73, 0xd2, 0x6f, 0x66, 0x0b, 0x6d, 0x22, 0xfa, 0x19, 0x62, 0x23, 0x03, + 0x0e, 0x52, 0x40, 0x3c, 0x82, 0x0e, 0x41, 0xd1, 0xf8, 0xc0, 0x71, 0x63, 0x19, 0xf6, 0x35, 0x9a, + 0xd3, 0xa5, 0x39, 0x84, 0x19, 0x85, 0x52, 0x1a, 0x6d, 0x1e, 0xc4, 0x18, 0xf1, 0x2e, 0x80, 0x3c, + 0x0d, 0x4c, 0xcf, 0x1e, 0x9b, 0xae, 0xdb, 0x07, 0x3a, 0x83, 0xc6, 0x98, 0x55, 0xd7, 0x15, 0x6f, + 0xe3, 0xf9, 0x4c, 0x7b, 0x1c, 0x47, 0xfd, 0xce, 0xcd, 0xd2, 0x52, 0xd5, 0xa8, 0x23, 0x38, 0x8a, + 0x90, 0xae, 0x96, 0x69, 0x1d, 0xc9, 0x7e, 0xf7, 0x66, 0x69, 0xa9, 0x66, 0x30, 0x80, 0xd8, 0x03, + 0x27, 0x8c, 0xe2, 0xfe, 0x25, 0xc6, 0x12, 0x20, 0xae, 0x42, 0xdd, 0x3f, 0x38, 0x88, 0x64, 0xdc, + 0xef, 0x11, 0x5a, 0x41, 0xfa, 0x0a, 0x68, 0xc4, 0x55, 0x44, 0xb5, 0xdb, 0x50, 0x3f, 0x41, 0x80, + 0x99, 0xaf, 0xb5, 0xd2, 0xc1, 0x63, 0xa7, 0x8c, 0x67, 0xa8, 0x4e, 0xfd, 0x3a, 0x34, 0xb7, 0x4d, + 0xef, 0x30, 0xe1, 0x56, 0x7c, 0x4e, 0x9a, 0xa0, 0x19, 0xd4, 0xd6, 0xff, 0xa0, 0x0c, 0x75, 0x43, + 0x46, 0x53, 0x37, 0x16, 0x1f, 0x02, 0xe0, 0x63, 0x4d, 0xcc, 0x38, 0x74, 0x4e, 0xd5, 0xaa, 0xd9, + 0x73, 0x69, 0x53, 0xc7, 0x7e, 0x4a, 0x5d, 0xe2, 0x01, 0xb4, 0x69, 0xf5, 0x64, 0x68, 0x39, 0x3b, + 0x40, 0x7a, 0x3e, 0xa3, 0x45, 0x43, 0xd4, 0x8c, 0xab, 0x50, 0x27, 0xfe, 0x60, 0x1e, 0xed, 0x18, + 0x0a, 0x12, 0xb7, 0xa1, 0xeb, 0x78, 0x31, 0xbe, 0x9f, 0x15, 0x8f, 0x6d, 0x19, 0x25, 0x0c, 0xd4, + 0x49, 0xb1, 0x1b, 0x32, 0x8a, 0xc5, 0x43, 0xe0, 0x47, 0x48, 0x36, 0xac, 0xd1, 0x86, 0xdd, 0xf4, + 0x71, 0x23, 0xde, 0x91, 0xc6, 0xa8, 0x1d, 0xef, 0x41, 0x0b, 0xef, 0x97, 0xcc, 0xa8, 0xd3, 0x8c, + 0x36, 0xdd, 0x46, 0x91, 0xc3, 0x00, 0x1c, 0xa0, 0x86, 0x23, 0x69, 0x90, 0x49, 0x99, 0xa9, 0xa8, + 0xad, 0x0f, 0xa1, 0xb6, 0x1b, 0xda, 0x32, 0x9c, 0x2b, 0x27, 0x02, 0xaa, 0xb6, 0x8c, 0x2c, 0x12, + 0xe1, 0xa6, 0x41, 0xed, 0x4c, 0x76, 0x2a, 0x39, 0xd9, 0xd1, 0xff, 0xb0, 0x04, 0xad, 0x3d, 0x3f, + 0x8c, 0x9f, 0xca, 0x28, 0x32, 0x0f, 0xa5, 0xb8, 0x01, 0x35, 0x1f, 0x97, 0x55, 0x14, 0xd6, 0xf0, + 0x4c, 0xb4, 0x8f, 0xc1, 0xf8, 0x99, 0x77, 0x28, 0x5f, 0xfc, 0x0e, 0xc8, 0x53, 0x24, 0x75, 0x15, + 0xc5, 0x53, 0x24, 0x73, 0x19, 0xf7, 0x54, 0xf3, 0xdc, 0x73, 0x21, 0x6b, 0xea, 0xdf, 0x02, 0xc0, + 0xf3, 0x7d, 0x4d, 0x2e, 0xd0, 0x7f, 0x56, 0x82, 0x96, 0x61, 0x1e, 0xc4, 0xeb, 0xbe, 0x17, 0xcb, + 0xd3, 0x58, 0x74, 0xa1, 0xec, 0xd8, 0x44, 0xa3, 0xba, 0x51, 0x76, 0x6c, 0x3c, 0xdd, 0x61, 0xe8, + 0x4f, 0x03, 0x22, 0x51, 0xc7, 0x60, 0x80, 0x68, 0x69, 0xdb, 0x21, 0x1d, 0x19, 0x69, 0x69, 0xdb, + 0xa1, 0xb8, 0x01, 0xad, 0xc8, 0x33, 0x83, 0xe8, 0xc8, 0x8f, 0xf1, 0x74, 0x55, 0x3a, 0x1d, 0x24, + 0xa8, 0x51, 0x84, 0x42, 0xe7, 0x44, 0x63, 0x57, 0x9a, 0xa1, 0x27, 0x43, 0x52, 0x24, 0x4d, 0x43, + 0x73, 0xa2, 0x6d, 0x46, 0xe8, 0x3f, 0xab, 0x40, 0xfd, 0xa9, 0x9c, 0xec, 0xcb, 0xf0, 0xdc, 0x21, + 0x1e, 0x40, 0x93, 0xf6, 0x1d, 0x3b, 0x36, 0x9f, 0x63, 0xed, 0xad, 0x97, 0x5f, 0xde, 0x58, 0x24, + 0xdc, 0x96, 0xfd, 0xb1, 0x3f, 0x71, 0x62, 0x39, 0x09, 0xe2, 0x33, 0xa3, 0xa1, 0x50, 0x73, 0x0f, + 0x78, 0x15, 0xea, 0xae, 0x34, 0xf1, 0xcd, 0x98, 0x3d, 0x15, 0x24, 0xee, 0x41, 0xc3, 0x9c, 0x8c, + 0x6d, 0x69, 0xda, 0x7c, 0xa8, 0xb5, 0x2b, 0x2f, 0xbf, 0xbc, 0xd1, 0x33, 0x27, 0x1b, 0xd2, 0xcc, + 0xaf, 0x5d, 0x67, 0x8c, 0xf8, 0x04, 0x79, 0x32, 0x8a, 0xc7, 0xd3, 0xc0, 0x36, 0x63, 0x49, 0xba, + 0xae, 0xba, 0xd6, 0x7f, 0xf9, 0xe5, 0x8d, 0x2b, 0x88, 0x7e, 0x4e, 0xd8, 0xdc, 0x34, 0xc8, 0xb0, + 0xa8, 0xf7, 0x92, 0xeb, 0x2b, 0xbd, 0xa7, 0x40, 0xb1, 0x05, 0x8b, 0x96, 0x3b, 0x8d, 0x50, 0x39, + 0x3b, 0xde, 0x81, 0x3f, 0xf6, 0x3d, 0xf7, 0x8c, 0x1e, 0xb8, 0xb9, 0xf6, 0xee, 0xcb, 0x2f, 0x6f, + 0x7c, 0x43, 0x75, 0x6e, 0x79, 0x07, 0xfe, 0xae, 0xe7, 0x9e, 0xe5, 0xd6, 0xbf, 0x34, 0xd3, 0x25, + 0x7e, 0x03, 0xba, 0x07, 0x7e, 0x68, 0xc9, 0x71, 0x4a, 0xb2, 0x2e, 0xad, 0x33, 0x78, 0xf9, 0xe5, + 0x8d, 0xab, 0xd4, 0xf3, 0xf8, 0x1c, 0xdd, 0xda, 0x79, 0xbc, 0xfe, 0xaf, 0x65, 0xa8, 0x51, 0x5b, + 0x3c, 0x80, 0xc6, 0x84, 0x9e, 0x24, 0xd1, 0x4f, 0x57, 0x91, 0x87, 0xa8, 0x6f, 0x99, 0xdf, 0x2a, + 0x1a, 0x7a, 0x71, 0x78, 0x66, 0x24, 0xc3, 0x70, 0x46, 0x6c, 0xee, 0xbb, 0x32, 0x8e, 0x14, 0xcf, + 0xe7, 0x66, 0x8c, 0xb8, 0x43, 0xcd, 0x50, 0xc3, 0x66, 0xf9, 0xa6, 0x72, 0x8e, 0x6f, 0x06, 0xd0, + 0xb4, 0x8e, 0xa4, 0x75, 0x1c, 0x4d, 0x27, 0x8a, 0xab, 0x52, 0x58, 0xdc, 0x82, 0x0e, 0xb5, 0x03, + 0xdf, 0xf1, 0x68, 0x7a, 0x8d, 0x06, 0xb4, 0x33, 0xe4, 0x28, 0x1a, 0x6c, 0x42, 0x3b, 0x7f, 0x58, + 0x34, 0xe7, 0xc7, 0xf2, 0x8c, 0xf8, 0xab, 0x6a, 0x60, 0x53, 0xdc, 0x84, 0x1a, 0x29, 0x3a, 0xe2, + 0xae, 0xd6, 0x0a, 0xe0, 0x99, 0x79, 0x8a, 0xc1, 0x1d, 0x9f, 0x96, 0xbf, 0x5d, 0xc2, 0x75, 0xf2, + 0x57, 0xc8, 0xaf, 0xa3, 0x5d, 0xbc, 0x0e, 0x4f, 0xc9, 0xad, 0xa3, 0xfb, 0xd0, 0xd8, 0x76, 0x2c, + 0xe9, 0x45, 0x64, 0xf4, 0xa7, 0x91, 0x4c, 0x95, 0x12, 0xb6, 0xf1, 0xbe, 0x13, 0xf3, 0x74, 0xc7, + 0xb7, 0x65, 0x44, 0xeb, 0x54, 0x8d, 0x14, 0xc6, 0x3e, 0x79, 0x1a, 0x38, 0xe1, 0xd9, 0x88, 0x29, + 0x55, 0x31, 0x52, 0x18, 0xb9, 0x4b, 0x7a, 0xb8, 0x99, 0x9d, 0x18, 0x70, 0x05, 0xea, 0x7f, 0x52, + 0x85, 0xf6, 0x8f, 0x64, 0xe8, 0x3f, 0x0b, 0xfd, 0xc0, 0x8f, 0x4c, 0x57, 0xac, 0x16, 0x69, 0xce, + 0x6f, 0x7b, 0x13, 0x4f, 0x9b, 0x1f, 0xb6, 0xbc, 0x97, 0x3e, 0x02, 0xbf, 0x59, 0xfe, 0x55, 0x74, + 0xa8, 0xf3, 0x9b, 0xcf, 0xa1, 0x99, 0xea, 0xc1, 0x31, 0xfc, 0xca, 0x74, 0xd6, 0x22, 0x3d, 0x54, + 0x0f, 0x4a, 0xe5, 0xc4, 0x3c, 0x7d, 0xbe, 0xb5, 0xa1, 0xde, 0x56, 0x41, 0x8a, 0x0a, 0xa3, 0x53, + 0x6f, 0x94, 0x3c, 0x6a, 0x0a, 0xe3, 0x4d, 0x91, 0x22, 0xd1, 0xd6, 0x46, 0xbf, 0x4d, 0x5d, 0x09, + 0x28, 0xde, 0x01, 0x6d, 0x62, 0x9e, 0xa2, 0x42, 0xdb, 0xb2, 0x59, 0x34, 0x8d, 0x0c, 0x21, 0xde, + 0x83, 0x4a, 0x7c, 0xea, 0x91, 0xec, 0xa1, 0x57, 0x81, 0x4e, 0xe6, 0xe8, 0xd4, 0x53, 0xaa, 0xcf, + 0xc0, 0x3e, 0x7c, 0x53, 0xcb, 0xb1, 0xc9, 0x89, 0xd0, 0x0c, 0x6c, 0x8a, 0xdb, 0xd0, 0x70, 0xf9, + 0xb5, 0xc8, 0x51, 0x68, 0xad, 0xb4, 0x58, 0x8f, 0x12, 0xca, 0x48, 0xfa, 0xc4, 0xc7, 0xd0, 0x4c, + 0xa8, 0xd3, 0x6f, 0xd1, 0xb8, 0x5e, 0x42, 0xcf, 0x84, 0x8c, 0x46, 0x3a, 0x42, 0x3c, 0x00, 0xcd, + 0x96, 0xae, 0x8c, 0xe5, 0xd8, 0x63, 0x45, 0xde, 0x62, 0x07, 0x72, 0x83, 0x90, 0x3b, 0x91, 0x21, + 0x7f, 0x32, 0x95, 0x51, 0x6c, 0x34, 0x6d, 0x85, 0x10, 0xef, 0x67, 0x82, 0xd5, 0xa5, 0xe7, 0xca, + 0x13, 0x33, 0xe9, 0x1a, 0x7c, 0x17, 0x2e, 0xcd, 0x3c, 0x5a, 0x9e, 0x4b, 0x3b, 0xcc, 0xa5, 0x57, + 0xf2, 0x5c, 0x5a, 0xcd, 0x71, 0xe6, 0xe7, 0xd5, 0x66, 0xb3, 0xa7, 0xe9, 0xff, 0x55, 0x81, 0x4b, + 0x4a, 0x60, 0x8e, 0x9c, 0x60, 0x2f, 0x56, 0xaa, 0x8b, 0x0c, 0x93, 0xe2, 0xd5, 0xaa, 0x91, 0x80, + 0xe2, 0xd7, 0xa1, 0x4e, 0x9a, 0x26, 0x11, 0xf8, 0x1b, 0x19, 0x23, 0xa4, 0xd3, 0x59, 0x01, 0x28, + 0x2e, 0x52, 0xc3, 0xc5, 0x37, 0xa1, 0xf6, 0x85, 0x0c, 0x7d, 0x36, 0xb4, 0xad, 0x95, 0xeb, 0xf3, + 0xe6, 0x21, 0xf9, 0xd4, 0x34, 0x1e, 0xfc, 0xbf, 0xe5, 0x17, 0xf8, 0x3a, 0xfc, 0xf2, 0x3e, 0x1a, + 0xdb, 0x89, 0x7f, 0x22, 0xed, 0x7e, 0x23, 0xa3, 0xb9, 0x62, 0xf2, 0xa4, 0x2b, 0x61, 0x99, 0xe6, + 0x5c, 0x96, 0xd1, 0x2e, 0x66, 0x99, 0xc1, 0x06, 0xb4, 0x72, 0x74, 0x99, 0xf3, 0x50, 0x37, 0x8a, + 0xea, 0x44, 0x4b, 0x55, 0x69, 0x5e, 0x2b, 0x6d, 0x00, 0x64, 0x54, 0xfa, 0x55, 0x75, 0x9b, 0xfe, + 0xdb, 0x25, 0xb8, 0xb4, 0xee, 0x7b, 0x9e, 0x24, 0x57, 0x9d, 0xdf, 0x3c, 0x13, 0xf1, 0xd2, 0x85, + 0x22, 0xfe, 0x11, 0xd4, 0x22, 0x1c, 0xac, 0x56, 0xbf, 0x3c, 0xe7, 0x11, 0x0d, 0x1e, 0x81, 0x8a, + 0x7e, 0x62, 0x9e, 0x8e, 0x03, 0xe9, 0xd9, 0x8e, 0x77, 0x98, 0x28, 0xfa, 0x89, 0x79, 0xfa, 0x8c, + 0x31, 0xfa, 0x5f, 0x95, 0x01, 0x3e, 0x93, 0xa6, 0x1b, 0x1f, 0xa1, 0x31, 0xc3, 0x17, 0x75, 0xbc, + 0x28, 0x36, 0x3d, 0x2b, 0x09, 0x94, 0x52, 0x18, 0x5f, 0x14, 0x6d, 0xba, 0x8c, 0x58, 0x45, 0x6a, + 0x46, 0x02, 0x22, 0x7f, 0xe0, 0x76, 0xd3, 0x48, 0xd9, 0x7e, 0x05, 0x65, 0x8e, 0x4c, 0x95, 0xd0, + 0xca, 0x91, 0xe9, 0x43, 0x03, 0x03, 0x0f, 0xc7, 0xf7, 0x88, 0x69, 0x34, 0x23, 0x01, 0x71, 0x9d, + 0x69, 0x10, 0x3b, 0x13, 0xb6, 0xf0, 0x15, 0x43, 0x41, 0x78, 0x2a, 0xb4, 0xe8, 0x43, 0xeb, 0xc8, + 0x27, 0x45, 0x52, 0x31, 0x52, 0x18, 0x57, 0xf3, 0xbd, 0x43, 0x1f, 0x6f, 0xd7, 0x24, 0xe7, 0x31, + 0x01, 0xf9, 0x2e, 0xb6, 0x3c, 0xc5, 0x2e, 0x8d, 0xba, 0x52, 0x18, 0xe9, 0x22, 0xe5, 0xf8, 0x40, + 0x9a, 0xf1, 0x34, 0x94, 0x51, 0x1f, 0xa8, 0x1b, 0xa4, 0xdc, 0x54, 0x18, 0xf1, 0x1e, 0xb4, 0x91, + 0x70, 0x66, 0x14, 0x39, 0x87, 0x9e, 0xb4, 0x49, 0xbd, 0x54, 0x0d, 0x24, 0xe6, 0xaa, 0x42, 0xe9, + 0x7f, 0x53, 0x86, 0x3a, 0xeb, 0x82, 0x82, 0xb3, 0x54, 0x7a, 0x23, 0x67, 0xe9, 0x1d, 0xd0, 0x82, + 0x50, 0xda, 0x8e, 0x95, 0xbc, 0xa3, 0x66, 0x64, 0x08, 0x8a, 0x6e, 0xd0, 0x3b, 0x20, 0x7a, 0x36, + 0x0d, 0x06, 0x84, 0x0e, 0x1d, 0xdf, 0x1b, 0xdb, 0x4e, 0x74, 0x3c, 0xde, 0x3f, 0x8b, 0x65, 0xa4, + 0x68, 0xd1, 0xf2, 0xbd, 0x0d, 0x27, 0x3a, 0x5e, 0x43, 0x14, 0x92, 0x90, 0x65, 0x84, 0x64, 0xa3, + 0x69, 0x28, 0x48, 0x3c, 0x02, 0x8d, 0x7c, 0x58, 0x72, 0x72, 0x34, 0x72, 0x4e, 0xae, 0xbe, 0xfc, + 0xf2, 0x86, 0x40, 0xe4, 0x8c, 0x77, 0xd3, 0x4c, 0x70, 0xe8, 0xa5, 0xe1, 0x64, 0x34, 0x57, 0x24, + 0xc3, 0xec, 0xa5, 0x21, 0x6a, 0x14, 0xe5, 0xbd, 0x34, 0xc6, 0x88, 0x7b, 0x20, 0xa6, 0x9e, 0xe5, + 0x4f, 0x02, 0x64, 0x0a, 0x69, 0xab, 0x43, 0xb6, 0xe8, 0x90, 0x8b, 0xf9, 0x1e, 0x3a, 0xaa, 0xfe, + 0x2f, 0x65, 0x68, 0x6f, 0x38, 0xa1, 0xb4, 0x62, 0x69, 0x0f, 0xed, 0x43, 0x89, 0x67, 0x97, 0x5e, + 0xec, 0xc4, 0x67, 0xca, 0x0d, 0x55, 0x50, 0x1a, 0x45, 0x94, 0x8b, 0xd1, 0x36, 0x4b, 0x58, 0x85, + 0x12, 0x04, 0x0c, 0x88, 0x15, 0x00, 0x8e, 0xaf, 0x28, 0x49, 0x50, 0xbd, 0x38, 0x49, 0xa0, 0xd1, + 0x30, 0x6c, 0x62, 0x10, 0xce, 0x73, 0x1c, 0xf6, 0x45, 0xeb, 0x94, 0x41, 0x98, 0x4a, 0xf6, 0x68, + 0x29, 0xec, 0x6b, 0xf0, 0xc6, 0xd8, 0x16, 0xb7, 0xa0, 0xec, 0x07, 0x44, 0x5c, 0xb5, 0x74, 0xfe, + 0x0a, 0xcb, 0xbb, 0x81, 0x51, 0xf6, 0x03, 0x94, 0x62, 0x8e, 0x7d, 0x89, 0xf1, 0x50, 0x8a, 0xd1, + 0xee, 0x51, 0xc4, 0x65, 0xa8, 0x1e, 0xa1, 0x43, 0xdb, 0x74, 0x5d, 0xff, 0xa7, 0xd2, 0x7e, 0x16, + 0x4a, 0x3b, 0xe1, 0xc1, 0x02, 0x0e, 0xb9, 0xc4, 0x33, 0x27, 0x32, 0x0a, 0x4c, 0x4b, 0x2a, 0x16, + 0xcc, 0x10, 0xfa, 0x55, 0x28, 0xef, 0x06, 0xa2, 0x01, 0x95, 0xbd, 0xe1, 0xa8, 0xb7, 0x80, 0x8d, + 0x8d, 0xe1, 0x76, 0x0f, 0x2d, 0x4a, 0xbd, 0xd7, 0xd0, 0xbf, 0x2a, 0x83, 0xf6, 0x74, 0x1a, 0x9b, + 0xa8, 0x5b, 0x22, 0xbc, 0x65, 0x91, 0x43, 0x33, 0x56, 0xfc, 0x06, 0x34, 0xa3, 0xd8, 0x0c, 0xc9, + 0x2b, 0x61, 0xeb, 0xd4, 0x20, 0x78, 0x14, 0x89, 0x0f, 0xa0, 0x26, 0xed, 0x43, 0x99, 0x98, 0x8b, + 0xde, 0xec, 0x7d, 0x0d, 0xee, 0x16, 0x4b, 0x50, 0x8f, 0xac, 0x23, 0x39, 0x31, 0xfb, 0xd5, 0x6c, + 0xe0, 0x1e, 0x61, 0xd8, 0x0d, 0x37, 0x54, 0xbf, 0x78, 0x1f, 0x6a, 0xf8, 0x36, 0x91, 0x8a, 0x2b, + 0x29, 0x12, 0xc5, 0x67, 0x50, 0xc3, 0xb8, 0x13, 0x19, 0xcf, 0x0e, 0xfd, 0x60, 0xec, 0x07, 0x44, + 0xfb, 0xee, 0xca, 0x15, 0xd2, 0x71, 0xc9, 0x6d, 0x96, 0x37, 0x42, 0x3f, 0xd8, 0x0d, 0x8c, 0xba, + 0x4d, 0xbf, 0x18, 0xe5, 0xd0, 0x70, 0xe6, 0x08, 0x36, 0x0a, 0x1a, 0x62, 0x38, 0x95, 0xb4, 0x04, + 0xcd, 0x89, 0x8c, 0x4d, 0xdb, 0x8c, 0x4d, 0x65, 0x1b, 0xda, 0xac, 0x32, 0x19, 0x67, 0xa4, 0xbd, + 0xfa, 0x7d, 0xa8, 0xf3, 0xd2, 0xa2, 0x09, 0xd5, 0x9d, 0xdd, 0x9d, 0x21, 0x93, 0x75, 0x75, 0x7b, + 0xbb, 0x57, 0x42, 0xd4, 0xc6, 0xea, 0x68, 0xb5, 0x57, 0xc6, 0xd6, 0xe8, 0x87, 0xcf, 0x86, 0xbd, + 0x8a, 0xfe, 0x0f, 0x25, 0x68, 0x26, 0xeb, 0x88, 0x4f, 0x01, 0x50, 0x84, 0xc7, 0x47, 0x8e, 0x97, + 0x3a, 0x78, 0xd7, 0xf2, 0x3b, 0x2d, 0xe3, 0xab, 0x7e, 0x86, 0xbd, 0x6c, 0x5e, 0x49, 0xe2, 0x09, + 0x1e, 0xec, 0x41, 0xb7, 0xd8, 0x39, 0xc7, 0xd3, 0xbd, 0x9b, 0xb7, 0x2a, 0xdd, 0x95, 0xb7, 0x0a, + 0x4b, 0xe3, 0x4c, 0x62, 0xed, 0x9c, 0x81, 0xb9, 0x07, 0xcd, 0x04, 0x2d, 0x5a, 0xd0, 0xd8, 0x18, + 0x6e, 0xae, 0x3e, 0xdf, 0x46, 0x56, 0x01, 0xa8, 0xef, 0x6d, 0xed, 0x3c, 0xde, 0x1e, 0xf2, 0xb5, + 0xb6, 0xb7, 0xf6, 0x46, 0xbd, 0xb2, 0xfe, 0xfb, 0x25, 0x68, 0x26, 0x9e, 0x8c, 0xf8, 0x08, 0x9d, + 0x0f, 0x72, 0xd2, 0x94, 0x25, 0xa2, 0x8c, 0x50, 0x2e, 0x6c, 0x35, 0x92, 0x7e, 0x94, 0x45, 0x52, + 0xac, 0x89, 0x6f, 0x43, 0x40, 0x3e, 0x6a, 0xae, 0x14, 0x12, 0x3a, 0x02, 0xaa, 0xb6, 0xef, 0x49, + 0xe5, 0x30, 0x53, 0x9b, 0x78, 0xd0, 0xf1, 0x2c, 0x99, 0x85, 0x13, 0x0d, 0x82, 0x47, 0x91, 0x1e, + 0xb3, 0x1f, 0x9d, 0x1e, 0x2c, 0xdd, 0xad, 0x94, 0xdf, 0xed, 0x5c, 0x50, 0x52, 0x3e, 0x1f, 0x94, + 0x64, 0x86, 0xb3, 0xf6, 0x3a, 0xc3, 0xa9, 0xff, 0x69, 0x15, 0xba, 0x86, 0x8c, 0x62, 0x3f, 0x94, + 0xca, 0x2f, 0x7c, 0x95, 0x08, 0xbd, 0x0b, 0x10, 0xf2, 0xe0, 0x6c, 0x6b, 0x4d, 0x61, 0x38, 0x9a, + 0x72, 0x7d, 0x8b, 0x78, 0x57, 0x59, 0xc8, 0x14, 0x16, 0xd7, 0x40, 0xdb, 0x37, 0xad, 0x63, 0x5e, + 0x96, 0xed, 0x64, 0x93, 0x11, 0xbc, 0xae, 0x69, 0x59, 0x32, 0x8a, 0xc6, 0xc8, 0x0a, 0x6c, 0x2d, + 0x35, 0xc6, 0x3c, 0x91, 0x67, 0xd8, 0x1d, 0x49, 0x2b, 0x94, 0x31, 0x75, 0xd7, 0xb9, 0x9b, 0x31, + 0xd8, 0x7d, 0x0b, 0x3a, 0x91, 0x8c, 0xd0, 0xb2, 0x8e, 0x63, 0xff, 0x58, 0x7a, 0x4a, 0x8f, 0xb5, + 0x15, 0x72, 0x84, 0x38, 0x54, 0x31, 0xa6, 0xe7, 0x7b, 0x67, 0x13, 0x7f, 0x1a, 0x29, 0x9b, 0x91, + 0x21, 0xc4, 0x32, 0x5c, 0x96, 0x9e, 0x15, 0x9e, 0x05, 0x78, 0x56, 0xdc, 0x65, 0x7c, 0xe0, 0xb8, + 0x52, 0xb9, 0xea, 0x8b, 0x59, 0xd7, 0x13, 0x79, 0xb6, 0xe9, 0xb8, 0x12, 0x4f, 0x74, 0x62, 0x4e, + 0xdd, 0x78, 0x4c, 0x99, 0x00, 0xe0, 0x13, 0x11, 0x66, 0xd5, 0xb6, 0x43, 0x71, 0x07, 0x16, 0xb9, + 0x3b, 0xf4, 0x5d, 0xe9, 0xd8, 0xbc, 0x58, 0x8b, 0x46, 0x5d, 0xa2, 0x0e, 0x83, 0xf0, 0xb4, 0xd4, + 0x32, 0x5c, 0xe6, 0xb1, 0x7c, 0xa1, 0x64, 0x74, 0x9b, 0xb7, 0xa6, 0xae, 0x3d, 0xd5, 0x53, 0xdc, + 0x3a, 0x30, 0xe3, 0x23, 0xf2, 0xef, 0x93, 0xad, 0x9f, 0x99, 0xf1, 0x11, 0x5a, 0x7c, 0xee, 0x3e, + 0x70, 0xa4, 0xcb, 0xf1, 0xb9, 0x66, 0xf0, 0x8c, 0x4d, 0xc4, 0xa0, 0xc5, 0x57, 0x03, 0xfc, 0x70, + 0x62, 0x72, 0x62, 0x51, 0x33, 0x78, 0xd2, 0x26, 0xa1, 0x70, 0x0b, 0xf5, 0x56, 0xde, 0x74, 0x42, + 0x29, 0xc6, 0xaa, 0xa1, 0x5e, 0x6f, 0x67, 0x3a, 0xd1, 0x5f, 0x56, 0xa0, 0x99, 0x86, 0x7b, 0x77, + 0x41, 0x9b, 0x24, 0xfa, 0x4a, 0x39, 0x6a, 0x9d, 0x82, 0x12, 0x33, 0xb2, 0x7e, 0xf1, 0x2e, 0x94, + 0x8f, 0x4f, 0x94, 0xee, 0xec, 0x2c, 0x73, 0xa2, 0x3d, 0xd8, 0x7f, 0xb4, 0xfc, 0xe4, 0x85, 0x51, + 0x3e, 0x3e, 0xf9, 0x1a, 0x7c, 0x2b, 0x3e, 0x84, 0x4b, 0x96, 0x2b, 0x4d, 0x6f, 0x9c, 0x79, 0x17, + 0xcc, 0x17, 0x5d, 0x42, 0x3f, 0x4b, 0x5d, 0x8c, 0xdb, 0x50, 0xb3, 0xa5, 0x1b, 0x9b, 0xf9, 0x7c, + 0xef, 0x6e, 0x68, 0x5a, 0xae, 0xdc, 0x40, 0xb4, 0xc1, 0xbd, 0xa8, 0x3b, 0xd3, 0x10, 0x2b, 0xa7, + 0x3b, 0xe7, 0x84, 0x57, 0xa9, 0x5c, 0x42, 0x5e, 0x2e, 0xef, 0xc2, 0xa2, 0x3c, 0x0d, 0xc8, 0x60, + 0x8c, 0xd3, 0x8c, 0x02, 0x5b, 0xb2, 0x5e, 0xd2, 0xb1, 0x9e, 0x64, 0x16, 0x3e, 0x46, 0x95, 0x41, + 0x42, 0x43, 0xcf, 0xdc, 0x5a, 0x11, 0xa4, 0x73, 0x0a, 0x62, 0x68, 0x24, 0x43, 0xc4, 0x47, 0xa0, + 0x59, 0xb6, 0x35, 0x66, 0xca, 0x74, 0xb2, 0xb3, 0xad, 0x6f, 0xac, 0x33, 0x49, 0x9a, 0x96, 0x6d, + 0xb1, 0x57, 0x5d, 0x08, 0xfd, 0xba, 0x6f, 0x12, 0xfa, 0xe5, 0x8d, 0x62, 0xaf, 0x60, 0x14, 0x3f, + 0xaf, 0x36, 0x1b, 0xbd, 0xa6, 0x7e, 0x0b, 0x9a, 0xc9, 0x46, 0xa8, 0xea, 0x22, 0xe9, 0xa9, 0xb0, + 0x9e, 0x54, 0x1d, 0x82, 0xa3, 0x48, 0xb7, 0xa0, 0xf2, 0xe4, 0xc5, 0x1e, 0x69, 0x3c, 0x34, 0x3e, + 0x35, 0xf2, 0x55, 0xa8, 0x9d, 0x6a, 0xc1, 0x72, 0x4e, 0x0b, 0x5e, 0x67, 0x03, 0x42, 0x0f, 0x94, + 0xe4, 0x42, 0x73, 0x18, 0x24, 0x31, 0x1b, 0xcf, 0x2a, 0xa7, 0x49, 0x09, 0xd0, 0xff, 0xa3, 0x02, + 0x0d, 0xe5, 0xdf, 0xa0, 0xd1, 0x98, 0xa6, 0x69, 0x3c, 0x6c, 0x16, 0x03, 0xcf, 0xd4, 0x51, 0xca, + 0xd7, 0x52, 0x2a, 0xaf, 0xaf, 0xa5, 0x88, 0x4f, 0xa1, 0x1d, 0x70, 0x5f, 0xde, 0xb5, 0x7a, 0x3b, + 0x3f, 0x47, 0xfd, 0xd2, 0xbc, 0x56, 0x90, 0x01, 0x48, 0x4a, 0x4a, 0x28, 0xc7, 0xe6, 0xa1, 0xa2, + 0x40, 0x03, 0xe1, 0x91, 0x79, 0xf8, 0x46, 0x7e, 0x52, 0x97, 0x1c, 0xae, 0x36, 0x29, 0x5c, 0xf4, + 0xad, 0xf2, 0x2f, 0xd3, 0x29, 0xba, 0x2b, 0xd7, 0x40, 0xb3, 0xfc, 0xc9, 0xc4, 0xa1, 0xbe, 0xae, + 0x4a, 0x5b, 0x11, 0x62, 0x14, 0xe9, 0xbf, 0x5b, 0x82, 0x86, 0xba, 0xd7, 0x39, 0x63, 0xb8, 0xb6, + 0xb5, 0xb3, 0x6a, 0xfc, 0xb0, 0x57, 0x42, 0x63, 0xbf, 0xb5, 0x33, 0xea, 0x95, 0x85, 0x06, 0xb5, + 0xcd, 0xed, 0xdd, 0xd5, 0x51, 0xaf, 0x82, 0x06, 0x72, 0x6d, 0x77, 0x77, 0xbb, 0x57, 0x15, 0x6d, + 0x68, 0x6e, 0xac, 0x8e, 0x86, 0xa3, 0xad, 0xa7, 0xc3, 0x5e, 0x0d, 0xc7, 0x3e, 0x1e, 0xee, 0xf6, + 0xea, 0xd8, 0x78, 0xbe, 0xb5, 0xd1, 0x6b, 0x60, 0xff, 0xb3, 0xd5, 0xbd, 0xbd, 0xef, 0xef, 0x1a, + 0x1b, 0xbd, 0x26, 0x19, 0xd9, 0x91, 0xb1, 0xb5, 0xf3, 0xb8, 0xa7, 0x61, 0x7b, 0x77, 0xed, 0xf3, + 0xe1, 0xfa, 0xa8, 0x07, 0xfa, 0x43, 0x68, 0xe5, 0x68, 0x85, 0xb3, 0x8d, 0xe1, 0x66, 0x6f, 0x01, + 0xb7, 0x7c, 0xb1, 0xba, 0xfd, 0x1c, 0x6d, 0x72, 0x17, 0x80, 0x9a, 0xe3, 0xed, 0xd5, 0x9d, 0xc7, + 0xbd, 0xb2, 0xf2, 0xe8, 0xbe, 0x07, 0xcd, 0xe7, 0x8e, 0xbd, 0xe6, 0xfa, 0xd6, 0x31, 0xb2, 0xcf, + 0xbe, 0x19, 0x49, 0xc5, 0x6f, 0xd4, 0x46, 0xff, 0x99, 0x84, 0x36, 0x52, 0x6f, 0xad, 0x20, 0xa4, + 0x98, 0x37, 0x9d, 0x8c, 0xa9, 0xde, 0x56, 0x61, 0xc3, 0xe5, 0x4d, 0x27, 0xcf, 0x1d, 0x3b, 0xd2, + 0x8f, 0xa1, 0xf1, 0xdc, 0xb1, 0x9f, 0x99, 0xd6, 0x31, 0x29, 0x37, 0x5c, 0x7a, 0x1c, 0x39, 0x5f, + 0x48, 0x65, 0xe0, 0x34, 0xc2, 0xec, 0x39, 0x5f, 0x48, 0xf1, 0x3e, 0xd4, 0x09, 0x48, 0x52, 0x0e, + 0x24, 0x6a, 0xc9, 0x71, 0x0c, 0xd5, 0x47, 0xe5, 0x2e, 0xd7, 0xf5, 0xad, 0x71, 0x28, 0x0f, 0xfa, + 0x6f, 0xf3, 0x0b, 0x10, 0xc2, 0x90, 0x07, 0xfa, 0xef, 0x95, 0xd2, 0x9b, 0x53, 0x55, 0xe5, 0x06, + 0x54, 0x03, 0xd3, 0x3a, 0x56, 0xfe, 0x45, 0x4b, 0x2d, 0x88, 0x87, 0x31, 0xa8, 0x43, 0x7c, 0x08, + 0x4d, 0xc5, 0x48, 0xc9, 0xae, 0xad, 0x1c, 0xc7, 0x19, 0x69, 0x67, 0xf1, 0xe1, 0x2b, 0xc5, 0x87, + 0xa7, 0xe8, 0x34, 0x70, 0x9d, 0x98, 0xc5, 0x06, 0x85, 0x93, 0x20, 0xfd, 0x9b, 0x00, 0x59, 0x81, + 0x6b, 0x8e, 0xbb, 0x75, 0x05, 0x6a, 0xa6, 0xeb, 0x98, 0x49, 0xb4, 0xcb, 0x80, 0xbe, 0x03, 0xad, + 0x5c, 0x59, 0x0c, 0x69, 0x6b, 0xba, 0x2e, 0x5a, 0x46, 0x96, 0xfd, 0xa6, 0xd1, 0x30, 0x5d, 0xf7, + 0x89, 0x3c, 0x8b, 0xd0, 0xd5, 0xe5, 0x8a, 0x5a, 0x79, 0xa6, 0xe8, 0x42, 0x53, 0x0d, 0xee, 0xd4, + 0x3f, 0x86, 0xfa, 0x66, 0x12, 0x10, 0x24, 0xc2, 0x50, 0xba, 0x48, 0x18, 0xf4, 0x4f, 0xd4, 0x99, + 0xa9, 0x6e, 0x23, 0xee, 0xaa, 0xca, 0x5d, 0xc4, 0x75, 0xc2, 0x52, 0x96, 0x2f, 0xe1, 0x41, 0xaa, + 0x68, 0x47, 0x83, 0xf5, 0x0d, 0x68, 0xbe, 0xb2, 0x16, 0xaa, 0x08, 0x50, 0xce, 0x08, 0x30, 0xa7, + 0x3a, 0xaa, 0xff, 0x18, 0x20, 0xab, 0xf0, 0x29, 0xd9, 0xe4, 0x55, 0x50, 0x36, 0xef, 0x40, 0xd3, + 0x3a, 0x72, 0x5c, 0x3b, 0x94, 0x5e, 0xe1, 0xd6, 0x59, 0x4d, 0x30, 0xed, 0x17, 0x37, 0xa1, 0x4a, + 0x85, 0xcb, 0x4a, 0xa6, 0xb9, 0xd3, 0xaa, 0x25, 0xf5, 0xe8, 0xa7, 0xd0, 0xe1, 0x18, 0xe2, 0x0d, + 0x3c, 0xb0, 0xa2, 0xea, 0x2c, 0x9f, 0x53, 0x9d, 0x57, 0xa1, 0x4e, 0x86, 0x3f, 0xb9, 0x8d, 0x82, + 0x2e, 0x50, 0xa9, 0xbf, 0x53, 0x06, 0xe0, 0xad, 0x77, 0x7c, 0x5b, 0x16, 0x83, 0xf5, 0xd2, 0x6c, + 0xb0, 0x2e, 0xa0, 0x9a, 0xd6, 0xa4, 0x35, 0x83, 0xda, 0x99, 0x31, 0x54, 0x01, 0x3c, 0x1b, 0xc3, + 0x77, 0x40, 0x23, 0x47, 0xcc, 0xf9, 0x82, 0x0a, 0x22, 0xb8, 0x61, 0x86, 0xc8, 0x57, 0x68, 0x6b, + 0xc5, 0x0a, 0x6d, 0x5a, 0xae, 0xaa, 0xf3, 0x6a, 0x5c, 0xae, 0x9a, 0x53, 0x79, 0xe3, 0x0c, 0x4a, + 0x24, 0xc3, 0x38, 0x09, 0xff, 0x19, 0x4a, 0x23, 0x59, 0x4d, 0x8d, 0x35, 0x39, 0x07, 0xe2, 0xf9, + 0x63, 0xcb, 0xf7, 0x0e, 0x5c, 0xc7, 0x8a, 0x55, 0x45, 0x16, 0x3c, 0x7f, 0x5d, 0x61, 0xf4, 0x4f, + 0xa1, 0x9d, 0xd0, 0x9f, 0x0a, 0x5c, 0x77, 0xd2, 0x28, 0xaf, 0x94, 0xbd, 0x6d, 0x46, 0xa6, 0xb5, + 0x72, 0xbf, 0x94, 0xc4, 0x79, 0xfa, 0x7f, 0x57, 0x92, 0xc9, 0xaa, 0x0e, 0xf3, 0x6a, 0x1a, 0x16, + 0x03, 0xf7, 0xf2, 0x1b, 0x05, 0xee, 0xdf, 0x06, 0xcd, 0xa6, 0x58, 0xd4, 0x39, 0x49, 0x8c, 0xd8, + 0x60, 0x36, 0xee, 0x54, 0xd1, 0xaa, 0x73, 0x22, 0x8d, 0x6c, 0xf0, 0x6b, 0xde, 0x21, 0xa5, 0x76, + 0x6d, 0x1e, 0xb5, 0xeb, 0xbf, 0x22, 0xb5, 0xdf, 0x83, 0xb6, 0xe7, 0x7b, 0x63, 0x6f, 0xea, 0xba, + 0xe6, 0xbe, 0x2b, 0x15, 0xb9, 0x5b, 0x9e, 0xef, 0xed, 0x28, 0x14, 0x7a, 0xc7, 0xf9, 0x21, 0x2c, + 0xd4, 0x2d, 0x1a, 0x77, 0x29, 0x37, 0x8e, 0x44, 0x7f, 0x09, 0x7a, 0xfe, 0xfe, 0x8f, 0xa5, 0x15, + 0x13, 0xc5, 0xc6, 0x24, 0xcd, 0xec, 0x1a, 0x77, 0x19, 0x8f, 0x24, 0xda, 0x41, 0xb9, 0x9e, 0x79, + 0xe6, 0xce, 0xb9, 0x67, 0xfe, 0x04, 0xb4, 0x94, 0x4a, 0xb9, 0xb8, 0x57, 0x83, 0xda, 0xd6, 0xce, + 0xc6, 0xf0, 0x07, 0xbd, 0x12, 0x9a, 0x4b, 0x63, 0xf8, 0x62, 0x68, 0xec, 0x0d, 0x7b, 0x65, 0x34, + 0x65, 0x1b, 0xc3, 0xed, 0xe1, 0x68, 0xd8, 0xab, 0xb0, 0x2b, 0x44, 0xe5, 0x10, 0xd7, 0xb1, 0x9c, + 0x58, 0xdf, 0x03, 0xc8, 0x82, 0x79, 0xd4, 0xca, 0xd9, 0xe1, 0x54, 0x36, 0x31, 0x4e, 0x8e, 0xb5, + 0x94, 0x0a, 0x64, 0xf9, 0xa2, 0x94, 0x01, 0xf7, 0xeb, 0x2b, 0xa0, 0x3d, 0x35, 0x83, 0xcf, 0xb8, + 0x70, 0x78, 0x1b, 0xba, 0x81, 0x19, 0xc6, 0x4e, 0x12, 0x8f, 0xb0, 0xb2, 0x6c, 0x1b, 0x9d, 0x14, + 0x8b, 0xba, 0x57, 0xff, 0xb3, 0x12, 0x5c, 0x79, 0xea, 0x9f, 0xc8, 0xd4, 0xdf, 0x7d, 0x66, 0x9e, + 0xb9, 0xbe, 0x69, 0xbf, 0x86, 0x0d, 0x31, 0xa0, 0xf2, 0xa7, 0x54, 0xc8, 0x4b, 0xca, 0x9e, 0x86, + 0xc6, 0x98, 0xc7, 0xea, 0x7b, 0x0d, 0x19, 0xc5, 0xd4, 0xa9, 0x0c, 0x29, 0xc2, 0xd8, 0xf5, 0x16, + 0xd4, 0xe3, 0x53, 0x2f, 0x2b, 0xc2, 0xd6, 0x62, 0xca, 0x82, 0xcf, 0x75, 0x7f, 0x6b, 0xf3, 0xdd, + 0x5f, 0x7d, 0x1d, 0xb4, 0xd1, 0x29, 0xe5, 0x81, 0xa7, 0x45, 0x07, 0xb4, 0xf4, 0x0a, 0x37, 0xa7, + 0x3c, 0xe3, 0xe6, 0xfc, 0x7b, 0x09, 0x5a, 0x39, 0x3f, 0x5e, 0xbc, 0x07, 0xd5, 0xf8, 0xd4, 0x2b, + 0x7e, 0xeb, 0x90, 0x6c, 0x62, 0x50, 0xd7, 0xb9, 0x5c, 0x67, 0xf9, 0x5c, 0xae, 0x53, 0x6c, 0xc3, + 0x25, 0xd6, 0xbc, 0xc9, 0x25, 0x92, 0x94, 0xd0, 0xad, 0x99, 0xb8, 0x81, 0x73, 0xe5, 0xc9, 0x95, + 0x54, 0x9e, 0xa3, 0x7b, 0x58, 0x40, 0x0e, 0x56, 0xe1, 0xf2, 0x9c, 0x61, 0x5f, 0xa7, 0x6a, 0xa2, + 0xdf, 0x80, 0xce, 0xe8, 0xd4, 0x1b, 0x39, 0x13, 0x19, 0xc5, 0xe6, 0x24, 0x20, 0x37, 0x51, 0x59, + 0xce, 0xaa, 0x51, 0x8e, 0x23, 0xfd, 0x03, 0x68, 0x3f, 0x93, 0x32, 0x34, 0x64, 0x14, 0xf8, 0x1e, + 0x3b, 0x47, 0x2a, 0x47, 0xcd, 0x66, 0x5a, 0x41, 0xfa, 0x6f, 0x81, 0x66, 0x98, 0x07, 0xf1, 0x9a, + 0x19, 0x5b, 0x47, 0x5f, 0x27, 0xe9, 0xf1, 0x01, 0x34, 0x02, 0xe6, 0x29, 0x15, 0xdd, 0xb5, 0xc9, + 0x5c, 0x2b, 0x3e, 0x33, 0x92, 0x4e, 0xfd, 0xd7, 0xa0, 0xab, 0x0a, 0x46, 0xc9, 0x49, 0x72, 0x55, + 0xa5, 0xd2, 0x85, 0x55, 0x25, 0xfd, 0x10, 0x3a, 0xc9, 0x3c, 0x36, 0x7e, 0x6f, 0x34, 0xed, 0xeb, + 0x97, 0xed, 0xf5, 0xdf, 0x84, 0xcb, 0x7b, 0xd3, 0xfd, 0xc8, 0x0a, 0x1d, 0x8a, 0xe4, 0x93, 0xed, + 0x06, 0xd0, 0x0c, 0x42, 0x79, 0xe0, 0x9c, 0xca, 0x44, 0xc4, 0x52, 0x58, 0xdc, 0x81, 0xc6, 0x04, + 0xe9, 0x25, 0x33, 0xe1, 0xcd, 0x62, 0xd6, 0xa7, 0xd8, 0x63, 0x24, 0x03, 0xf4, 0xef, 0xc0, 0x95, + 0xe2, 0xf2, 0x8a, 0x0a, 0xb7, 0xa0, 0x72, 0x7c, 0x12, 0x29, 0x32, 0x2f, 0x16, 0x62, 0x5e, 0xfa, + 0x5e, 0x02, 0x7b, 0xf5, 0xbf, 0x2e, 0x41, 0x65, 0x67, 0x3a, 0xc9, 0x7f, 0x0c, 0x56, 0xe5, 0x8f, + 0xc1, 0xae, 0xe5, 0xf3, 0xd9, 0x1c, 0x43, 0x65, 0x79, 0xeb, 0x77, 0x40, 0x3b, 0xf0, 0xc3, 0x9f, + 0x9a, 0xa1, 0x2d, 0x6d, 0x65, 0x81, 0x33, 0x04, 0xb9, 0xce, 0xd3, 0x49, 0xa0, 0xd4, 0x3b, 0xb5, + 0xc5, 0x6d, 0x65, 0xc3, 0x39, 0xae, 0x59, 0x44, 0xca, 0xee, 0x4c, 0x27, 0xcb, 0xae, 0x34, 0x23, + 0x32, 0x36, 0x6c, 0xd6, 0xf5, 0xbb, 0xa0, 0xa5, 0x28, 0x54, 0x90, 0x3b, 0x7b, 0xe3, 0xad, 0x0d, + 0xce, 0x11, 0x62, 0x04, 0x50, 0x42, 0xe5, 0x38, 0xfa, 0xc1, 0xce, 0x78, 0xb4, 0xd7, 0x2b, 0xeb, + 0x3f, 0x82, 0x56, 0x22, 0x3f, 0x5b, 0x36, 0x15, 0xc4, 0x48, 0x80, 0xb7, 0xec, 0x82, 0x3c, 0x6f, + 0x51, 0x88, 0x26, 0x3d, 0x7b, 0x2b, 0x11, 0x3c, 0x06, 0x8a, 0x37, 0x54, 0xd5, 0xb5, 0xe4, 0x86, + 0xfa, 0x10, 0x16, 0x0d, 0x4a, 0xec, 0xa3, 0xe1, 0x4d, 0x9e, 0xec, 0x2a, 0xd4, 0x3d, 0xdf, 0x96, + 0xe9, 0x06, 0x0a, 0xc2, 0x9d, 0xd5, 0x63, 0x2b, 0x95, 0x96, 0xbe, 0xbd, 0x84, 0x45, 0xd4, 0x92, + 0x45, 0x46, 0x2b, 0x24, 0x9d, 0x4b, 0x33, 0x49, 0x67, 0xdc, 0x44, 0xd5, 0x97, 0xd9, 0xdf, 0x49, + 0x6a, 0xca, 0x03, 0x68, 0xda, 0x51, 0x4c, 0x62, 0xad, 0x74, 0x63, 0x0a, 0xeb, 0xf7, 0xe1, 0xf2, + 0x6a, 0x10, 0xb8, 0x67, 0x49, 0x35, 0x4e, 0x6d, 0xd4, 0xcf, 0x4a, 0x76, 0x25, 0x15, 0x17, 0x32, + 0xa8, 0x6f, 0x42, 0x3b, 0xc9, 0x30, 0x3c, 0x95, 0xb1, 0x49, 0x1a, 0xcf, 0x75, 0x0a, 0x21, 0x76, + 0x93, 0x11, 0xa3, 0x62, 0x6a, 0x7b, 0xe6, 0x7e, 0xcb, 0x50, 0x57, 0xea, 0x54, 0x40, 0xd5, 0xf2, + 0x6d, 0xde, 0xa8, 0x66, 0x50, 0x1b, 0xb9, 0x6a, 0x12, 0x1d, 0x26, 0x1e, 0xef, 0x24, 0x3a, 0xd4, + 0xff, 0xa9, 0x0c, 0x9d, 0x35, 0xca, 0xe7, 0x24, 0x67, 0xcc, 0x65, 0x31, 0x4b, 0x85, 0x2c, 0x66, + 0x3e, 0x63, 0x59, 0x2e, 0x64, 0x2c, 0x0b, 0x07, 0xaa, 0x14, 0xdd, 0xd4, 0xb7, 0xa1, 0x31, 0xf5, + 0x9c, 0xd3, 0xc4, 0x4e, 0x68, 0x46, 0x1d, 0xc1, 0x51, 0x24, 0x6e, 0x42, 0x0b, 0x4d, 0x89, 0xe3, + 0x71, 0x96, 0x90, 0x53, 0x7d, 0x79, 0xd4, 0x4c, 0x2e, 0xb0, 0xfe, 0xea, 0x5c, 0x60, 0xe3, 0xb5, + 0xb9, 0xc0, 0xe6, 0xeb, 0x72, 0x81, 0xda, 0x6c, 0x2e, 0xb0, 0xe8, 0x62, 0xc3, 0x39, 0x17, 0xfb, + 0x5d, 0x00, 0xfe, 0x08, 0xe6, 0x60, 0xea, 0xba, 0xca, 0x6f, 0xd1, 0x08, 0xb3, 0x39, 0x75, 0x5d, + 0x7d, 0x1b, 0xba, 0x09, 0x69, 0x95, 0x0a, 0xf8, 0x14, 0x2e, 0xa9, 0x2c, 0xbf, 0x0c, 0x55, 0xa2, + 0x8c, 0x35, 0x1b, 0xc9, 0x1f, 0x27, 0xe2, 0x55, 0x8f, 0xd1, 0xb5, 0xf3, 0x60, 0xa4, 0xff, 0xbc, + 0x04, 0x9d, 0xc2, 0x08, 0xf1, 0x30, 0xab, 0x19, 0x94, 0x48, 0x8a, 0xfb, 0xe7, 0x56, 0x79, 0x75, + 0xdd, 0xa0, 0x3c, 0x53, 0x37, 0xd0, 0xef, 0xa5, 0xd5, 0x00, 0x55, 0x03, 0x58, 0x48, 0x6b, 0x00, + 0x94, 0x36, 0x5f, 0x1d, 0x8d, 0x8c, 0x5e, 0x59, 0xd4, 0xa1, 0xbc, 0xb3, 0xd7, 0xab, 0xe8, 0x7f, + 0x51, 0x86, 0xce, 0xf0, 0x34, 0xa0, 0x0f, 0xc2, 0x5e, 0x1b, 0xaf, 0xe4, 0xf8, 0xaa, 0x5c, 0xe0, + 0xab, 0x1c, 0x87, 0x54, 0x54, 0x11, 0x94, 0x39, 0x04, 0x23, 0x18, 0xce, 0x4c, 0x2a, 0xce, 0x61, + 0xe8, 0xff, 0x03, 0xe7, 0x14, 0x34, 0x0a, 0xcc, 0x96, 0xb1, 0xb6, 0xa1, 0x9b, 0x90, 0x4d, 0x31, + 0xc6, 0x1b, 0x09, 0x2b, 0x7f, 0x02, 0xea, 0xa6, 0x89, 0x32, 0x06, 0xf4, 0x3f, 0x2e, 0x83, 0xc6, + 0x7c, 0x86, 0x87, 0xff, 0x48, 0xe9, 0xf5, 0x52, 0x56, 0x31, 0x49, 0x3b, 0x97, 0x9f, 0xc8, 0xb3, + 0x4c, 0xb7, 0xcf, 0xad, 0x32, 0xaa, 0x74, 0x1a, 0x67, 0x14, 0x28, 0x9d, 0x76, 0x0d, 0x34, 0x76, + 0xcb, 0xa6, 0x2a, 0x5d, 0x5f, 0x35, 0xd8, 0x4f, 0x7b, 0xee, 0x90, 0x65, 0x89, 0x65, 0x38, 0x51, + 0x6f, 0x40, 0xed, 0x62, 0xec, 0xd6, 0x49, 0xa2, 0x89, 0x02, 0x45, 0x1a, 0xb3, 0x14, 0x39, 0x82, + 0x86, 0x3a, 0x1b, 0xba, 0xde, 0xcf, 0x77, 0x9e, 0xec, 0xec, 0x7e, 0x7f, 0xa7, 0xc0, 0x7d, 0xa9, + 0x73, 0x5e, 0xce, 0x3b, 0xe7, 0x15, 0xc4, 0xaf, 0xef, 0x3e, 0xdf, 0x19, 0xf5, 0xaa, 0xa2, 0x03, + 0x1a, 0x35, 0xc7, 0xc6, 0xf0, 0x45, 0xaf, 0x46, 0xd9, 0xa8, 0xf5, 0xcf, 0x86, 0x4f, 0x57, 0x7b, + 0xf5, 0xb4, 0x7e, 0xd5, 0xd0, 0xff, 0xa8, 0x04, 0x8b, 0x4c, 0x90, 0x7c, 0x62, 0x26, 0xff, 0x71, + 0x76, 0x95, 0x3f, 0xce, 0xfe, 0xbf, 0xcd, 0xc5, 0xe0, 0xa4, 0xa9, 0x93, 0x54, 0x8c, 0x39, 0x49, + 0xd8, 0x9c, 0x3a, 0xaa, 0x50, 0xfc, 0x77, 0x25, 0x18, 0x70, 0x4c, 0xf0, 0x38, 0x34, 0x83, 0xa3, + 0xef, 0x6d, 0x9f, 0xcb, 0x0a, 0x5c, 0xe4, 0x29, 0xdf, 0x86, 0x2e, 0x7d, 0xbe, 0xfe, 0x13, 0x77, + 0xac, 0x22, 0x57, 0x7e, 0xdd, 0x8e, 0xc2, 0xf2, 0x42, 0xe2, 0x11, 0xb4, 0xf9, 0x33, 0x77, 0xca, + 0x9a, 0x17, 0xaa, 0x9d, 0x85, 0x88, 0xa4, 0xc5, 0xa3, 0xb8, 0x36, 0xfb, 0x30, 0x9d, 0x94, 0x25, + 0x10, 0xce, 0x17, 0x34, 0xd5, 0x94, 0x11, 0xa5, 0x15, 0xee, 0xc3, 0xb5, 0xb9, 0xf7, 0x50, 0x6c, + 0x9f, 0x4b, 0xde, 0x32, 0xb7, 0xe9, 0xff, 0x5c, 0x82, 0xe6, 0xda, 0xd4, 0x3d, 0x26, 0x23, 0xf8, + 0x2e, 0x80, 0xb4, 0x0f, 0xa5, 0xfa, 0x5e, 0xbc, 0x44, 0xca, 0x41, 0x43, 0x0c, 0x7f, 0x31, 0xfe, + 0x29, 0x00, 0xdf, 0x71, 0x3c, 0x31, 0x03, 0xf5, 0x44, 0x54, 0x7d, 0x4c, 0x16, 0x50, 0x77, 0x79, + 0x6a, 0x06, 0xaa, 0xfa, 0x18, 0x25, 0x70, 0x56, 0x95, 0xad, 0xbc, 0xa2, 0x2a, 0x3b, 0xd8, 0x81, + 0x6e, 0x71, 0x89, 0x39, 0x49, 0xb3, 0x0f, 0x8a, 0x5f, 0xbe, 0x9c, 0xa7, 0x61, 0xce, 0x87, 0xff, + 0x1c, 0x2e, 0xcd, 0x24, 0xe0, 0x5f, 0xa5, 0x31, 0x0b, 0x22, 0x53, 0x9e, 0x15, 0x99, 0x8f, 0x61, + 0x71, 0x64, 0x46, 0xc7, 0x2a, 0xae, 0xc9, 0x8c, 0x77, 0x6c, 0x46, 0xc7, 0xe3, 0x94, 0xa8, 0x75, + 0x04, 0xb7, 0x6c, 0xfd, 0x21, 0x88, 0xfc, 0x68, 0x45, 0x7f, 0x8c, 0x57, 0x71, 0xf8, 0x44, 0xc6, + 0x66, 0xe2, 0x65, 0x20, 0x02, 0x89, 0xb7, 0xf2, 0xb7, 0x25, 0xa8, 0x62, 0x20, 0x20, 0xee, 0x81, + 0xf6, 0x99, 0x34, 0xc3, 0x78, 0x5f, 0x9a, 0xb1, 0x28, 0x38, 0xfd, 0x03, 0xa2, 0x5b, 0xf6, 0x35, + 0x8d, 0xbe, 0xf0, 0xa0, 0x24, 0x96, 0xf9, 0x5b, 0xdf, 0xe4, 0x1b, 0xe6, 0x4e, 0x12, 0x50, 0x50, + 0xc0, 0x31, 0x28, 0xcc, 0xd7, 0x17, 0x96, 0x68, 0xfc, 0xe7, 0xbe, 0xe3, 0xad, 0xf3, 0x17, 0xa6, + 0x62, 0x36, 0x00, 0x99, 0x9d, 0x21, 0xee, 0x41, 0x7d, 0x2b, 0xc2, 0x48, 0xe7, 0xfc, 0x50, 0x22, + 0x7e, 0x3e, 0x08, 0xd2, 0x17, 0x56, 0xfe, 0xbc, 0x06, 0xd5, 0x1f, 0xc9, 0xd0, 0x17, 0x1f, 0x43, + 0x43, 0x7d, 0x7b, 0x24, 0x72, 0xdf, 0x18, 0x0d, 0x28, 0xe9, 0x32, 0xf3, 0x51, 0x12, 0xed, 0xd2, + 0xe3, 0xf7, 0xcb, 0xca, 0x4e, 0x22, 0xfb, 0x34, 0xea, 0xdc, 0xa1, 0x3e, 0x81, 0xde, 0x5e, 0x1c, + 0x4a, 0x73, 0x92, 0x1b, 0x5e, 0x24, 0xd5, 0xbc, 0x1a, 0x16, 0xd1, 0xeb, 0x2e, 0xd4, 0x39, 0x9c, + 0x9c, 0x99, 0x30, 0x5b, 0xa0, 0xa2, 0xc1, 0x1f, 0x42, 0x6b, 0xef, 0xc8, 0x9f, 0xba, 0xf6, 0x9e, + 0x0c, 0x4f, 0xa4, 0xc8, 0x45, 0x44, 0x83, 0x5c, 0x5b, 0x5f, 0x10, 0x0f, 0xa1, 0x8e, 0x2f, 0x12, + 0x4e, 0xc4, 0x62, 0x2e, 0x6a, 0x62, 0x36, 0x19, 0x88, 0x3c, 0x2a, 0xa1, 0x94, 0xf8, 0x10, 0x34, + 0x76, 0xdf, 0xd1, 0x79, 0x6f, 0xa8, 0x88, 0x80, 0x8f, 0x91, 0x73, 0xeb, 0xf5, 0x05, 0xb1, 0x04, + 0x90, 0x8b, 0x43, 0x5f, 0x35, 0xf2, 0x11, 0x74, 0xd6, 0x49, 0x13, 0xee, 0x86, 0xab, 0xfb, 0x7e, + 0x18, 0x8b, 0xd9, 0xef, 0x21, 0x07, 0xb3, 0x08, 0x7d, 0x01, 0x23, 0xba, 0x51, 0x78, 0xc6, 0xe3, + 0x17, 0x55, 0xf8, 0x9e, 0xed, 0x37, 0x87, 0x2e, 0xe2, 0x9b, 0xa9, 0x5c, 0xa5, 0x5e, 0xfb, 0xbc, + 0x6a, 0x17, 0x93, 0x88, 0x65, 0x80, 0x48, 0x04, 0x59, 0x48, 0x21, 0xde, 0xe2, 0xca, 0xdb, 0x4c, + 0x88, 0x71, 0x7e, 0x4a, 0x16, 0x3e, 0xf0, 0x94, 0x73, 0xe1, 0xc4, 0xcc, 0x94, 0x6f, 0x41, 0x3b, + 0x1f, 0x0a, 0x08, 0x2a, 0x21, 0xcd, 0x09, 0x0e, 0x8a, 0xd3, 0x56, 0xfe, 0xb3, 0x06, 0xf5, 0xef, + 0xfb, 0xe1, 0xb1, 0x0c, 0xc5, 0x1d, 0xa8, 0x53, 0x0d, 0x55, 0xc9, 0x52, 0x5a, 0x4f, 0x9d, 0x47, + 0xbb, 0xf7, 0x41, 0x23, 0xce, 0x40, 0x61, 0x67, 0x7e, 0xa5, 0xff, 0xef, 0xf0, 0xe2, 0x9c, 0xd5, + 0x24, 0xe6, 0xee, 0x32, 0xb7, 0xa6, 0xdf, 0x18, 0x14, 0x6a, 0x9c, 0x03, 0x7a, 0xd2, 0x27, 0x2f, + 0xf6, 0x50, 0x3e, 0x1f, 0x94, 0xd0, 0xa7, 0xd8, 0xe3, 0xc7, 0xc3, 0x41, 0xd9, 0xff, 0x13, 0x58, + 0xfc, 0xb3, 0x3f, 0x04, 0xe8, 0x0b, 0xe2, 0x3e, 0xd4, 0x95, 0x89, 0x59, 0xcc, 0x14, 0x61, 0x72, + 0xc3, 0x5e, 0x1e, 0xa5, 0x26, 0x3c, 0x84, 0x3a, 0x9b, 0x63, 0x9e, 0x50, 0x88, 0x45, 0x98, 0x4f, + 0x8b, 0x3e, 0xb4, 0xbe, 0x20, 0xee, 0x42, 0x43, 0x55, 0x48, 0xc5, 0x9c, 0x72, 0xe9, 0xb9, 0x17, + 0xab, 0xb3, 0xaf, 0xc5, 0xeb, 0x17, 0xdc, 0x55, 0x5e, 0xbf, 0xe8, 0x8a, 0xb1, 0xe8, 0x1b, 0xd2, + 0x92, 0x4e, 0x2e, 0x99, 0x26, 0x12, 0x8a, 0xcc, 0xd1, 0x5f, 0x9f, 0x40, 0xa7, 0x90, 0x78, 0x13, + 0xfd, 0x84, 0x2d, 0x66, 0x73, 0x71, 0xe7, 0xb4, 0xc6, 0x77, 0x40, 0x53, 0xa9, 0x82, 0x7d, 0xc5, + 0x18, 0x73, 0x12, 0x13, 0x83, 0xf3, 0xb9, 0x02, 0x52, 0x05, 0x3f, 0x80, 0xcb, 0x73, 0x6c, 0xab, + 0xa0, 0x2f, 0x5c, 0x2f, 0x76, 0x1e, 0x06, 0x37, 0x2e, 0xec, 0x4f, 0x09, 0xf0, 0xab, 0x89, 0xd3, + 0x77, 0x01, 0x32, 0x13, 0xc3, 0xb2, 0x71, 0xce, 0x40, 0x0d, 0xae, 0xce, 0xa2, 0x93, 0x4d, 0xd7, + 0xfa, 0x7f, 0xff, 0xd5, 0xf5, 0xd2, 0x2f, 0xbe, 0xba, 0x5e, 0xfa, 0xb7, 0xaf, 0xae, 0x97, 0x7e, + 0xfe, 0xcb, 0xeb, 0x0b, 0xbf, 0xf8, 0xe5, 0xf5, 0x85, 0x7f, 0xfc, 0xe5, 0xf5, 0x85, 0xfd, 0x3a, + 0xfd, 0x91, 0xee, 0xd1, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0xae, 0x6b, 0x5c, 0xe0, 0xbe, 0x37, + 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -10958,6 +10969,16 @@ func (m *Num) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Bump { + i-- + if m.Bump { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x28 + } if m.Type != 0 { i = encodeVarintPb(dAtA, i, uint64(m.Type)) i-- @@ -13437,6 +13458,9 @@ func (m *Num) Size() (n int) { if m.Type != 0 { n += 1 + sovPb(uint64(m.Type)) } + if m.Bump { + n += 2 + } return n } @@ -23958,6 +23982,26 @@ func (m *Num) Unmarshal(dAtA []byte) error { break } } + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Bump", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowPb + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Bump = bool(v != 0) default: iNdEx = preIndex skippy, err := skipPb(dAtA[iNdEx:]) diff --git a/systest/backup/common/utils.go b/systest/backup/common/utils.go index 02d1aafa777..866a1a7ae8f 100644 --- a/systest/backup/common/utils.go +++ b/systest/backup/common/utils.go @@ -25,7 +25,6 @@ import ( "github.com/dgraph-io/dgraph/testutil" "github.com/dgraph-io/dgraph/worker" - "github.com/dgraph-io/dgraph/x" ) var ( @@ -47,8 +46,8 @@ func RunFailingRestore(t *testing.T, backupLocation, lastDir string, commitTs ui // calling restore. require.NoError(t, os.RemoveAll(restoreDir)) - result := worker.RunRestore("./data/restore", backupLocation, lastDir, - x.Sensitive(nil), options.Snappy, 0) + result := worker.RunOfflineRestore(restoreDir, backupLocation, lastDir, + "", nil, options.Snappy, 0) require.Error(t, result.Err) require.Contains(t, result.Err.Error(), "expected a BackupNum value of 1") } diff --git a/systest/backup/encryption/backup_test.go b/systest/backup/encryption/backup_test.go index ae2e511da88..e9f0bce0412 100644 --- a/systest/backup/encryption/backup_test.go +++ b/systest/backup/encryption/backup_test.go @@ -320,10 +320,8 @@ func runRestore(t *testing.T, lastDir string, commitTs uint64) map[string]string t.Logf("--- Restoring from: %q", localBackupDst) testutil.KeyFile = "../../../ee/enc/test-fixtures/enc-key" - key, err := ioutil.ReadFile("../../../ee/enc/test-fixtures/enc-key") - require.NoError(t, err) - result := worker.RunRestore("./data/restore", localBackupDst, lastDir, - x.Sensitive(key), options.Snappy, 0) + result := worker.RunOfflineRestore(restoreDir, localBackupDst, lastDir, + testutil.KeyFile, nil, options.Snappy, 0) require.NoError(t, result.Err) for i, pdir := range []string{"p1", "p2", "p3"} { @@ -358,15 +356,10 @@ func runFailingRestore(t *testing.T, backupLocation, lastDir string, commitTs ui // Recreate the restore directory to make sure there's no previous data when // calling restore. require.NoError(t, os.RemoveAll(restoreDir)) + keyFile := "../../../ee/enc/test-fixtures/enc-key" - // Get key. - config := getEncConfig() - config.Set("encryption", ee.BuildEncFlag("../../../ee/enc/test-fixtures/enc-key")) - keys, err := ee.GetKeys(config) - require.NoError(t, err) - require.NotNil(t, keys.EncKey) - - result := worker.RunRestore("./data/restore", backupLocation, lastDir, keys.EncKey, options.Snappy, 0) + result := worker.RunOfflineRestore(restoreDir, backupLocation, lastDir, keyFile, nil, + options.Snappy, 0) require.Error(t, result.Err) require.Contains(t, result.Err.Error(), "expected a BackupNum value of 1") } diff --git a/systest/backup/filesystem/backup_test.go b/systest/backup/filesystem/backup_test.go index 2e831e8fbf9..a567cf5ac32 100644 --- a/systest/backup/filesystem/backup_test.go +++ b/systest/backup/filesystem/backup_test.go @@ -466,7 +466,8 @@ func runRestore(t *testing.T, backupLocation, lastDir string, commitTs uint64) m require.NoError(t, os.RemoveAll(restoreDir)) t.Logf("--- Restoring from: %q", backupLocation) - result := worker.RunRestore("./data/restore", backupLocation, lastDir, x.Sensitive(nil), options.Snappy, 0) + result := worker.RunOfflineRestore(restoreDir, backupLocation, + lastDir, "", nil, options.Snappy, 0) require.NoError(t, result.Err) for i, pdir := range []string{"p1", "p2", "p3"} { diff --git a/systest/backup/minio-large/backup_test.go b/systest/backup/minio-large/backup_test.go index eb742504441..e2eea45b6ac 100644 --- a/systest/backup/minio-large/backup_test.go +++ b/systest/backup/minio-large/backup_test.go @@ -186,7 +186,8 @@ func runRestore(t *testing.T, backupLocation, lastDir string, commitTs uint64) m require.NoError(t, os.MkdirAll(restoreDir, os.ModePerm)) t.Logf("--- Restoring from: %q", backupLocation) - result := worker.RunRestore("./data/restore", backupLocation, lastDir, x.Sensitive(nil), options.Snappy, 0) + result := worker.RunOfflineRestore(restoreDir, backupLocation, + lastDir, "", nil, options.Snappy, 0) require.NoError(t, result.Err) restored1, err := testutil.GetPredicateValues("./data/restore/p1", x.GalaxyAttr("name1"), commitTs) diff --git a/systest/backup/minio/backup_test.go b/systest/backup/minio/backup_test.go index 44573869a54..fdedc7c6573 100644 --- a/systest/backup/minio/backup_test.go +++ b/systest/backup/minio/backup_test.go @@ -345,8 +345,8 @@ func runRestore(t *testing.T, lastDir string, commitTs uint64) map[string]string require.NoError(t, os.RemoveAll(restoreDir)) t.Logf("--- Restoring from: %q", localBackupDst) - result := worker.RunRestore("./data/restore", localBackupDst, lastDir, - x.Sensitive(nil), options.Snappy, 0) + result := worker.RunOfflineRestore(restoreDir, localBackupDst, + lastDir, "", nil, options.Snappy, 0) require.NoError(t, result.Err) for i, pdir := range []string{"p1", "p2", "p3"} { @@ -368,7 +368,8 @@ func runFailingRestore(t *testing.T, backupLocation, lastDir string, commitTs ui // calling restore. require.NoError(t, os.RemoveAll(restoreDir)) - result := worker.RunRestore("./data/restore", backupLocation, lastDir, x.Sensitive(nil), options.Snappy, 0) + result := worker.RunOfflineRestore(restoreDir, backupLocation, + lastDir, "", nil, options.Snappy, 0) require.Error(t, result.Err) require.Contains(t, result.Err.Error(), "expected a BackupNum value of 1") } diff --git a/systest/backup/multi-tenancy/backup_test.go b/systest/backup/multi-tenancy/backup_test.go index 68340da83fd..a17d769be9e 100644 --- a/systest/backup/multi-tenancy/backup_test.go +++ b/systest/backup/multi-tenancy/backup_test.go @@ -19,21 +19,15 @@ import ( "bytes" "context" "encoding/json" - "fmt" - "math" - "os" - "path/filepath" "strings" "testing" - "github.com/dgraph-io/badger/v3/options" "github.com/dgraph-io/dgo/v210" "github.com/dgraph-io/dgo/v210/protos/api" "github.com/stretchr/testify/require" "github.com/dgraph-io/dgraph/systest/backup/common" "github.com/dgraph-io/dgraph/testutil" - "github.com/dgraph-io/dgraph/worker" "github.com/dgraph-io/dgraph/x" ) @@ -54,7 +48,7 @@ func TestBackupMultiTenancy(t *testing.T) { ctx := context.Background() dg := testutil.DgClientWithLogin(t, "groot", "password", x.GalaxyNamespace) - require.NoError(t, dg.Alter(ctx, &api.Operation{DropAll: true})) + testutil.DropAll(t, dg) galaxyCreds := &testutil.LoginParams{UserID: "groot", Passwd: "password", Namespace: x.GalaxyNamespace} galaxyToken := testutil.Login(t, galaxyCreds) @@ -78,10 +72,6 @@ func TestBackupMultiTenancy(t *testing.T) { addData := func(dg *dgo.Dgraph, name string) *api.Response { var buf bytes.Buffer - for i := 0; i < 10000; i++ { - buf.Write([]byte(fmt.Sprintf(`<_:x%d> "%s" . - `, i, name))) - } // Add initial data. _, err = dg.NewTxn().Mutate(ctx, &api.Mutation{ CommitNow: true, @@ -92,11 +82,11 @@ func TestBackupMultiTenancy(t *testing.T) { original, err := dg.NewTxn().Mutate(ctx, &api.Mutation{ CommitNow: true, SetNquads: []byte(` - <_:x1> "BIRDS MAN OR (THE UNEXPECTED VIRTUE OF IGNORANCE)" . - <_:x2> "Spotlight" . - <_:x3> "Moonlight" . - <_:x4> "THE SHAPE OF WATERLOO" . - <_:x5> "BLACK PUNTER" . + <_:x1> "a" . + <_:x2> "b" . + <_:x3> "c" . + <_:x4> "d" . + <_:x5> "e" . `), }) require.NoError(t, err) @@ -113,200 +103,28 @@ func TestBackupMultiTenancy(t *testing.T) { // Send backup request. _ = runBackup(t, galaxyToken, 3, 1) - restored := runRestore(t, copyBackupDir, "", math.MaxUint64, []uint64{x.GalaxyNamespace, ns}) - - preds := []string{"dgraph.graphql.schema", "name", "dgraph.graphql.xid", "dgraph.type", "movie", - "dgraph.graphql.p_query", "dgraph.drop.op", "dgraph.xid", "dgraph.acl.rule", - "dgraph.password", "dgraph.user.group", "dgraph.rule.predicate", "dgraph.rule.permission"} - preds = append(preds, preds...) - types := []string{"Node", "dgraph.graphql", "dgraph.graphql.persisted_query", - "dgraph.type.Rule", "dgraph.type.User", "dgraph.type.Group"} // ACL - types = append(types, types...) - testutil.CheckSchema(t, preds, types) - - verifyUids := func(dg *dgo.Dgraph, name string, count int) { - query := fmt.Sprintf(` - { - me(func: eq(name, "%s")) { - count(uid) - } - }`, name) - res, err := dg.NewTxn().Query(context.Background(), query) - require.NoError(t, err) - require.JSONEq(t, string(res.GetJson()), fmt.Sprintf(`{"me":[{"count":%d}]}`, count)) - } - verifyUids(dg, "galaxy", 10000) - verifyUids(dg1, "ns", 10000) - - checks := []struct { - blank, expected string - }{ - {blank: "x1", expected: "BIRDS MAN OR (THE UNEXPECTED VIRTUE OF IGNORANCE)"}, - {blank: "x2", expected: "Spotlight"}, - {blank: "x3", expected: "Moonlight"}, - {blank: "x4", expected: "THE SHAPE OF WATERLOO"}, - {blank: "x5", expected: "BLACK PUNTER"}, - } - for ns, orig := range original { - for _, check := range checks { - require.EqualValues(t, check.expected, restored[ns][orig.Uids[check.blank]]) - } - } - - addMoreData := func(dg *dgo.Dgraph, ns uint64) *api.Response { - // Add more data for the incremental backup. - incr1, err := dg.NewTxn().Mutate(ctx, &api.Mutation{ - CommitNow: true, - SetNquads: []byte(fmt.Sprintf(` - <%s> "Birdman or (The Unexpected Virtue of Ignorance)" . - <%s> "The Shape of Waterloo" . - `, original[ns].Uids["x1"], original[ns].Uids["x4"])), - }) - t.Logf("%+v", incr1) - require.NoError(t, err) - - // Update schema and types to make sure updates to the schema are backed up. - require.NoError(t, dg.Alter(ctx, &api.Operation{Schema: ` - movie: string . - actor: string . - name: string @index(hash) . - type Node { - movie - } - type NewNode { - actor - }`})) - return incr1 - } - - incr1 := make(map[uint64]*api.Response) - incr1[x.GalaxyNamespace] = addMoreData(dg, x.GalaxyNamespace) - incr1[ns] = addMoreData(dg1, ns) - - // Perform first incremental backup. - _ = runBackup(t, galaxyToken, 6, 2) - restored = runRestore(t, copyBackupDir, "", - x.Max(incr1[x.GalaxyNamespace].Txn.CommitTs, incr1[ns].Txn.CommitTs), []uint64{x.GalaxyNamespace, ns}) - - // Check the predicates and types in the schema are as expected. - preds = append(preds, "actor", "actor") - types = append(types, "NewNode", "NewNode") - testutil.CheckSchema(t, preds, types) - - // Perform some checks on the restored values. - checks = []struct { - blank, expected string - }{ - {blank: "x1", expected: "Birdman or (The Unexpected Virtue of Ignorance)"}, - {blank: "x4", expected: "The Shape of Waterloo"}, - } - for ns, orig := range original { - for _, check := range checks { - require.EqualValues(t, check.expected, restored[ns][orig.Uids[check.blank]]) - } - } - - addMoreData2 := func(dg *dgo.Dgraph, ns uint64) *api.Response { - // Add more data for the incremental backup. - incr2, err := dg.NewTxn().Mutate(ctx, &api.Mutation{ - CommitNow: true, - SetNquads: []byte(fmt.Sprintf(` - <%s> "The Shape of Water" . - <%s> "The Black Panther" . - `, original[ns].Uids["x4"], original[ns].Uids["x5"])), - }) - require.NoError(t, err) - return incr2 - } - - incr2 := make(map[uint64]*api.Response) - incr2[x.GalaxyNamespace] = addMoreData2(dg, x.GalaxyNamespace) - incr2[ns] = addMoreData2(dg1, ns) + testutil.DropAll(t, dg) + sendRestoreRequest(t, alphaBackupDir, galaxyToken.AccessJwt) + testutil.WaitForRestore(t, dg) - // Perform second incremental backup. - _ = runBackup(t, galaxyToken, 9, 3) - restored = runRestore(t, copyBackupDir, "", - x.Max(incr2[x.GalaxyNamespace].Txn.CommitTs, incr2[ns].Txn.CommitTs), []uint64{x.GalaxyNamespace, ns}) - testutil.CheckSchema(t, preds, types) - - checks = []struct { - blank, expected string - }{ - {blank: "x4", expected: "The Shape of Water"}, - {blank: "x5", expected: "The Black Panther"}, - } - for ns, orig := range original { - for _, check := range checks { - require.EqualValues(t, check.expected, restored[ns][orig.Uids[check.blank]]) - } - } - - addMoreData3 := func(dg *dgo.Dgraph, ns uint64) *api.Response { - // Add more data for the incremental backup. - incr2, err := dg.NewTxn().Mutate(ctx, &api.Mutation{ - CommitNow: true, - SetNquads: []byte(fmt.Sprintf(` - <%s> "El laberinto del fauno" . - <%s> "Black Panther 2" . - `, original[ns].Uids["x4"], original[ns].Uids["x5"])), - }) - require.NoError(t, err) - return incr2 - } - incr3 := make(map[uint64]*api.Response) - incr3[x.GalaxyNamespace] = addMoreData3(dg, x.GalaxyNamespace) - incr3[ns] = addMoreData3(dg1, ns) - - // Perform second full backup. - _ = runBackupInternal(t, galaxyToken, true, 12, 4) - restored = runRestore(t, copyBackupDir, "", - x.Max(incr3[x.GalaxyNamespace].Txn.CommitTs, incr3[ns].Txn.CommitTs), []uint64{x.GalaxyNamespace, ns}) - testutil.CheckSchema(t, preds, types) - - // Check all the values were restored to their most recent value. - checks = []struct { - blank, expected string - }{ - {blank: "x1", expected: "Birdman or (The Unexpected Virtue of Ignorance)"}, - {blank: "x2", expected: "Spotlight"}, - {blank: "x3", expected: "Moonlight"}, - {blank: "x4", expected: "El laberinto del fauno"}, - {blank: "x5", expected: "Black Panther 2"}, - } - for ns, orig := range original { - for _, check := range checks { - require.EqualValues(t, check.expected, restored[ns][orig.Uids[check.blank]]) - } - } - - verifyUids(dg, "galaxy", 10000) - verifyUids(dg1, "ns", 10000) - - // Do a DROP_DATA. This will return an error. - err = dg1.Alter(ctx, &api.Operation{DropOp: api.Operation_DATA}) - require.Error(t, err) - require.Contains(t, err.Error(), "Drop data can only be called by the guardian of the galaxy") - verifyUids(dg, "galaxy", 10000) - verifyUids(dg1, "ns", 10000) + query := `{ q(func: has(movie)) { count(uid) } }` + expectedResponse := `{ "q": [{ "count": 5 }]}` + testutil.VerifyQueryResponse(t, dg, query, expectedResponse) + testutil.VerifyQueryResponse(t, dg1, query, expectedResponse) // After deleting a namespace in incremental backup, we should not be able to get the data from // banned namespace. require.NoError(t, testutil.DeleteNamespace(t, galaxyToken, ns)) - dirs := runBackup(t, galaxyToken, 15, 5) - restored = runRestore(t, copyBackupDir, "", math.MaxUint64, []uint64{x.GalaxyNamespace, ns}) - - // Check that we do not restore the data from ns namespace. - require.Len(t, restored[x.GalaxyNamespace], 5) - require.Len(t, restored[ns], 0) - verifyUids(dg, "galaxy", 10000) - - // Remove the full backup testDirs and verify restore catches the error. - require.NoError(t, os.RemoveAll(dirs[0])) - require.NoError(t, os.RemoveAll(dirs[3])) - common.RunFailingRestore(t, copyBackupDir, "", - x.Max(incr3[x.GalaxyNamespace].Txn.CommitTs, incr3[ns].Txn.CommitTs)) + _ = runBackup(t, galaxyToken, 6, 2) + testutil.DropAll(t, dg) + sendRestoreRequest(t, alphaBackupDir, galaxyToken.AccessJwt) + testutil.WaitForRestore(t, dg) + query = `{ q(func: has(movie)) { count(uid) } }` + expectedResponse = `{ "q": [{ "count": 5 }]}` + testutil.VerifyQueryResponse(t, dg, query, expectedResponse) + expectedResponse = `{ "q": [{ "count": 0 }]}` + testutil.VerifyQueryResponse(t, dg1, query, expectedResponse) - // Clean up test directories. common.DirCleanup(t) } @@ -356,30 +174,32 @@ func runBackupInternal(t *testing.T, token *testutil.HttpToken, forceFull bool, return dirs } -func runRestore(t *testing.T, backupLocation, lastDir string, commitTs uint64, - ns []uint64) map[uint64]map[string]string { - // Recreate the restore directory to make sure there's no previous data when - // calling restore. - require.NoError(t, os.RemoveAll(restoreDir)) - - t.Logf("--- Restoring from: %q", backupLocation) - result := worker.RunRestore("./data/restore", backupLocation, lastDir, x.Sensitive(nil), options.Snappy, 0) - require.NoError(t, result.Err) - - for i, pdir := range []string{"p1", "p2", "p3"} { - pdir = filepath.Join("./data/restore", pdir) - groupId, err := x.ReadGroupIdFile(pdir) - require.NoError(t, err) - require.Equal(t, uint32(i+1), groupId) +func sendRestoreRequest(t *testing.T, location string, token string) { + if location == "" { + location = "/data/backup" } + params := testutil.GraphQLParams{ + Query: `mutation restore($location: String!) { + restore(input: {location: $location}) { + code + message + } + }`, + Variables: map[string]interface{}{ + "location": location, + }, + } + resp := testutil.MakeGQLRequestWithAccessJwt(t, ¶ms, token) + resp.RequireNoGraphQLErrors(t) - restored := make(map[uint64]map[string]string) - var err error - pdir := "./data/restore/p1" - for _, n := range ns { - restored[n], err = testutil.GetPredicateValues(pdir, x.NamespaceAttr(n, "movie"), commitTs) + var restoreResp struct { + Restore struct { + Code string + Message string + } } - require.NoError(t, err) - t.Logf("--- Restored values: %+v\n", restored) - return restored + + require.NoError(t, json.Unmarshal(resp.Data, &restoreResp)) + require.Equal(t, restoreResp.Restore.Code, "Success") + return } diff --git a/systest/online-restore/online_restore_test.go b/systest/online-restore/online_restore_test.go index 399561e3fbd..957b3f4fe29 100644 --- a/systest/online-restore/online_restore_test.go +++ b/systest/online-restore/online_restore_test.go @@ -210,7 +210,7 @@ func TestBasicRestore(t *testing.T) { sendRestoreRequest(t, "", "youthful_rhodes3", 0) testutil.WaitForRestore(t, dg) // Snapshot must be taken just after the restore and hence the snapshotTs be updated. - require.NoError(t, x.RetryUntilSuccess(3, 1*time.Second, func() error { + require.NoError(t, x.RetryUntilSuccess(3, 2*time.Second, func() error { if getSnapshotTs(t) <= snapshotTs { return errors.Errorf("snapshot not taken after restore") } diff --git a/worker/acl_cache.go b/worker/acl_cache.go index df5dd952c0d..5992a4859c4 100644 --- a/worker/acl_cache.go +++ b/worker/acl_cache.go @@ -18,6 +18,7 @@ import ( "github.com/dgraph-io/dgraph/ee/acl" "github.com/dgraph-io/dgraph/x" + "github.com/pkg/errors" ) diff --git a/worker/acl_cache_test.go b/worker/acl_cache_test.go index 5067e1fbb1c..ab3e543547a 100644 --- a/worker/acl_cache_test.go +++ b/worker/acl_cache_test.go @@ -1,3 +1,4 @@ +//go:build !oss // +build !oss /* @@ -17,6 +18,7 @@ import ( "github.com/dgraph-io/dgraph/ee/acl" "github.com/dgraph-io/dgraph/x" + "github.com/stretchr/testify/require" ) diff --git a/worker/aggregator.go b/worker/aggregator.go index 94930dedf12..8b7bc492a8a 100644 --- a/worker/aggregator.go +++ b/worker/aggregator.go @@ -16,7 +16,9 @@ package worker -import "github.com/dgraph-io/dgraph/types" +import ( + "github.com/dgraph-io/dgraph/types" +) func couldApplyAggregatorOn(agrtr string, typ types.TypeID) bool { if !typ.IsScalar() { diff --git a/worker/backup.go b/worker/backup.go index 6338dc1056e..222ec0171eb 100644 --- a/worker/backup.go +++ b/worker/backup.go @@ -1,5 +1,3 @@ -// +build oss - /* * Copyright 2022 Dgraph Labs, Inc. and Contributors * @@ -20,25 +18,112 @@ package worker import ( "context" + "math" + "sync" + "github.com/dgraph-io/badger/v3" "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/x" - "github.com/golang/glog" + + "github.com/pkg/errors" ) -// Backup implements the Worker interface. -func (w *grpcWorker) Backup(ctx context.Context, req *pb.BackupRequest) (*pb.BackupResponse, error) { - glog.Warningf("Backup failed: %v", x.ErrNotSupported) - return nil, x.ErrNotSupported +// predicateSet is a map whose keys are predicates. It is meant to be used as a set. +type predicateSet map[string]struct{} + +// Manifest records backup details, these are values used during restore. +// Since is the timestamp from which the next incremental backup should start (it's set +// to the readTs of the current backup). +// Groups are the IDs of the groups involved. +type Manifest struct { + sync.Mutex + //Type is the type of backup, either full or incremental. + Type string `json:"type"` + // SinceTsDeprecated is kept for backward compatibility. Use readTs instead of sinceTs. + SinceTsDeprecated uint64 `json:"since"` + // ReadTs is the timestamp at which this backup was taken. This would be + // the since timestamp for the next incremental backup. + ReadTs uint64 `json:"read_ts"` + // Groups is the map of valid groups to predicates at the time the backup was created. + Groups map[uint32][]string `json:"groups"` + // BackupId is a unique ID assigned to all the backups in the same series + // (from the first full backup to the last incremental backup). + BackupId string `json:"backup_id"` + // BackupNum is a monotonically increasing number assigned to each backup in + // a series. The full backup as BackupNum equal to one and each incremental + // backup gets assigned the next available number. Used to verify the integrity + // of the data during a restore. + BackupNum uint64 `json:"backup_num"` + // Version specifies the Dgraph version, the backup was taken on. For the backup taken on older + // versions (<= 20.11), the predicates in Group map do not have namespace. Version will be zero + // for older versions. + Version int `json:"version"` + // Path is the name of the backup directory to which this manifest belongs to. + Path string `json:"path"` + // Encrypted indicates whether this backup was encrypted or not. + Encrypted bool `json:"encrypted"` + // DropOperations lists the various DROP operations that took place since the last backup. + // These are used during restore to redo those operations before applying the backup. + DropOperations []*pb.DropOperation `json:"drop_operations"` + // Compression keeps track of the compression that was used for the data. + Compression string `json:"compression"` } -func ProcessBackupRequest(ctx context.Context, req *pb.BackupRequest) error { - glog.Warningf("Backup failed: %v", x.ErrNotSupported) - return x.ErrNotSupported +// ValidReadTs function returns the valid read timestamp. The backup can have +// the readTs=0 if the backup was done on an older version of dgraph. The +// SinceTsDecprecated is kept for backward compatibility. +func (m *Manifest) ValidReadTs() uint64 { + if m.ReadTs == 0 { + return m.SinceTsDeprecated + } + return m.ReadTs } -func ProcessListBackups(ctx context.Context, location string, creds *x.MinioCredentials) ( - []*Manifest, error) { +type MasterManifest struct { + Manifests []*Manifest +} + +func (m *Manifest) getPredsInGroup(gid uint32) predicateSet { + preds, ok := m.Groups[gid] + if !ok { + return nil + } + + predSet := make(predicateSet) + for _, pred := range preds { + if m.Version == 0 { + // For older versions, preds set will contain attribute without namespace. + pred = x.NamespaceAttr(x.GalaxyNamespace, pred) + } + predSet[pred] = struct{}{} + } + return predSet +} + +// GetCredentialsFromRequest extracts the credentials from a backup request. +func GetCredentialsFromRequest(req *pb.BackupRequest) *x.MinioCredentials { + return &x.MinioCredentials{ + AccessKey: req.GetAccessKey(), + SecretKey: req.GetSecretKey(), + SessionToken: req.GetSessionToken(), + Anonymous: req.GetAnonymous(), + } +} + +func StoreExport(request *pb.ExportRequest, dir string, key x.Sensitive) error { + db, err := badger.OpenManaged(badger.DefaultOptions(dir). + WithSyncWrites(false). + WithValueThreshold(1 << 10). + WithNumVersionsToKeep(math.MaxInt32). + WithEncryptionKey(key)) + + if err != nil { + return err + } - return nil, x.ErrNotSupported + _, err = exportInternal(context.Background(), request, db, true) + // It is important to close the db before sending err to ch. Else, we will see a memory + // leak. + db.Close() + return errors.Wrapf(err, "cannot export data inside DB at %s", dir) } diff --git a/worker/backup_common.go b/worker/backup_common.go deleted file mode 100644 index e6a4e828274..00000000000 --- a/worker/backup_common.go +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Copyright 2022 Dgraph Labs, Inc. and Contributors - * - * 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 worker - -import ( - "context" - "math" - "sync" - - "github.com/dgraph-io/badger/v3" - "github.com/pkg/errors" - - "github.com/dgraph-io/dgraph/protos/pb" - "github.com/dgraph-io/dgraph/x" -) - -// predicateSet is a map whose keys are predicates. It is meant to be used as a set. -type predicateSet map[string]struct{} - -// Manifest records backup details, these are values used during restore. -// ReadTs will be used to create the next incremental backup. -// Groups are the IDs of the groups involved. -type Manifest struct { - sync.Mutex - //Type is the type of backup, either full or incremental. - Type string `json:"type"` - // SinceTsDeprecated is kept for backward compatibility. Use readTs instead of sinceTs. - SinceTsDeprecated uint64 `json:"since"` - // ReadTs is the timestamp at which this backup was taken. This would be - // the since timestamp for the next incremental backup. - ReadTs uint64 `json:"read_ts"` - // Groups is the map of valid groups to predicates at the time the backup was created. - Groups map[uint32][]string `json:"groups"` - // BackupId is a unique ID assigned to all the backups in the same series - // (from the first full backup to the last incremental backup). - BackupId string `json:"backup_id"` - // BackupNum is a monotonically increasing number assigned to each backup in - // a series. The full backup as BackupNum equal to one and each incremental - // backup gets assigned the next available number. Used to verify the integrity - // of the data during a restore. - BackupNum uint64 `json:"backup_num"` - // Version specifies the Dgraph version, the backup was taken on. For the backup taken on older - // versions (<= 20.11), the predicates in Group map do not have namespace. Version will be zero - // for older versions. - Version int `json:"version"` - // Path is the name of the backup directory to which this manifest belongs to. - Path string `json:"path"` - // Encrypted indicates whether this backup was encrypted or not. - Encrypted bool `json:"encrypted"` - // DropOperations lists the various DROP operations that took place since the last backup. - // These are used during restore to redo those operations before applying the backup. - DropOperations []*pb.DropOperation `json:"drop_operations"` - // Compression keeps track of the compression that was used for the data. - Compression string `json:"compression"` -} - -// ValidReadTs function returns the valid read timestamp. The backup can have -// the readTs=0 if the backup was done on an older version of dgraph. The -// SinceTsDecprecated is kept for backward compatibility. -func (m *Manifest) ValidReadTs() uint64 { - if m.ReadTs == 0 { - return m.SinceTsDeprecated - } - return m.ReadTs -} - -type MasterManifest struct { - Manifests []*Manifest -} - -func (m *Manifest) getPredsInGroup(gid uint32) predicateSet { - preds, ok := m.Groups[gid] - if !ok { - return nil - } - - predSet := make(predicateSet) - for _, pred := range preds { - if m.Version == 0 { - // For older versions, preds set will contain attribute without namespace. - pred = x.NamespaceAttr(x.GalaxyNamespace, pred) - } - predSet[pred] = struct{}{} - } - return predSet -} - -// GetCredentialsFromRequest extracts the credentials from a backup request. -func GetCredentialsFromRequest(req *pb.BackupRequest) *x.MinioCredentials { - return &x.MinioCredentials{ - AccessKey: req.GetAccessKey(), - SecretKey: req.GetSecretKey(), - SessionToken: req.GetSessionToken(), - Anonymous: req.GetAnonymous(), - } -} - -func StoreExport(request *pb.ExportRequest, dir string, key x.Sensitive) error { - db, err := badger.OpenManaged(badger.DefaultOptions(dir). - WithSyncWrites(false). - WithValueThreshold(1 << 10). - WithNumVersionsToKeep(math.MaxInt32). - WithEncryptionKey(key)) - - if err != nil { - return err - } - - _, err = exportInternal(context.Background(), request, db, true) - // It is important to close the db before sending err to ch. Else, we will see a memory - // leak. - db.Close() - return errors.Wrapf(err, "cannot export data inside DB at %s", dir) -} diff --git a/worker/backup_ee.go b/worker/backup_ee.go index 648fdcbdf43..2ddcd5c0010 100644 --- a/worker/backup_ee.go +++ b/worker/backup_ee.go @@ -1,3 +1,4 @@ +//go:build !oss // +build !oss /* @@ -14,19 +15,30 @@ package worker import ( "context" + "encoding/binary" + "encoding/hex" "fmt" + "io" "net/url" + "reflect" + "strings" "sync" "time" + "github.com/dgraph-io/badger/v3" + bpb "github.com/dgraph-io/badger/v3/pb" + "github.com/dgraph-io/badger/v3/y" + "github.com/dgraph-io/dgraph/ee/enc" "github.com/dgraph-io/dgraph/posting" "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/x" - ostats "go.opencensus.io/stats" + "github.com/dgraph-io/ristretto/z" "github.com/golang/glog" "github.com/golang/protobuf/proto" + "github.com/golang/snappy" "github.com/pkg/errors" + ostats "go.opencensus.io/stats" ) // Backup handles a request coming from another node. @@ -135,13 +147,12 @@ func ProcessBackupRequest(ctx context.Context, req *pb.BackupRequest) error { if err != nil { return err } - latestManifest, err := handler.GetLatestManifest(uri) + latestManifest, err := GetLatestManifest(handler, uri) if err != nil { return err } req.SinceTs = latestManifest.ValidReadTs() - // To force a full backup we'll set the sinceTs to zero. if req.ForceFull { req.SinceTs = 0 @@ -191,26 +202,24 @@ func ProcessBackupRequest(ctx context.Context, req *pb.BackupRequest) error { ctx, cancel := context.WithCancel(ctx) defer cancel() - var dropOperations []*pb.DropOperation - { // This is the code which sends out Backup requests and waits for them to finish. - resCh := make(chan BackupRes, len(state.Groups)) - for _, gid := range groups { - br := proto.Clone(req).(*pb.BackupRequest) - br.GroupId = gid - br.Predicates = predMap[gid] - go func(req *pb.BackupRequest) { - res, err := BackupGroup(ctx, req) - resCh <- BackupRes{res: res, err: err} - }(br) - } + resCh := make(chan BackupRes, len(state.Groups)) + for _, gid := range groups { + br := proto.Clone(req).(*pb.BackupRequest) + br.GroupId = gid + br.Predicates = predMap[gid] + go func(req *pb.BackupRequest) { + res, err := BackupGroup(ctx, req) + resCh <- BackupRes{res: res, err: err} + }(br) + } - for range groups { - if backupRes := <-resCh; backupRes.err != nil { - glog.Errorf("Error received during backup: %v", backupRes.err) - return backupRes.err - } else { - dropOperations = append(dropOperations, backupRes.res.GetDropOperations()...) - } + var dropOperations []*pb.DropOperation + for range groups { + if backupRes := <-resCh; backupRes.err != nil { + glog.Errorf("Error received during backup: %v", backupRes.err) + return backupRes.err + } else { + dropOperations = append(dropOperations, backupRes.res.GetDropOperations()...) } } @@ -260,3 +269,429 @@ func ProcessListBackups(ctx context.Context, location string, creds *x.MinioCred } return res, nil } + +// BackupProcessor handles the different stages of the backup process. +type BackupProcessor struct { + // DB is the Badger pstore managed by this node. + DB *badger.DB + // Request stores the backup request containing the parameters for this backup. + Request *pb.BackupRequest + + // txn is used for the iterators in the threadLocal + txn *badger.Txn + threads []*threadLocal +} + +type threadLocal struct { + Request *pb.BackupRequest + // pre-allocated pb.PostingList object. + pl pb.PostingList + // pre-allocated pb.BackupPostingList object. + bpl pb.BackupPostingList + alloc *z.Allocator + itr *badger.Iterator + buf *z.Buffer +} + +func NewBackupProcessor(db *badger.DB, req *pb.BackupRequest) *BackupProcessor { + bp := &BackupProcessor{ + DB: db, + Request: req, + threads: make([]*threadLocal, x.WorkerConfig.Badger.NumGoroutines), + } + if req.SinceTs > 0 && db != nil { + bp.txn = db.NewTransactionAt(req.ReadTs, false) + } + for i := range bp.threads { + buf := z.NewBuffer(32<<20, "Worker.BackupProcessor") + + bp.threads[i] = &threadLocal{ + Request: bp.Request, + buf: buf, + } + if bp.txn != nil { + iopt := badger.DefaultIteratorOptions + iopt.AllVersions = true + bp.threads[i].itr = bp.txn.NewIterator(iopt) + } + } + return bp +} + +func (pr *BackupProcessor) Close() { + for _, th := range pr.threads { + if pr.txn != nil { + th.itr.Close() + } + th.buf.Release() + } + if pr.txn != nil { + pr.txn.Discard() + } +} + +// LoadResult holds the output of a Load operation. +type LoadResult struct { + // Version is the timestamp at which the database is after loading a backup. + Version uint64 + // MaxLeaseUid is the max UID seen by the load operation. Needed to request zero + // for the proper number of UIDs. + MaxLeaseUid uint64 + // MaxLeaseNsId is the max namespace ID seen by the load operation. + MaxLeaseNsId uint64 + // The error, if any, of the load operation. + Err error +} + +// WriteBackup uses the request values to create a stream writer then hand off the data +// retrieval to stream.Orchestrate. The writer will create all the fd's needed to +// collect the data and later move to the target. +// Returns errors on failure, nil on success. +func (pr *BackupProcessor) WriteBackup(ctx context.Context) (*pb.BackupResponse, error) { + if err := ctx.Err(); err != nil { + return nil, err + } + uri, err := url.Parse(pr.Request.Destination) + if err != nil { + return nil, err + } + handler, err := NewUriHandler(uri, GetCredentialsFromRequest(pr.Request)) + if err != nil { + return nil, err + } + w, err := createBackupFile(handler, uri, pr.Request) + if err != nil { + return nil, err + } + glog.V(3).Infof("Backup manifest version: %d", pr.Request.SinceTs) + + eWriter, err := enc.GetWriter(x.WorkerConfig.EncryptionKey, w) + if err != nil { + return nil, err + } + + // Snappy is much faster than gzip compression, even with the BestSpeed + // gzip option. In fact, in my experiments, gzip compression caused the + // output speed to be ~30 MBps. Snappy can write at ~90 MBps, and overall + // the speed is similar to writing uncompressed data on disk. + // + // These are the times I saw: + // Without compression: 7m2s 33GB output. + // With snappy: 7m11s 9.5GB output. + // With snappy + S3: 7m54s 9.5GB output. + cWriter := snappy.NewBufferedWriter(eWriter) + + stream := pr.DB.NewStreamAt(pr.Request.ReadTs) + stream.LogPrefix = "Dgraph.Backup" + // Ignore versions less than given sinceTs timestamp, or skip older versions of + // the given key by returning an empty list. + // Do not do this for schema and type keys. Those keys always have a + // version of one. They're handled separately. + stream.SinceTs = pr.Request.SinceTs + stream.Prefix = []byte{x.ByteData} + + var response pb.BackupResponse + stream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { + tl := pr.threads[itr.ThreadId] + tl.alloc = itr.Alloc + + bitr := itr + // Use the threadlocal iterator because "itr" has the sinceTs set and + // it will not be able to read all the data. + if tl.itr != nil { + bitr = tl.itr + bitr.Seek(key) + } + + kvList, dropOp, err := tl.toBackupList(key, bitr) + if err != nil { + return nil, err + } + // we don't want to append a nil value to the slice, so need to check. + if dropOp != nil { + response.DropOperations = append(response.DropOperations, dropOp) + } + return kvList, nil + } + + predMap := make(map[string]struct{}) + for _, pred := range pr.Request.Predicates { + predMap[pred] = struct{}{} + } + stream.ChooseKey = func(item *badger.Item) bool { + parsedKey, err := x.Parse(item.Key()) + if err != nil { + glog.Errorf("error %v while parsing key %v during backup. Skipping...", + err, hex.EncodeToString(item.Key())) + return false + } + + // Do not choose keys that contain parts of a multi-part list. These keys + // will be accessed from the main list. + if parsedKey.HasStartUid { + return false + } + + // Skip backing up the schema and type keys. They will be backed up separately. + if parsedKey.IsSchema() || parsedKey.IsType() { + return false + } + _, ok := predMap[parsedKey.Attr] + return ok + } + + var maxVersion uint64 + stream.Send = func(buf *z.Buffer) error { + list, err := badger.BufferToKVList(buf) + if err != nil { + return err + } + for _, kv := range list.Kv { + if maxVersion < kv.Version { + maxVersion = kv.Version + } + } + return writeKVList(list, cWriter) + } + + // This is where the execution happens. + if err := stream.Orchestrate(ctx); err != nil { + glog.Errorf("While taking backup: %v", err) + return &response, err + } + + // This is used to backup the schema and types. + writePrefix := func(prefix byte) error { + tl := threadLocal{ + alloc: z.NewAllocator(1<<10, "BackupProcessor.WritePrefix"), + } + defer tl.alloc.Release() + + txn := pr.DB.NewTransactionAt(pr.Request.ReadTs, false) + defer txn.Discard() + // We don't need to iterate over all versions. + iopts := badger.DefaultIteratorOptions + iopts.Prefix = []byte{prefix} + + itr := txn.NewIterator(iopts) + defer itr.Close() + + list := &bpb.KVList{} + for itr.Rewind(); itr.Valid(); itr.Next() { + item := itr.Item() + // Don't export deleted items. + if item.IsDeletedOrExpired() { + continue + } + parsedKey, err := x.Parse(item.Key()) + if err != nil { + glog.Errorf("error %v while parsing key %v during backup. Skipping...", + err, hex.EncodeToString(item.Key())) + continue + } + // This check makes sense only for the schema keys. The types are not stored in it. + if _, ok := predMap[parsedKey.Attr]; !parsedKey.IsType() && !ok { + continue + } + kv := y.NewKV(tl.alloc) + if err := item.Value(func(val []byte) error { + kv.Value = append(kv.Value, val...) + return nil + }); err != nil { + return errors.Wrapf(err, "while copying value") + } + + backupKey, err := tl.toBackupKey(item.Key()) + if err != nil { + return err + } + kv.Key = backupKey + kv.UserMeta = tl.alloc.Copy([]byte{item.UserMeta()}) + kv.Version = item.Version() + kv.ExpiresAt = item.ExpiresAt() + list.Kv = append(list.Kv, kv) + } + return writeKVList(list, cWriter) + } + + for _, prefix := range []byte{x.ByteSchema, x.ByteType} { + if err := writePrefix(prefix); err != nil { + glog.Errorf("While writing prefix %d to backup: %v", prefix, err) + return &response, err + } + } + + if maxVersion > pr.Request.ReadTs { + glog.Errorf("Max timestamp seen during backup (%d) is greater than readTs (%d)", + maxVersion, pr.Request.ReadTs) + } + + glog.V(2).Infof("Backup group %d version: %d", pr.Request.GroupId, pr.Request.ReadTs) + if err = cWriter.Close(); err != nil { + glog.Errorf("While closing gzipped writer: %v", err) + return &response, err + } + + if err = w.Close(); err != nil { + glog.Errorf("While closing handler: %v", err) + return &response, err + } + glog.Infof("Backup complete: group %d at %d", pr.Request.GroupId, pr.Request.ReadTs) + return &response, nil +} + +// CompleteBackup will finalize a backup by writing the manifest at the backup destination. +func (pr *BackupProcessor) CompleteBackup(ctx context.Context, m *Manifest) error { + if err := ctx.Err(); err != nil { + return err + } + uri, err := url.Parse(pr.Request.Destination) + if err != nil { + return err + } + handler, err := NewUriHandler(uri, GetCredentialsFromRequest(pr.Request)) + if err != nil { + return err + } + + manifest, err := GetManifest(handler, uri) + if err != nil { + return err + } + manifest.Manifests = append(manifest.Manifests, m) + + if err := createManifest(handler, uri, manifest); err != nil { + return errors.Wrap(err, "Complete backup failed") + } + glog.Infof("Backup completed OK.") + return nil +} + +// GoString implements the GoStringer interface for Manifest. +func (m *Manifest) GoString() string { + return fmt.Sprintf(`Manifest{Since: %d, ReadTs: %d, Groups: %v, Encrypted: %v}`, + m.SinceTsDeprecated, m.ReadTs, m.Groups, m.Encrypted) +} + +func (tl *threadLocal) toBackupList(key []byte, itr *badger.Iterator) ( + *bpb.KVList, *pb.DropOperation, error) { + list := &bpb.KVList{} + var dropOp *pb.DropOperation + + item := itr.Item() + if item.Version() < tl.Request.SinceTs { + return list, nil, + errors.Errorf("toBackupList: Item.Version(): %d should be less than sinceTs: %d", + item.Version(), tl.Request.SinceTs) + } + if item.IsDeletedOrExpired() { + return list, nil, nil + } + + switch item.UserMeta() { + case posting.BitEmptyPosting, posting.BitCompletePosting, posting.BitDeltaPosting: + l, err := posting.ReadPostingList(key, itr) + if err != nil { + return nil, nil, errors.Wrapf(err, "while reading posting list") + } + + // Don't allocate kv on tl.alloc, because we don't need it by the end of this func. + kv, err := l.ToBackupPostingList(&tl.bpl, tl.alloc, tl.buf) + if err != nil { + return nil, nil, errors.Wrapf(err, "while rolling up list") + } + + backupKey, err := tl.toBackupKey(kv.Key) + if err != nil { + return nil, nil, err + } + + // check if this key was storing a DROP operation record. If yes, get the drop operation. + dropOp, err = checkAndGetDropOp(key, l, tl.Request.ReadTs) + if err != nil { + return nil, nil, err + } + + kv.Key = backupKey + list.Kv = append(list.Kv, kv) + default: + return nil, nil, errors.Errorf( + "Unexpected meta: %d for key: %s", item.UserMeta(), hex.Dump(key)) + } + return list, dropOp, nil +} + +func (tl *threadLocal) toBackupKey(key []byte) ([]byte, error) { + parsedKey, err := x.Parse(key) + if err != nil { + return nil, errors.Wrapf(err, "could not parse key %s", hex.Dump(key)) + } + bk := parsedKey.ToBackupKey() + + out := tl.alloc.Allocate(bk.Size()) + n, err := bk.MarshalToSizedBuffer(out) + return out[:n], err +} + +func writeKVList(list *bpb.KVList, w io.Writer) error { + if err := binary.Write(w, binary.LittleEndian, uint64(list.Size())); err != nil { + return err + } + buf, err := list.Marshal() + if err != nil { + return err + } + _, err = w.Write(buf) + return err +} + +func checkAndGetDropOp(key []byte, l *posting.List, readTs uint64) (*pb.DropOperation, error) { + isDropOpKey, err := x.IsDropOpKey(key) + if err != nil || !isDropOpKey { + return nil, err + } + + vals, err := l.AllValues(readTs) + if err != nil { + return nil, errors.Wrapf(err, "cannot read value of dgraph.drop.op") + } + switch len(vals) { + case 0: + // do nothing, it means this one was deleted with S * * deletion. + // So, no need to consider it. + return nil, nil + case 1: + val, ok := vals[0].Value.([]byte) + if !ok { + return nil, errors.Errorf("cannot convert value of dgraph.drop.op to byte array, "+ + "got type: %s, value: %v, tid: %v", reflect.TypeOf(vals[0].Value), vals[0].Value, + vals[0].Tid) + } + // A dgraph.drop.op record can have values in only one of the following formats: + // * DROP_ALL; + // * DROP_DATA; + // * DROP_ATTR;attrName + // So, accordingly construct the *pb.DropOperation. + dropOp := &pb.DropOperation{} + dropInfo := strings.Split(string(val), ";") + if len(dropInfo) != 2 { + return nil, errors.Errorf("Unexpected value: %s for dgraph.drop.op", val) + } + switch dropInfo[0] { + case "DROP_ALL": + dropOp.DropOp = pb.DropOperation_ALL + case "DROP_DATA": + dropOp.DropOp = pb.DropOperation_DATA + case "DROP_ATTR": + dropOp.DropOp = pb.DropOperation_ATTR + dropOp.DropValue = dropInfo[1] + case "DROP_NS": + dropOp.DropOp = pb.DropOperation_NS + dropOp.DropValue = dropInfo[1] // contains namespace. + } + return dropOp, nil + default: + // getting more than one values for a non-list predicate is an error + return nil, errors.Errorf("found multiple values for dgraph.drop.op: %v", vals) + } +} diff --git a/worker/backup_handler.go b/worker/backup_handler.go index 7885d7d31b0..1d40de13d48 100644 --- a/worker/backup_handler.go +++ b/worker/backup_handler.go @@ -1,3 +1,4 @@ +//go:build !oss // +build !oss /* @@ -13,14 +14,21 @@ package worker import ( + "bytes" "fmt" "io" + "io/ioutil" "net/url" - "sort" + "os" + "path/filepath" + "time" "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/x" + "github.com/golang/glog" + "github.com/minio/minio-go/v6" + "github.com/minio/minio-go/v6/pkg/credentials" "github.com/pkg/errors" ) @@ -53,254 +61,361 @@ const ( tmpManifest = `manifest_tmp.json` ) +func createBackupFile(h UriHandler, uri *url.URL, req *pb.BackupRequest) (io.WriteCloser, error) { + if !h.DirExists("./") { + if err := h.CreateDir("./"); err != nil { + return nil, errors.Wrap(err, "while creating backup file") + } + } + fileName := backupName(req.ReadTs, req.GroupId) + dir := fmt.Sprintf(backupPathFmt, req.UnixTs) + if err := h.CreateDir(dir); err != nil { + return nil, errors.Wrap(err, "while creating backup file") + } + backupFile := filepath.Join(dir, fileName) + w, err := h.CreateFile(backupFile) + return w, errors.Wrap(err, "while creating backup file") +} + +func backupName(since uint64, groupId uint32) string { + return fmt.Sprintf(backupNameFmt, since, groupId) +} + // UriHandler interface is implemented by URI scheme handlers. // When adding new scheme handles, for example 'azure://', an object will implement // this interface to supply Dgraph with a way to create or load backup files into DB. // For all methods below, the URL object is parsed as described in `newHandler' and // the Processor object has the DB, estimated tablets size, and backup parameters. type UriHandler interface { - // Handlers must know how to Write to their URI location. - // These function calls are used by both Create and Load. - io.WriteCloser - - // BytesWritten returns the number of bytes written. - BytesWritten() int - - // GetManifest returns the master manifest, containing information about all the - // backups. If the backup directory is using old formats (version < 21.03) of manifests, - // then it will return a consolidated master manifest. - GetManifest(*url.URL) (*MasterManifest, error) - - // GetManifests returns the list of manifest for the given backup series ID - // and backup number at the specified location. If backupNum is set to zero, - // all the manifests for the backup series will be returned. If it's greater - // than zero, manifests from one to backupNum will be returned. - GetManifests(*url.URL, string, uint64) ([]*Manifest, error) - - // GetLatestManifest reads the manifests at the given URL and returns the - // latest manifest. - GetLatestManifest(*url.URL) (*Manifest, error) - - // CreateBackupFile prepares the object or file to save the backup file. - CreateBackupFile(*url.URL, *pb.BackupRequest) error - - // CreateManifest creates the given manifest. - CreateManifest(*url.URL, *MasterManifest) error - - // Load will scan location URI for backup files, then load them via loadFn. - // It optionally takes the name of the last directory to consider. Any backup directories - // created after will be ignored. - // Objects implementing this function will be used for retrieving (dowload) backup files - // and loading the data into a DB. The restore CLI command uses this call. - Load(*url.URL, string, uint64, loadFn) LoadResult - - // Verify checks that the specified backup can be restored to a cluster with the - // given groups. The last manifest of that backup should have the same number of - // groups as given list of groups. - Verify(*url.URL, *pb.RestoreRequest, []uint32) error + // CreateDir creates a directory relative to the root path of the handler. + CreateDir(path string) error + // CreateFile creates a file relative to the root path of the handler. It also makes the + // handler's descriptor to point to this file. + CreateFile(path string) (io.WriteCloser, error) + // DirExists returns true if the directory relative to the root path of the handler exists. + DirExists(path string) bool + // FileExists returns true if the file relative to the root path of the handler exists. + FileExists(path string) bool + // JoinPath appends the given path to the root path of the handler. + JoinPath(path string) string + // ListPaths returns a list of all the valid paths from the given root path. The given root path + // should be relative to the handler's root path. + ListPaths(path string) []string + // Read reads the file at given relative path and returns the read bytes. + Read(path string) ([]byte, error) + // Rename renames the src file to the destination file. + Rename(src, dst string) error + // Stream would stream the path via an instance of io.ReadCloser. Close must be called at the + // end to release resources appropriately. + Stream(path string) (io.ReadCloser, error) } // NewUriHandler parses the requested URI and finds the corresponding UriHandler. // If the passed credentials are not nil, they will be used to override the // default credentials (only for backups to minio or S3). // Target URI formats: -// [scheme]://[host]/[path]?[args] -// [scheme]:///[path]?[args] -// /[path]?[args] (only for local or NFS) +// +// [scheme]://[host]/[path]?[args] +// [scheme]:///[path]?[args] +// /[path]?[args] (only for local or NFS) // // Target URI parts: -// scheme - service handler, one of: "file", "s3", "minio" -// host - remote address. ex: "dgraph.s3.amazonaws.com" -// path - directory, bucket or container at target. ex: "/dgraph/backups/" -// args - specific arguments that are ok to appear in logs. +// +// scheme - service handler, one of: "file", "s3", "minio" +// host - remote address. ex: "dgraph.s3.amazonaws.com" +// path - directory, bucket or container at target. ex: "/dgraph/backups/" +// args - specific arguments that are ok to appear in logs. // // Global args (if supported by the handler): -// secure - true|false turn on/off TLS. -// trace - true|false turn on/off HTTP tracing. -// compress - true|false turn on/off data compression. -// encrypt - true|false turn on/off data encryption. +// +// secure - true|false turn on/off TLS. +// trace - true|false turn on/off HTTP tracing. +// compress - true|false turn on/off data compression. +// encrypt - true|false turn on/off data encryption. // // Examples: -// s3://dgraph.s3.amazonaws.com/dgraph/backups?secure=true -// minio://localhost:9000/dgraph?secure=true -// file:///tmp/dgraph/backups -// /tmp/dgraph/backups?compress=gzip +// +// s3://dgraph.s3.amazonaws.com/dgraph/backups?secure=true +// minio://localhost:9000/dgraph?secure=true +// file:///tmp/dgraph/backups +// /tmp/dgraph/backups?compress=gzip func NewUriHandler(uri *url.URL, creds *x.MinioCredentials) (UriHandler, error) { switch uri.Scheme { case "file", "": - return &fileHandler{}, nil + return NewFileHandler(uri), nil case "minio", "s3": return NewS3Handler(uri, creds) } return nil, errors.Errorf("Unable to handle url: %s", uri) +} +// fileHandler is used for 'file:' URI scheme. +type fileHandler struct { + rootDir string + prefix string } -// loadFn is a function that will receive the current file being read. -// A reader, the backup groupId, and a map whose keys are the predicates to restore -// are passed as arguments. -type loadFn func(groupId uint32, in *loadBackupInput) (uint64, uint64, error) +func NewFileHandler(uri *url.URL) *fileHandler { + h := &fileHandler{} + h.rootDir, h.prefix = filepath.Split(uri.Path) + return h +} -// LoadBackup will scan location l for backup files in the given backup series and load them -// sequentially. Returns the maximum Since value on success, otherwise an error. -func LoadBackup(location, backupId string, backupNum uint64, creds *x.MinioCredentials, - fn loadFn) LoadResult { - uri, err := url.Parse(location) - if err != nil { - return LoadResult{Err: err} +func (h *fileHandler) DirExists(path string) bool { return pathExist(h.JoinPath(path)) } +func (h *fileHandler) FileExists(path string) bool { return pathExist(h.JoinPath(path)) } +func (h *fileHandler) Read(path string) ([]byte, error) { return ioutil.ReadFile(h.JoinPath(path)) } + +func (h *fileHandler) JoinPath(path string) string { + return filepath.Join(h.rootDir, h.prefix, path) +} +func (h *fileHandler) Stream(path string) (io.ReadCloser, error) { + return os.Open(h.JoinPath(path)) +} +func (h *fileHandler) ListPaths(path string) []string { + path = h.JoinPath(path) + return x.WalkPathFunc(path, func(path string, isDis bool) bool { + return true + }) +} +func (h *fileHandler) CreateDir(path string) error { + path = h.JoinPath(path) + if err := os.MkdirAll(path, 0755); err != nil { + return errors.Errorf("Create path failed to create path %s, got error: %v", path, err) } + return nil +} - h, err := NewUriHandler(uri, creds) - if err != nil { - return LoadResult{Err: errors.Errorf("Unsupported URI: %v", uri)} +type fileSyncer struct { + fp *os.File +} + +func (fs *fileSyncer) Write(p []byte) (n int, err error) { return fs.fp.Write(p) } +func (fs *fileSyncer) Close() error { + if err := fs.fp.Sync(); err != nil { + return errors.Wrapf(err, "while syncing file: %s", fs.fp.Name()) } + err := fs.fp.Close() + return errors.Wrapf(err, "while closing file: %s", fs.fp.Name()) +} - return h.Load(uri, backupId, backupNum, fn) +func (h *fileHandler) CreateFile(path string) (io.WriteCloser, error) { + path = h.JoinPath(path) + fp, err := os.Create(path) + return &fileSyncer{fp}, errors.Wrapf(err, "File handler failed to create file %s", path) +} + +func (h *fileHandler) Rename(src, dst string) error { + src = h.JoinPath(src) + dst = h.JoinPath(dst) + return os.Rename(src, dst) +} + +// pathExist checks if a path (file or dir) is found at target. +// Returns true if found, false otherwise. +func pathExist(path string) bool { + _, err := os.Stat(path) + if err == nil { + return true + } + return !os.IsNotExist(err) && !os.IsPermission(err) } -// VerifyBackup will access the backup location and verify that the specified backup can -// be restored to the cluster. -func VerifyBackup(req *pb.RestoreRequest, creds *x.MinioCredentials, currentGroups []uint32) error { - uri, err := url.Parse(req.GetLocation()) +// S3 Handler. + +// FillRestoreCredentials fills the empty values with the default credentials so that +// a restore request is sent to all the groups with the same credentials. +func FillRestoreCredentials(location string, req *pb.RestoreRequest) error { + uri, err := url.Parse(location) if err != nil { return err } - h, err := NewUriHandler(uri, creds) - if err != nil { - return errors.Wrap(err, "VerifyBackup") + defaultCreds := credentials.Value{ + AccessKeyID: req.AccessKey, + SecretAccessKey: req.SecretKey, + SessionToken: req.SessionToken, } + provider := x.MinioCredentialsProvider(uri.Scheme, defaultCreds) - return h.Verify(uri, req, currentGroups) + creds, _ := provider.Retrieve() // Error is always nil. + + req.AccessKey = creds.AccessKeyID + req.SecretKey = creds.SecretAccessKey + req.SessionToken = creds.SessionToken + + return nil } -// ListBackupManifests scans location l for backup files and returns the list of manifests. -func ListBackupManifests(l string, creds *x.MinioCredentials) ([]*Manifest, error) { - uri, err := url.Parse(l) - if err != nil { - return nil, err - } +// s3Handler is used for 's3:' and 'minio:' URI schemes. +type s3Handler struct { + bucketName string + objectPrefix string + creds *x.MinioCredentials + uri *url.URL + mc *x.MinioClient +} - h, err := NewUriHandler(uri, creds) - if err != nil { - return nil, errors.Wrap(err, "ListBackupManifests") +// NewS3Handler creates a new session, checks valid bucket at uri.Path, and configures a +// minio client. It also fills in values used by the handler in subsequent calls. +// Returns a new S3 minio client, otherwise a nil client with an error. +func NewS3Handler(uri *url.URL, creds *x.MinioCredentials) (*s3Handler, error) { + h := &s3Handler{ + creds: creds, + uri: uri, } - - m, err := h.GetManifest(uri) + mc, err := x.NewMinioClient(uri, creds) if err != nil { return nil, err } - return m.Manifests, nil + h.mc = mc + h.bucketName, h.objectPrefix = mc.ParseBucketAndPrefix(uri.Path) + return h, nil } -// filterManifests takes a list of manifests and returns the list of manifests -// that should be considered during a restore. -func filterManifests(manifests []*Manifest, backupId string) ([]*Manifest, error) { - // Go through the files in reverse order and stop when the latest full backup is found. - var filteredManifests []*Manifest - for i := len(manifests) - 1; i >= 0; i-- { - // If backupId is not empty, skip all the manifests that do not match the given - // backupId. If it's empty, do not skip any manifests as the default behavior is - // to restore the latest series of backups. - if len(backupId) > 0 && manifests[i].BackupId != backupId { - continue - } +func (h *s3Handler) CreateDir(path string) error { return nil } +func (h *s3Handler) DirExists(path string) bool { return true } - filteredManifests = append(filteredManifests, manifests[i]) - if manifests[i].Type == "full" { - break +func (h *s3Handler) FileExists(path string) bool { + objectPath := h.getObjectPath(path) + _, err := h.mc.StatObject(h.bucketName, objectPath, minio.StatObjectOptions{}) + if err != nil { + errResponse := minio.ToErrorResponse(err) + if errResponse.Code == "NoSuchKey" { + return false + } else { + glog.Errorf("Failed to verify object existence: %v", err) + return false } } + return true +} - // Reverse the filtered lists since the original iteration happened in reverse. - for i := len(filteredManifests)/2 - 1; i >= 0; i-- { - opp := len(filteredManifests) - 1 - i - filteredManifests[i], filteredManifests[opp] = filteredManifests[opp], filteredManifests[i] - } - - if err := verifyManifests(filteredManifests); err != nil { - return nil, err - } - - return filteredManifests, nil +func (h *s3Handler) JoinPath(path string) string { + return filepath.Join(h.bucketName, h.objectPrefix, path) } -func verifyManifests(manifests []*Manifest) error { - if len(manifests) == 0 { - return nil - } +func (h *s3Handler) Read(path string) ([]byte, error) { + objectPath := h.getObjectPath(path) + var buf bytes.Buffer - if manifests[0].BackupNum != 1 { - return errors.Errorf("expected a BackupNum value of 1 for first manifest but got %d", - manifests[0].BackupNum) + reader, err := h.mc.GetObject(h.bucketName, objectPath, minio.GetObjectOptions{}) + if err != nil { + return buf.Bytes(), errors.Wrap(err, "Failed to read s3 object") } + defer reader.Close() - backupId := manifests[0].BackupId - var backupNum uint64 - for _, manifest := range manifests { - if manifest.BackupId != backupId { - return errors.Errorf("found a manifest with backup ID %s but expected %s", - manifest.BackupId, backupId) - } + if _, err := buf.ReadFrom(reader); err != nil { + return buf.Bytes(), errors.Wrap(err, "Failed to read the s3 object") + } + return buf.Bytes(), nil +} - backupNum++ - if manifest.BackupNum != backupNum { - return errors.Errorf("found a manifest with backup number %d but expected %d", - manifest.BackupNum, backupNum) - } +func (h *s3Handler) Stream(path string) (io.ReadCloser, error) { + objectPath := h.getObjectPath(path) + reader, err := h.mc.GetObject(h.bucketName, objectPath, minio.GetObjectOptions{}) + if err != nil { + return nil, err } + return reader, nil +} - return nil +func (h *s3Handler) ListPaths(path string) []string { + var paths []string + done := make(chan struct{}) + defer close(done) + path = h.getObjectPath(path) + for object := range h.mc.ListObjects(h.bucketName, path, true, done) { + paths = append(paths, object.Key) + } + return paths } -func backupName(since uint64, groupId uint32) string { - return fmt.Sprintf(backupNameFmt, since, groupId) +type s3Writer struct { + pwriter *io.PipeWriter + preader *io.PipeReader + bucketName string + cerr chan error } -// verifyRequest verifies the manifests satisfy the requirements to process the given -// restore request. -func verifyRequest(req *pb.RestoreRequest, manifests []*Manifest, currentGroups []uint32) error { - if len(manifests) == 0 { - return errors.Errorf("No backups with the specified backup ID %s", req.GetBackupId()) +func (sw *s3Writer) Write(p []byte) (n int, err error) { return sw.pwriter.Write(p) } +func (sw *s3Writer) Close() error { + if sw.pwriter == nil { + return nil + } + if err := sw.pwriter.CloseWithError(nil); err != nil && err != io.EOF { + glog.Errorf("Unexpected error when closing pipe: %v", err) } + sw.pwriter = nil + glog.V(2).Infof("Backup waiting for upload to complete.") + return <-sw.cerr +} - if err := verifyManifests(manifests); err != nil { +// upload will block until it's done or an error occurs. +func (sw *s3Writer) upload(mc *x.MinioClient, object string) { + f := func() error { + start := time.Now() + + // We don't need to have a progress object, because we're using a Pipe. A write to Pipe + // would block until it can be fully read. So, the rate of the writes here would be equal to + // the rate of upload. We're already tracking progress of the writes in stream.Lists, so no + // need to track the progress of read. By definition, it must be the same. + // + // PutObject would block until sw.preader returns EOF. + n, err := mc.PutObject(sw.bucketName, object, sw.preader, -1, minio.PutObjectOptions{}) + glog.V(2).Infof("Backup sent %d bytes. Time elapsed: %s", + n, time.Since(start).Round(time.Second)) + + if err != nil { + // This should cause Write to fail as well. + glog.Errorf("Backup: Closing RW pipe due to error: %v", err) + if err := sw.pwriter.Close(); err != nil { + return err + } + if err := sw.preader.Close(); err != nil { + return err + } + } return err } + sw.cerr <- f() +} - lastManifest := manifests[len(manifests)-1] - if len(currentGroups) != len(lastManifest.Groups) { - return errors.Errorf("groups in cluster and latest backup manifest differ") - } +func (h *s3Handler) CreateFile(path string) (io.WriteCloser, error) { + objectPath := h.getObjectPath(path) + glog.V(2).Infof("Sending data to %s blob %q ...", h.uri.Scheme, objectPath) - for _, group := range currentGroups { - if _, ok := lastManifest.Groups[group]; !ok { - return errors.Errorf("groups in cluster and latest backup manifest differ") - } + sw := &s3Writer{ + bucketName: h.bucketName, + cerr: make(chan error, 1), } - return nil + sw.preader, sw.pwriter = io.Pipe() + go sw.upload(h.mc, objectPath) + return sw, nil } -func getManifests(manifests []*Manifest, backupId string, - backupNum uint64) ([]*Manifest, error) { - - manifests, err := filterManifests(manifests, backupId) +func (h *s3Handler) Rename(srcPath, dstPath string) error { + srcPath = h.getObjectPath(srcPath) + dstPath = h.getObjectPath(dstPath) + src := minio.NewSourceInfo(h.bucketName, srcPath, nil) + dst, err := minio.NewDestinationInfo(h.bucketName, dstPath, nil, nil) if err != nil { - return nil, err + return errors.Wrap(err, "Rename failed to create dstInfo") } - - // Sort manifests in the ascending order of their BackupNum so that the first - // manifest corresponds to the first full backup and so on. - sort.Slice(manifests, func(i, j int) bool { - return manifests[i].BackupNum < manifests[j].BackupNum - }) - - if backupNum > 0 { - if len(manifests) < int(backupNum) { - return nil, errors.Errorf("not enough backups to restore manifest with backupNum %d", - backupNum) + // We try copying 100 times, if it still fails, then the user should manually rename. + err = x.RetryUntilSuccess(100, time.Second, func() error { + if err := h.mc.CopyObject(dst, src); err != nil { + return errors.Wrapf(err, "While renaming object in s3, copy failed") } - manifests = manifests[:backupNum] + return nil + }) + if err != nil { + return err } - return manifests, nil + + err = h.mc.RemoveObject(h.bucketName, srcPath) + return errors.Wrap(err, "Rename failed to remove temporary file") +} + +func (h *s3Handler) getObjectPath(path string) string { + return filepath.Join(h.objectPrefix, path) } diff --git a/worker/backup_handler_test.go b/worker/backup_handler_test.go deleted file mode 100644 index b34049629c6..00000000000 --- a/worker/backup_handler_test.go +++ /dev/null @@ -1,123 +0,0 @@ -// +build !oss - -/* - * Copyright 2022 Dgraph Labs, Inc. and Contributors - * - * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt - */ - -package worker - -import ( - "testing" - - "github.com/stretchr/testify/require" -) - -func TestFilterManifestDefault(t *testing.T) { - manifests := []*Manifest{ - { - Type: "full", - BackupId: "aa", - BackupNum: 1, - }, - { - Type: "full", - BackupId: "ab", - BackupNum: 1, - }, - } - expected := []*Manifest{ - { - Type: "full", - BackupId: "ab", - BackupNum: 1, - }, - } - manifests, err := filterManifests(manifests, "") - require.NoError(t, err) - require.Equal(t, manifests, expected) -} - -func TestFilterManifestSelectSeries(t *testing.T) { - manifests := []*Manifest{ - { - Type: "full", - BackupId: "aa", - BackupNum: 1, - }, - { - Type: "full", - BackupId: "ab", - BackupNum: 1, - }, - } - expected := []*Manifest{ - { - Type: "full", - BackupId: "aa", - BackupNum: 1, - }, - } - manifests, err := filterManifests(manifests, "aa") - require.NoError(t, err) - require.Equal(t, manifests, expected) -} - -func TestFilterManifestMissingBackup(t *testing.T) { - manifests := []*Manifest{ - { - Type: "full", - BackupId: "aa", - BackupNum: 1, - }, - { - Type: "incremental", - BackupId: "aa", - BackupNum: 3, - }, - } - _, err := filterManifests(manifests, "aa") - require.Error(t, err) - require.Contains(t, err.Error(), "found a manifest with backup number") -} - -func TestFilterManifestMissingFirstBackup(t *testing.T) { - manifests := []*Manifest{ - { - Type: "incremental", - BackupId: "aa", - BackupNum: 2, - }, - { - Type: "incremental", - BackupId: "aa", - BackupNum: 3, - }, - } - _, err := filterManifests(manifests, "aa") - require.Error(t, err) - require.Contains(t, err.Error(), "expected a BackupNum value of 1 for first manifest") -} - -func TestFilterManifestDifferentSeries(t *testing.T) { - manifests := []*Manifest{ - { - Type: "full", - BackupId: "aa", - BackupNum: 1, - }, - { - Type: "incremental", - BackupId: "ab", - BackupNum: 2, - }, - } - _, err := filterManifests(manifests, "") - require.Error(t, err) - require.Contains(t, err.Error(), "found a manifest with backup ID") -} diff --git a/worker/backup_manifest.go b/worker/backup_manifest.go new file mode 100644 index 00000000000..482ae215a40 --- /dev/null +++ b/worker/backup_manifest.go @@ -0,0 +1,259 @@ +//go:build !oss +// +build !oss + +/* + * Copyright 2021 Dgraph Labs, Inc. and Contributors + * + * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt + */ + +package worker + +import ( + "encoding/json" + "net/url" + "path/filepath" + "sort" + "strings" + + "github.com/dgraph-io/dgraph/protos/pb" + "github.com/dgraph-io/dgraph/x" + + "github.com/pkg/errors" +) + +func verifyManifests(manifests []*Manifest) error { + if len(manifests) == 0 { + return nil + } + + lastIndex := len(manifests) - 1 + if manifests[lastIndex].BackupNum != 1 { + return errors.Errorf("expected a BackupNum value of 1 for first manifest but got %d", + manifests[lastIndex].BackupNum) + } + + backupId := manifests[lastIndex].BackupId + backupNum := uint64(len(manifests)) + for _, manifest := range manifests { + if manifest.BackupId != backupId { + return errors.Errorf("found a manifest with backup ID %s but expected %s", + manifest.BackupId, backupId) + } + + if manifest.BackupNum != backupNum { + return errors.Errorf("found a manifest with backup number %d but expected %d", + manifest.BackupNum, backupNum) + } + backupNum-- + } + + return nil +} + +func getManifestsToRestore( + h UriHandler, uri *url.URL, req *pb.RestoreRequest) ([]*Manifest, error) { + + if !h.DirExists("") { + return nil, errors.Errorf("getManifestsToRestore: The uri path: %q doesn't exists", + uri.Path) + } + manifest, err := getConsolidatedManifest(h, uri) + if err != nil { + return nil, errors.Wrap(err, "failed to get consolidated manifest") + } + return getFilteredManifests(h, manifest.Manifests, req) +} + +func getFilteredManifests(h UriHandler, manifests []*Manifest, + req *pb.RestoreRequest) ([]*Manifest, error) { + + // filter takes a list of manifests and returns the list of manifests + // that should be considered during a restore. + filter := func(manifests []*Manifest, backupId string) ([]*Manifest, error) { + // Go through the files in reverse order and stop when the latest full backup is found. + var out []*Manifest + for i := len(manifests) - 1; i >= 0; i-- { + // If backupId is not empty, skip all the manifests that do not match the given + // backupId. If it's empty, do not skip any manifests as the default behavior is + // to restore the latest series of backups. + if len(backupId) > 0 && manifests[i].BackupId != backupId { + continue + } + + out = append(out, manifests[i]) + if manifests[i].Type == "full" { + break + } + } + + if err := verifyManifests(out); err != nil { + return nil, err + } + return out, nil + } + + // validManifests are the ones for which the corresponding backup files exists. + var validManifests []*Manifest + for _, m := range manifests { + missingFiles := false + for g, _ := range m.Groups { + path := filepath.Join(m.Path, backupName(m.ValidReadTs(), g)) + if !h.FileExists(path) { + missingFiles = true + break + } + } + if !missingFiles { + validManifests = append(validManifests, m) + } + } + manifests, err := filter(validManifests, req.BackupId) + if err != nil { + return nil, err + } + + if req.BackupNum > 0 { + if len(manifests) < int(req.BackupNum) { + return nil, errors.Errorf("not enough backups to restore manifest with backupNum %d", + req.BackupNum) + } + manifests = manifests[len(manifests)-int(req.BackupNum):] + } + return manifests, nil +} + +// getConsolidatedManifest walks over all the backup directories and generates a master manifest. +func getConsolidatedManifest(h UriHandler, uri *url.URL) (*MasterManifest, error) { + // If there is a master manifest already, we just return it. + if h.FileExists(backupManifest) { + manifest, err := readMasterManifest(h, backupManifest) + if err != nil { + return &MasterManifest{}, errors.Wrap(err, "failed to read master manifest: ") + } + return manifest, nil + } + + // Otherwise, we create a master manifest by going through all the backup directories. + paths := h.ListPaths("") + + var manifestPaths []string + suffix := filepath.Join(string(filepath.Separator), backupManifest) + for _, p := range paths { + if strings.HasSuffix(p, suffix) { + manifestPaths = append(manifestPaths, p) + } + } + + sort.Strings(manifestPaths) + var mlist []*Manifest + + for _, path := range manifestPaths { + path = filepath.Dir(path) + _, path = filepath.Split(path) + m, err := readManifest(h, filepath.Join(path, backupManifest)) + if err != nil { + return nil, errors.Wrap(err, "while Getting latest manifest: ") + } + m.Path = path + mlist = append(mlist, m) + } + return &MasterManifest{Manifests: mlist}, nil +} + +func readManifest(h UriHandler, path string) (*Manifest, error) { + var m Manifest + b, err := h.Read(path) + if err != nil { + return &m, errors.Wrap(err, "readManifest failed to read the file: ") + } + if err := json.Unmarshal(b, &m); err != nil { + return &m, errors.Wrap(err, "readManifest failed to unmarshal: ") + } + return &m, nil +} + +func GetLatestManifest(h UriHandler, uri *url.URL) (*Manifest, error) { + manifest, err := GetManifest(h, uri) + if err != nil { + return &Manifest{}, errors.Wrap(err, "failed to get the manifest: ") + } + if len(manifest.Manifests) == 0 { + return &Manifest{}, nil + } + return manifest.Manifests[len(manifest.Manifests)-1], nil +} + +func readMasterManifest(h UriHandler, path string) (*MasterManifest, error) { + var m MasterManifest + b, err := h.Read(path) + if err != nil { + return &m, errors.Wrap(err, "readMasterManifest failed to read the file: ") + } + if err := json.Unmarshal(b, &m); err != nil { + return &m, errors.Wrap(err, "readMasterManifest failed to unmarshal: ") + } + return &m, nil +} + +func GetManifest(h UriHandler, uri *url.URL) (*MasterManifest, error) { + if !h.DirExists("") { + return &MasterManifest{}, errors.Errorf("getManifest: The uri path: %q doesn't exists", + uri.Path) + } + manifest, err := getConsolidatedManifest(h, uri) + if err != nil { + return nil, errors.Wrap(err, "failed to get consolidated manifest: ") + } + return manifest, nil +} + +func createManifest(h UriHandler, uri *url.URL, manifest *MasterManifest) error { + var err error + if !h.DirExists("./") { + if err := h.CreateDir("./"); err != nil { + return errors.Wrap(err, "createManifest failed to create path: ") + } + } + + w, err := h.CreateFile(tmpManifest) + if err != nil { + return errors.Wrap(err, "createManifest failed to create tmp path: ") + } + if err = json.NewEncoder(w).Encode(manifest); err != nil { + return err + } + if err := w.Close(); err != nil { + return err + } + // Move the tmpManifest to backupManifest, this operation is not atomic for s3. + // We try our best to move the file but if it fails then the user must move it manually. + err = h.Rename(tmpManifest, backupManifest) + return errors.Wrapf(err, "MOVING TEMPORARY MANIFEST TO MAIN MANIFEST FAILED!\n"+ + "It is possible that the manifest would have been corrupted. You must move "+ + "the file: %s to: %s in order to "+ + "fix the backup manifest.", tmpManifest, backupManifest) +} + +// ListBackupManifests scans location l for backup files and returns the list of manifests. +func ListBackupManifests(l string, creds *x.MinioCredentials) ([]*Manifest, error) { + uri, err := url.Parse(l) + if err != nil { + return nil, err + } + + h, err := NewUriHandler(uri, creds) + if err != nil { + return nil, errors.Wrap(err, "error in listBackupManifests") + } + + m, err := GetManifest(h, uri) + if err != nil { + return nil, err + } + return m.Manifests, nil +} diff --git a/worker/backup_oss.go b/worker/backup_oss.go new file mode 100644 index 00000000000..815632a241a --- /dev/null +++ b/worker/backup_oss.go @@ -0,0 +1,46 @@ +//go:build oss +// +build oss + +/* + * Copyright 2018 Dgraph Labs, Inc. and Contributors + * + * 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 worker + +import ( + "context" + + "github.com/dgraph-io/dgraph/protos/pb" + "github.com/dgraph-io/dgraph/x" + + "github.com/golang/glog" +) + +// Backup implements the Worker interface. +func (w *grpcWorker) Backup(ctx context.Context, req *pb.BackupRequest) (*pb.BackupResponse, error) { + glog.Warningf("Backup failed: %v", x.ErrNotSupported) + return nil, x.ErrNotSupported +} + +func ProcessBackupRequest(ctx context.Context, req *pb.BackupRequest) error { + glog.Warningf("Backup failed: %v", x.ErrNotSupported) + return x.ErrNotSupported +} + +func ProcessListBackups(ctx context.Context, location string, creds *x.MinioCredentials) ( + []*Manifest, error) { + + return nil, x.ErrNotSupported +} diff --git a/worker/backup_processor.go b/worker/backup_processor.go deleted file mode 100644 index 938c716375b..00000000000 --- a/worker/backup_processor.go +++ /dev/null @@ -1,473 +0,0 @@ -// +build !oss - -/* - * Copyright 2022 Dgraph Labs, Inc. and Contributors - * - * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt - */ - -package worker - -import ( - "context" - "encoding/binary" - "encoding/hex" - "fmt" - "io" - "net/url" - "reflect" - "strings" - - "github.com/dgraph-io/badger/v3" - bpb "github.com/dgraph-io/badger/v3/pb" - "github.com/dgraph-io/badger/v3/y" - "github.com/dgraph-io/ristretto/z" - "github.com/dustin/go-humanize" - "github.com/golang/glog" - "github.com/golang/snappy" - "github.com/pkg/errors" - - "github.com/dgraph-io/dgraph/ee/enc" - "github.com/dgraph-io/dgraph/posting" - "github.com/dgraph-io/dgraph/protos/pb" - "github.com/dgraph-io/dgraph/x" -) - -// BackupProcessor handles the different stages of the backup process. -type BackupProcessor struct { - // DB is the Badger pstore managed by this node. - DB *badger.DB - // Request stores the backup request containing the parameters for this backup. - Request *pb.BackupRequest - - // txn is used for the iterators in the threadLocal - txn *badger.Txn - threads []*threadLocal -} - -type threadLocal struct { - Request *pb.BackupRequest - // pre-allocated pb.PostingList object. - pl pb.PostingList - // pre-allocated pb.BackupPostingList object. - bpl pb.BackupPostingList - alloc *z.Allocator - itr *badger.Iterator - buf *z.Buffer -} - -func NewBackupProcessor(db *badger.DB, req *pb.BackupRequest) *BackupProcessor { - bp := &BackupProcessor{ - DB: db, - Request: req, - threads: make([]*threadLocal, x.WorkerConfig.Badger.NumGoroutines), - } - if req.SinceTs > 0 && db != nil { - bp.txn = db.NewTransactionAt(req.ReadTs, false) - } - for i := range bp.threads { - buf := z.NewBuffer(32<<20, "Worker.BackupProcessor"). - WithAutoMmap(1<<30, ""). - WithMaxSize(32 << 30) - - bp.threads[i] = &threadLocal{ - Request: bp.Request, - buf: buf, - } - if bp.txn != nil { - iopt := badger.DefaultIteratorOptions - iopt.AllVersions = true - bp.threads[i].itr = bp.txn.NewIterator(iopt) - } - } - return bp -} - -func (pr *BackupProcessor) Close() { - for _, th := range pr.threads { - if pr.txn != nil { - th.itr.Close() - } - th.buf.Release() - } - if pr.txn != nil { - pr.txn.Discard() - } -} - -// LoadResult holds the output of a Load operation. -type LoadResult struct { - // Version is the timestamp at which the database is after loading a backup. - Version uint64 - // MaxLeaseUid is the max UID seen by the load operation. Needed to request zero - // for the proper number of UIDs. - MaxLeaseUid uint64 - // MaxLeaseNsId is the max namespace ID seen by the load operation. - MaxLeaseNsId uint64 - // The error, if any, of the load operation. - Err error -} - -// WriteBackup uses the request values to create a stream writer then hand off the data -// retrieval to stream.Orchestrate. The writer will create all the fd's needed to -// collect the data and later move to the target. -// Returns errors on failure, nil on success. -func (pr *BackupProcessor) WriteBackup(ctx context.Context) (*pb.BackupResponse, error) { - var response pb.BackupResponse - - if err := ctx.Err(); err != nil { - return nil, err - } - - uri, err := url.Parse(pr.Request.Destination) - if err != nil { - return &response, err - } - - handler, err := NewUriHandler(uri, GetCredentialsFromRequest(pr.Request)) - if err != nil { - return &response, err - } - - if err := handler.CreateBackupFile(uri, pr.Request); err != nil { - return &response, err - } - - glog.V(3).Infof("Backup manifest version: %d", pr.Request.SinceTs) - - predMap := make(map[string]struct{}) - for _, pred := range pr.Request.Predicates { - predMap[pred] = struct{}{} - } - - var maxVersion uint64 - - iwriter, err := enc.GetWriter(x.WorkerConfig.EncryptionKey, handler) - if err != nil { - return &response, errors.Wrap(err, "failed to get encWriter") - } - // Snappy is much faster than gzip compression. In fact, in my experiments, gzip compression - // caused the output speed to be ~30 MBps. Snappy can write at ~90 MBps, and overall the speed - // is similar to writing uncompressed data on disk. - // These are the times I saw: - // Without compression: 7m2s 33GB output. - // With snappy: 7m11s 9.5GB output. - // With snappy + S3: 7m54s 9.5GB output. - cWriter := snappy.NewBufferedWriter(iwriter) - - stream := pr.DB.NewStreamAt(pr.Request.ReadTs) - stream.LogPrefix = "Dgraph.Backup" - // Ignore versions less than given sinceTs timestamp, or skip older versions of - // the given key by returning an empty list. - // Do not do this for schema and type keys. Those keys always have a - // version of one. They're handled separately. - stream.SinceTs = pr.Request.SinceTs - stream.Prefix = []byte{x.ByteData} - - stream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { - tl := pr.threads[itr.ThreadId] - tl.alloc = itr.Alloc - - bitr := itr - // Use the threadlocal iterator because "itr" has the sinceTs set and - // it will not be able to read all the data. - if tl.itr != nil { - bitr = tl.itr - bitr.Seek(key) - } - - kvList, dropOp, err := tl.toBackupList(key, bitr) - if err != nil { - return nil, err - } - // we don't want to append a nil value to the slice, so need to check. - if dropOp != nil { - response.DropOperations = append(response.DropOperations, dropOp) - } - return kvList, nil - } - - stream.ChooseKey = func(item *badger.Item) bool { - parsedKey, err := x.Parse(item.Key()) - if err != nil { - glog.Errorf("error %v while parsing key %v during backup. Skip.", err, hex.EncodeToString(item.Key())) - return false - } - - // Do not choose keys that contain parts of a multi-part list. These keys - // will be accessed from the main list. - if parsedKey.HasStartUid { - return false - } - - // Skip backing up the schema and type keys. They will be backed up separately. - if parsedKey.IsSchema() || parsedKey.IsType() { - return false - } - _, ok := predMap[parsedKey.Attr] - return ok - } - stream.Send = func(buf *z.Buffer) error { - list, err := badger.BufferToKVList(buf) - if err != nil { - return err - } - for _, kv := range list.Kv { - if maxVersion < kv.Version { - maxVersion = kv.Version - } - } - return writeKVList(list, cWriter) - } - - if err := stream.Orchestrate(ctx); err != nil { - glog.Errorf("While taking backup: %v", err) - return &response, err - } - - // This is used to backup the schema and types. - writePrefix := func(prefix byte) error { - tl := threadLocal{ - alloc: z.NewAllocator(1<<10, "BackupProcessor.WritePrefix"), - } - defer tl.alloc.Release() - - txn := pr.DB.NewTransactionAt(pr.Request.ReadTs, false) - defer txn.Discard() - // We don't need to iterate over all versions. - iopts := badger.DefaultIteratorOptions - iopts.Prefix = []byte{prefix} - - itr := txn.NewIterator(iopts) - defer itr.Close() - - list := &bpb.KVList{} - for itr.Rewind(); itr.Valid(); itr.Next() { - item := itr.Item() - // Don't export deleted items. - if item.IsDeletedOrExpired() { - continue - } - parsedKey, err := x.Parse(item.Key()) - if err != nil { - glog.Errorf("error %v while parsing key %v during backup. Skip.", err, hex.EncodeToString(item.Key())) - continue - } - // This check makes sense only for the schema keys. The types are not stored in it. - if _, ok := predMap[parsedKey.Attr]; !parsedKey.IsType() && !ok { - continue - } - kv := y.NewKV(tl.alloc) - if err := item.Value(func(val []byte) error { - kv.Value = append(kv.Value, val...) - return nil - }); err != nil { - return errors.Wrapf(err, "while copying value") - } - - backupKey, err := tl.toBackupKey(item.Key()) - if err != nil { - return err - } - kv.Key = backupKey - kv.UserMeta = tl.alloc.Copy([]byte{item.UserMeta()}) - kv.Version = item.Version() - kv.ExpiresAt = item.ExpiresAt() - list.Kv = append(list.Kv, kv) - } - return writeKVList(list, cWriter) - } - - for _, prefix := range []byte{x.ByteSchema, x.ByteType} { - if err := writePrefix(prefix); err != nil { - glog.Errorf("While writing prefix %d to backup: %v", prefix, err) - return &response, err - } - } - - if maxVersion > pr.Request.ReadTs { - glog.Errorf("Max timestamp seen during backup (%d) is greater than readTs (%d)", - maxVersion, pr.Request.ReadTs) - } - - glog.V(2).Infof("Backup group %d version: %d", pr.Request.GroupId, pr.Request.ReadTs) - if err = cWriter.Close(); err != nil { - glog.Errorf("While closing gzipped writer: %v", err) - return &response, err - } - if err = handler.Close(); err != nil { - glog.Errorf("While closing handler: %v", err) - return &response, err - } - glog.Infof("Backup complete: group %d at %d. Bytes Written: %s\n", - pr.Request.GroupId, pr.Request.ReadTs, - humanize.IBytes(uint64(handler.BytesWritten()))) - return &response, nil -} - -// CompleteBackup will finalize a backup by writing the manifest at the backup destination. -func (pr *BackupProcessor) CompleteBackup(ctx context.Context, m *Manifest) error { - if err := ctx.Err(); err != nil { - return err - } - - uri, err := url.Parse(pr.Request.Destination) - if err != nil { - return err - } - - handler, err := NewUriHandler(uri, GetCredentialsFromRequest(pr.Request)) - if err != nil { - return err - } - - manifest, err := handler.GetManifest(uri) - if err != nil { - return err - } - - manifest.Manifests = append(manifest.Manifests, m) - - if err := handler.CreateManifest(uri, manifest); err != nil { - return errors.Wrap(err, "Complete backup failed") - } - - if err = handler.Close(); err != nil { - return err - } - glog.Infof("Backup completed OK.") - return nil -} - -// GoString implements the GoStringer interface for Manifest. -func (m *Manifest) GoString() string { - return fmt.Sprintf(`Manifest{Since: %d, ReadTs: %d, Groups: %v, Encrypted: %v}`, - m.SinceTsDeprecated, m.ReadTs, m.Groups, m.Encrypted) -} - -func (tl *threadLocal) toBackupList(key []byte, itr *badger.Iterator) ( - *bpb.KVList, *pb.DropOperation, error) { - list := &bpb.KVList{} - var dropOp *pb.DropOperation - - item := itr.Item() - if item.Version() < tl.Request.SinceTs { - return list, nil, - errors.Errorf("toBackupList: Item.Version(): %d should be less than sinceTs: %d", - item.Version(), tl.Request.SinceTs) - } - if item.IsDeletedOrExpired() { - return list, nil, nil - } - - switch item.UserMeta() { - case posting.BitEmptyPosting, posting.BitCompletePosting, posting.BitDeltaPosting: - l, err := posting.ReadPostingList(key, itr) - if err != nil { - return nil, nil, errors.Wrapf(err, "while reading posting list") - } - - // Don't allocate kv on tl.alloc, because we don't need it by the end of this func. - kv, err := l.ToBackupPostingList(&tl.bpl, tl.alloc, tl.buf) - if err != nil { - return nil, nil, errors.Wrapf(err, "while rolling up list") - } - - backupKey, err := tl.toBackupKey(kv.Key) - if err != nil { - return nil, nil, err - } - - // check if this key was storing a DROP operation record. If yes, get the drop operation. - dropOp, err = checkAndGetDropOp(key, l, tl.Request.ReadTs) - if err != nil { - return nil, nil, err - } - - kv.Key = backupKey - list.Kv = append(list.Kv, kv) - default: - return nil, nil, errors.Errorf( - "Unexpected meta: %d for key: %s", item.UserMeta(), hex.Dump(key)) - } - return list, dropOp, nil -} - -func (tl *threadLocal) toBackupKey(key []byte) ([]byte, error) { - parsedKey, err := x.Parse(key) - if err != nil { - return nil, errors.Wrapf(err, "could not parse key %s", hex.Dump(key)) - } - bk := parsedKey.ToBackupKey() - - out := tl.alloc.Allocate(bk.Size()) - n, err := bk.MarshalToSizedBuffer(out) - return out[:n], err -} - -func writeKVList(list *bpb.KVList, w io.Writer) error { - if err := binary.Write(w, binary.LittleEndian, uint64(list.Size())); err != nil { - return err - } - buf, err := list.Marshal() - if err != nil { - return err - } - _, err = w.Write(buf) - return err -} - -func checkAndGetDropOp(key []byte, l *posting.List, readTs uint64) (*pb.DropOperation, error) { - isDropOpKey, err := x.IsDropOpKey(key) - if err != nil || !isDropOpKey { - return nil, err - } - - vals, err := l.AllValues(readTs) - if err != nil { - return nil, errors.Wrapf(err, "cannot read value of dgraph.drop.op") - } - switch len(vals) { - case 0: - // do nothing, it means this one was deleted with S * * deletion. - // So, no need to consider it. - return nil, nil - case 1: - val, ok := vals[0].Value.([]byte) - if !ok { - return nil, errors.Errorf("cannot convert value of dgraph.drop.op to byte array, "+ - "got type: %s, value: %v, tid: %v", reflect.TypeOf(vals[0].Value), vals[0].Value, - vals[0].Tid) - } - // A dgraph.drop.op record can have values in only one of the following formats: - // * DROP_ALL; - // * DROP_DATA; - // * DROP_ATTR;attrName - // So, accordingly construct the *pb.DropOperation. - dropOp := &pb.DropOperation{} - dropInfo := strings.Split(string(val), ";") - if len(dropInfo) != 2 { - return nil, errors.Errorf("Unexpected value: %s for dgraph.drop.op", val) - } - switch dropInfo[0] { - case "DROP_ALL": - dropOp.DropOp = pb.DropOperation_ALL - case "DROP_DATA": - dropOp.DropOp = pb.DropOperation_DATA - case "DROP_ATTR": - dropOp.DropOp = pb.DropOperation_ATTR - dropOp.DropValue = dropInfo[1] - case "DROP_NS": - dropOp.DropOp = pb.DropOperation_NS - dropOp.DropValue = dropInfo[1] // contains namespace. - } - return dropOp, nil - default: - // getting more than one values for a non-list predicate is an error - return nil, errors.Errorf("found multiple values for dgraph.drop.op: %v", vals) - } -} diff --git a/worker/cdc_ee.go b/worker/cdc_ee.go index c19cca4a2f8..9eb3a00dcec 100644 --- a/worker/cdc_ee.go +++ b/worker/cdc_ee.go @@ -1,3 +1,4 @@ +//go:build !oss // +build !oss /* @@ -27,6 +28,7 @@ import ( "github.com/dgraph-io/dgraph/types" "github.com/dgraph-io/dgraph/x" "github.com/dgraph-io/ristretto/z" + "github.com/golang/glog" "github.com/pkg/errors" "go.etcd.io/etcd/raft/raftpb" diff --git a/worker/draft.go b/worker/draft.go index 0804888685f..8f858da1497 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -28,17 +28,6 @@ import ( "sync/atomic" "time" - "github.com/dustin/go-humanize" - "github.com/golang/glog" - "github.com/pkg/errors" - "go.etcd.io/etcd/raft" - "go.etcd.io/etcd/raft/raftpb" - "golang.org/x/net/trace" - - ostats "go.opencensus.io/stats" - "go.opencensus.io/tag" - otrace "go.opencensus.io/trace" - "github.com/dgraph-io/badger/v3" bpb "github.com/dgraph-io/badger/v3/pb" "github.com/dgraph-io/dgraph/conn" @@ -49,6 +38,16 @@ import ( "github.com/dgraph-io/dgraph/types" "github.com/dgraph-io/dgraph/x" "github.com/dgraph-io/ristretto/z" + + "github.com/dustin/go-humanize" + "github.com/golang/glog" + "github.com/pkg/errors" + "go.etcd.io/etcd/raft" + "go.etcd.io/etcd/raft/raftpb" + ostats "go.opencensus.io/stats" + "go.opencensus.io/tag" + otrace "go.opencensus.io/trace" + "golang.org/x/net/trace" ) type operation struct { diff --git a/worker/draft_test.go b/worker/draft_test.go index f52ce9afdc3..c569e95a940 100644 --- a/worker/draft_test.go +++ b/worker/draft_test.go @@ -25,6 +25,7 @@ import ( "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/raftwal" "github.com/dgraph-io/dgraph/x" + "github.com/stretchr/testify/require" "go.etcd.io/etcd/raft/raftpb" ) diff --git a/worker/executor.go b/worker/executor.go index 26865a27dd4..99c82616171 100644 --- a/worker/executor.go +++ b/worker/executor.go @@ -33,6 +33,7 @@ import ( "github.com/dgraph-io/dgraph/types" "github.com/dgraph-io/dgraph/x" "github.com/dgraph-io/ristretto/z" + "github.com/dgryski/go-farm" "github.com/golang/glog" ) diff --git a/worker/export.go b/worker/export.go index d1fed9a94a8..0788f3142b8 100644 --- a/worker/export.go +++ b/worker/export.go @@ -32,22 +32,20 @@ import ( "strings" "time" - "github.com/golang/glog" - "github.com/minio/minio-go/v6" - "github.com/pkg/errors" - "github.com/dgraph-io/badger/v3" bpb "github.com/dgraph-io/badger/v3/pb" - "github.com/dgraph-io/ristretto/z" - "github.com/dgraph-io/dgo/v210/protos/api" - "github.com/dgraph-io/dgraph/ee/enc" "github.com/dgraph-io/dgraph/posting" "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/types" "github.com/dgraph-io/dgraph/types/facets" "github.com/dgraph-io/dgraph/x" + "github.com/dgraph-io/ristretto/z" + + "github.com/golang/glog" + "github.com/minio/minio-go/v6" + "github.com/pkg/errors" ) // DefaultExportFormat stores the name of the default format for exports. @@ -368,7 +366,7 @@ func fieldToString(update *pb.SchemaUpdate) string { return builder.String() } -type fileWriter struct { +type ExportWriter struct { fd *os.File bw *bufio.Writer gw *gzip.Writer @@ -376,7 +374,7 @@ type fileWriter struct { hasDataBefore bool } -func (writer *fileWriter) open(fpath string) error { +func (writer *ExportWriter) open(fpath string) error { var err error writer.fd, err = os.Create(fpath) if err != nil { @@ -391,7 +389,7 @@ func (writer *fileWriter) open(fpath string) error { return err } -func (writer *fileWriter) Close() error { +func (writer *ExportWriter) Close() error { if err := writer.gw.Flush(); err != nil { return err } @@ -410,9 +408,9 @@ func (writer *fileWriter) Close() error { // ExportedFiles has the relative path of files that were written during export type ExportedFiles []string -type exportStorage interface { - openFile(relativePath string) (*fileWriter, error) - finishWriting(fs ...*fileWriter) (ExportedFiles, error) +type ExportStorage interface { + OpenFile(relativePath string) (*ExportWriter, error) + FinishWriting(w *Writers) (ExportedFiles, error) } type localExportStorage struct { @@ -441,8 +439,8 @@ func newLocalExportStorage(destination, backupName string) (*localExportStorage, return &localExportStorage{destination, backupName}, nil } -func (l *localExportStorage) openFile(fileName string) (*fileWriter, error) { - fw := &fileWriter{relativePath: filepath.Join(l.relativePath, fileName)} +func (l *localExportStorage) OpenFile(fileName string) (*ExportWriter, error) { + fw := &ExportWriter{relativePath: filepath.Join(l.relativePath, fileName)} filePath, err := filepath.Abs(filepath.Join(l.destination, fw.relativePath)) if err != nil { @@ -458,17 +456,21 @@ func (l *localExportStorage) openFile(fileName string) (*fileWriter, error) { return fw, nil } -func (l *localExportStorage) finishWriting(fs ...*fileWriter) (ExportedFiles, error) { - var files ExportedFiles - - for _, file := range fs { - err := file.Close() - if err != nil { - return nil, err - } - files = append(files, file.relativePath) +func (l *localExportStorage) FinishWriting(w *Writers) (ExportedFiles, error) { + if err := w.DataWriter.Close(); err != nil { + return nil, err + } + if err := w.SchemaWriter.Close(); err != nil { + return nil, err + } + if err := w.GqlSchemaWriter.Close(); err != nil { + return nil, err + } + files := ExportedFiles{ + w.DataWriter.relativePath, + w.SchemaWriter.relativePath, + w.GqlSchemaWriter.relativePath, } - return files, nil } @@ -505,12 +507,12 @@ func newRemoteExportStorage(in *pb.ExportRequest, backupName string) (*remoteExp return &remoteExportStorage{mc, bucket, prefix, localStorage}, nil } -func (r *remoteExportStorage) openFile(fileName string) (*fileWriter, error) { - return r.les.openFile(fileName) +func (r *remoteExportStorage) OpenFile(fileName string) (*ExportWriter, error) { + return r.les.OpenFile(fileName) } -func (r *remoteExportStorage) finishWriting(fs ...*fileWriter) (ExportedFiles, error) { - files, err := r.les.finishWriting(fs...) +func (r *remoteExportStorage) FinishWriting(w *Writers) (ExportedFiles, error) { + files, err := r.les.FinishWriting(w) if err != nil { return nil, err } @@ -536,7 +538,7 @@ func (r *remoteExportStorage) finishWriting(fs ...*fileWriter) (ExportedFiles, e return files, nil } -func newExportStorage(in *pb.ExportRequest, backupName string) (exportStorage, error) { +func NewExportStorage(in *pb.ExportRequest, backupName string) (ExportStorage, error) { switch { case strings.HasPrefix(in.Destination, "/"): return newLocalExportStorage(in.Destination, backupName) @@ -565,6 +567,170 @@ func export(ctx context.Context, in *pb.ExportRequest) (ExportedFiles, error) { return exportInternal(ctx, in, pstore, false) } +func ToExportKvList(pk x.ParsedKey, pl *posting.List, in *pb.ExportRequest) (*bpb.KVList, error) { + e := &exporter{ + readTs: in.ReadTs, + uid: pk.Uid, + namespace: x.ParseNamespace(pk.Attr), + attr: x.ParseAttr(pk.Attr), + pl: pl, + } + + emptyList := &bpb.KVList{} + switch { + // These predicates are not required in the export data. + case e.attr == "dgraph.graphql.xid": + case e.attr == "dgraph.drop.op": + case e.attr == "dgraph.graphql.p_query": + + case pk.IsData() && e.attr == "dgraph.graphql.schema": + // Export the graphql schema. + vals, err := pl.AllValues(in.ReadTs) + if err != nil { + return emptyList, errors.Wrapf(err, "cannot read value of GraphQL schema") + } + // if the GraphQL schema node was deleted with S * * delete mutation, + // then the data key will be overwritten with nil value. + // So, just skip exporting it as there will be no value for this data key. + if len(vals) == 0 { + return emptyList, nil + } + // Give an error only if we find more than one value for the schema. + if len(vals) > 1 { + return emptyList, errors.Errorf("found multiple values for the GraphQL schema") + } + val, ok := vals[0].Value.([]byte) + if !ok { + return emptyList, errors.Errorf("cannot convert value of GraphQL schema to byte array") + } + + exported := x.ExportedGQLSchema{ + Namespace: e.namespace, + Schema: string(val), + } + if val, err = json.Marshal(exported); err != nil { + return emptyList, errors.Wrapf(err, "Error marshalling GraphQL schema to json") + } + kv := &bpb.KV{ + Value: val, + Version: 2, // GraphQL schema value + } + return listWrap(kv), nil + + // below predicates no longer exist internally starting v21.03 but leaving them here + // so that users with a binary with version >= 21.03 can export data from a version < 21.03 + // without this internal data showing up. + case e.attr == "dgraph.cors": + case e.attr == "dgraph.graphql.schema_created_at": + case e.attr == "dgraph.graphql.schema_history": + case e.attr == "dgraph.graphql.p_sha256hash": + + case pk.IsData(): + // The GraphQL layer will create a node of type "dgraph.graphql". That entry + // should not be exported. + if e.attr == "dgraph.type" { + vals, err := e.pl.AllValues(in.ReadTs) + if err != nil { + return emptyList, errors.Wrapf(err, "cannot read value of dgraph.type entry") + } + if len(vals) == 1 { + val, ok := vals[0].Value.([]byte) + if !ok { + return emptyList, errors.Errorf("cannot read value of dgraph.type entry") + } + if string(val) == "dgraph.graphql" { + return emptyList, nil + } + } + } + + switch in.Format { + case "json": + return e.toJSON() + case "rdf": + return e.toRDF() + default: + glog.Fatalf("Invalid export format found: %s", in.Format) + } + + default: + glog.Fatalf("Invalid key found: %+v %v\n", pk, hex.Dump([]byte(pk.Attr))) + } + return emptyList, nil +} + +func WriteExport(writers *Writers, kv *bpb.KV, format string) error { + // Skip nodes that have no data. Otherwise, the exported data could have + // formatting and/or syntax errors. + if len(kv.Value) == 0 { + return nil + } + + var dataSeparator []byte + switch format { + case "json": + dataSeparator = []byte(",\n") + case "rdf": + // The separator for RDF should be empty since the toRDF function already + // adds newline to each RDF entry. + default: + glog.Fatalf("Invalid export format found: %s", format) + } + + var writer *ExportWriter + var sep []byte + switch kv.Version { + case 1: // data + writer = writers.DataWriter + sep = dataSeparator + case 2: // graphQL schema + writer = writers.GqlSchemaWriter + sep = []byte(",\n") // use json separator. + case 3: // graphQL schema + writer = writers.SchemaWriter + default: + glog.Fatalf("Invalid data type found: %x", kv.Key) + } + + if writer.hasDataBefore { + if _, err := writer.gw.Write(sep); err != nil { + return err + } + } + // change the hasDataBefore flag so that the next data entry will have a separator + // prepended + writer.hasDataBefore = true + + _, err := writer.gw.Write(kv.Value) + return err +} + +type Writers struct { + DataWriter *ExportWriter + SchemaWriter *ExportWriter + GqlSchemaWriter *ExportWriter +} + +func InitWriters(s ExportStorage, in *pb.ExportRequest) (*Writers, error) { + xfmt := exportFormats[in.Format] + w := &Writers{} + fileName := func(ext string) string { + return fmt.Sprintf("g%02d%s", in.GroupId, ext) + } + + var err error + if w.DataWriter, err = s.OpenFile(fileName(xfmt.ext + ".gz")); err != nil { + return w, err + } + if w.SchemaWriter, err = s.OpenFile(fileName(".schema.gz")); err != nil { + return w, err + } + if w.GqlSchemaWriter, err = s.OpenFile(fileName(".gql_schema.gz")); err != nil { + return w, err + } + return w, nil +} + // exportInternal contains the core logic to export a Dgraph database. If skipZero is set to // false, the parts of this method that require to talk to zero will be skipped. This is useful // when exporting a p directory directly from disk without a running cluster. @@ -572,31 +738,17 @@ func export(ctx context.Context, in *pb.ExportRequest) (ExportedFiles, error) { // and types. func exportInternal(ctx context.Context, in *pb.ExportRequest, db *badger.DB, skipZero bool) (ExportedFiles, error) { + uts := time.Unix(in.UnixTs, 0) - exportStorage, err := newExportStorage(in, + exportStorage, err := NewExportStorage(in, fmt.Sprintf("dgraph.r%d.u%s", in.ReadTs, uts.UTC().Format("0102.1504"))) if err != nil { return nil, err } - - xfmt := exportFormats[in.Format] - - dataWriter, err := exportStorage.openFile(fmt.Sprintf("g%02d%s", in.GroupId, xfmt.ext+".gz")) - if err != nil { - return nil, err - } - - schemaWriter, err := exportStorage.openFile(fmt.Sprintf("g%02d%s", in.GroupId, ".schema.gz")) - if err != nil { - return nil, err - } - - gqlSchemaWriter, err := exportStorage.openFile( - fmt.Sprintf("g%02d%s", in.GroupId, ".gql_schema.gz")) + writers, err := InitWriters(exportStorage, in) if err != nil { - return nil, err + return nil, errors.Wrap(err, "exportInternal failed") } - // This stream exports only the data and the graphQL schema. stream := db.NewStreamAt(in.ReadTs) stream.Prefix = []byte{x.DefaultPrefix} @@ -636,6 +788,7 @@ func exportInternal(ctx context.Context, in *pb.ExportRequest, db *badger.DB, } return pk.IsData() } + stream.KeyToList = func(key []byte, itr *badger.Iterator) (*bpb.KVList, error) { item := itr.Item() pk, err := x.Parse(item.Key()) @@ -644,115 +797,11 @@ func exportInternal(ctx context.Context, in *pb.ExportRequest, db *badger.DB, hex.EncodeToString(item.Key())) return nil, err } - e := &exporter{ - readTs: in.ReadTs, - } - e.uid = pk.Uid - e.namespace, e.attr = x.ParseNamespaceAttr(pk.Attr) - - switch { - case e.attr == "dgraph.graphql.xid": - // Ignore this predicate. - case e.attr == "dgraph.drop.op": - // Ignore this predicate. - case e.attr == "dgraph.graphql.p_query": - // Ignore this predicate. - case pk.IsData() && e.attr == "dgraph.graphql.schema": - // Export the graphql schema. - pl, err := posting.ReadPostingList(key, itr) - if err != nil { - return nil, errors.Wrapf(err, "cannot read posting list for GraphQL schema") - } - vals, err := pl.AllValues(in.ReadTs) - if err != nil { - return nil, errors.Wrapf(err, "cannot read value of GraphQL schema") - } - // if the GraphQL schema node was deleted with S * * delete mutation, - // then the data key will be overwritten with nil value. - // So, just skip exporting it as there will be no value for this data key. - if len(vals) == 0 { - return nil, nil - } - // Give an error only if we find more than one value for the schema. - if len(vals) > 1 { - return nil, errors.Errorf("found multiple values for the GraphQL schema") - } - val, ok := vals[0].Value.([]byte) - if !ok { - return nil, errors.Errorf("cannot convert value of GraphQL schema to byte array") - } - - exported := x.ExportedGQLSchema{ - Namespace: e.namespace, - Schema: string(val), - } - if val, err = json.Marshal(exported); err != nil { - return nil, errors.Wrapf(err, "Error marshalling GraphQL schema to json") - } - - kv := &bpb.KV{ - Value: val, - Version: 2, // GraphQL schema value - } - return listWrap(kv), nil - - // below predicates no longer exist internally starting v21.03 but leaving them here - // so that users with a binary with version >= 21.03 can export data from a version < 21.03 - // without this internal data showing up. - case e.attr == "dgraph.cors": - case e.attr == "dgraph.graphql.schema_created_at": - case e.attr == "dgraph.graphql.schema_history": - case e.attr == "dgraph.graphql.p_sha256hash": - // Ignore these predicates. - - case pk.IsData(): - e.pl, err = posting.ReadPostingList(key, itr) - if err != nil { - return nil, err - } - - // The GraphQL layer will create a node of type "dgraph.graphql". That entry - // should not be exported. - if e.attr == "dgraph.type" { - vals, err := e.pl.AllValues(in.ReadTs) - if err != nil { - return nil, errors.Wrapf(err, "cannot read value of dgraph.type entry") - } - if len(vals) == 1 { - val, ok := vals[0].Value.([]byte) - if !ok { - return nil, errors.Errorf("cannot read value of dgraph.type entry") - } - if string(val) == "dgraph.graphql" { - return nil, nil - } - } - } - - switch in.Format { - case "json": - return e.toJSON() - case "rdf": - return e.toRDF() - default: - glog.Fatalf("Invalid export format found: %s", in.Format) - } - - default: - glog.Fatalf("Invalid key found: %+v\n", pk) + pl, err := posting.ReadPostingList(key, itr) + if err != nil { + return nil, errors.Wrapf(err, "cannot read posting list") } - return nil, nil - } - - var dataSeparator []byte - switch in.Format { - case "json": - dataSeparator = []byte(",\n") - case "rdf": - // The separator for RDF should be empty since the toRDF function already - // adds newline to each RDF entry. - default: - glog.Fatalf("Invalid export format found: %s", in.Format) + return ToExportKvList(pk, pl, in) } stream.Send = func(buf *z.Buffer) error { @@ -762,36 +811,7 @@ func exportInternal(ctx context.Context, in *pb.ExportRequest, db *badger.DB, if err := kv.Unmarshal(s); err != nil { return err } - // Skip nodes that have no data. Otherwise, the exported data could have - // formatting and/or syntax errors. - if len(kv.Value) == 0 { - return nil - } - - var writer *fileWriter - var separator []byte - switch kv.Version { - case 1: // data - writer = dataWriter - separator = dataSeparator - case 2: // graphQL schema - writer = gqlSchemaWriter - separator = []byte(",\n") // use json separator. - default: - glog.Fatalf("Invalid data type found: %x", kv.Key) - } - - if writer.hasDataBefore { - if _, err := writer.gw.Write(separator); err != nil { - return err - } - } - // change the hasDataBefore flag so that the next data entry will have a separator - // prepended - writer.hasDataBefore = true - - _, err = writer.gw.Write(kv.Value) - return err + return WriteExport(writers, kv, in.Format) }) } @@ -821,65 +841,53 @@ func exportInternal(ctx context.Context, in *pb.ExportRequest, db *badger.DB, return err } + val, err := item.ValueCopy(nil) + if err != nil { + return errors.Wrap(err, "writePrefix failed to get value") + } var kv *bpb.KV switch prefix { case x.ByteSchema: - if !skipZero { - servesTablet, err := groups().ServesTablet(pk.Attr) - if err != nil || !servesTablet { - continue - } - } - - var update pb.SchemaUpdate - err = item.Value(func(val []byte) error { - return update.Unmarshal(val) - }) + kv, err = SchemaExportKv(pk.Attr, val, skipZero) if err != nil { // Let's not propagate this error. We just log this and continue onwards. - glog.Errorf("Unable to unmarshal schema: %+v. Err=%v\n", pk, err) + glog.Errorf("Unable to export schema: %+v. Err=%v\n", pk, err) continue } - kv = toSchema(pk.Attr, &update) - case x.ByteType: - var update pb.TypeUpdate - err := item.Value(func(val []byte) error { - return update.Unmarshal(val) - }) + kv, err = TypeExportKv(pk.Attr, val) if err != nil { // Let's not propagate this error. We just log this and continue onwards. - glog.Errorf("Unable to unmarshal type: %+v. Err=%v\n", pk, err) - return nil + glog.Errorf("Unable to export type: %+v. Err=%v\n", pk, err) + continue } - kv = toType(pk.Attr, update) - default: glog.Fatalf("Unhandled byte prefix: %v", prefix) } // Write to the appropriate writer. - if _, err := schemaWriter.gw.Write(kv.Value); err != nil { + if _, err := writers.SchemaWriter.gw.Write(kv.Value); err != nil { return err } } return nil } + xfmt := exportFormats[in.Format] // All prepwork done. Time to roll. - if _, err = gqlSchemaWriter.gw.Write([]byte(exportFormats["json"].pre)); err != nil { + if _, err = writers.GqlSchemaWriter.gw.Write([]byte(exportFormats["json"].pre)); err != nil { return nil, err } - if _, err = dataWriter.gw.Write([]byte(xfmt.pre)); err != nil { + if _, err = writers.DataWriter.gw.Write([]byte(xfmt.pre)); err != nil { return nil, err } if err := stream.Orchestrate(ctx); err != nil { return nil, err } - if _, err = dataWriter.gw.Write([]byte(xfmt.post)); err != nil { + if _, err = writers.DataWriter.gw.Write([]byte(xfmt.post)); err != nil { return nil, err } - if _, err = gqlSchemaWriter.gw.Write([]byte(exportFormats["json"].post)); err != nil { + if _, err = writers.GqlSchemaWriter.gw.Write([]byte(exportFormats["json"].post)); err != nil { return nil, err } @@ -892,7 +900,30 @@ func exportInternal(ctx context.Context, in *pb.ExportRequest, db *badger.DB, } glog.Infof("Export DONE for group %d at timestamp %d.", in.GroupId, in.ReadTs) - return exportStorage.finishWriting(dataWriter, schemaWriter, gqlSchemaWriter) + return exportStorage.FinishWriting(writers) +} + +func SchemaExportKv(attr string, val []byte, skipZero bool) (*bpb.KV, error) { + if !skipZero { + servesTablet, err := groups().ServesTablet(attr) + if err != nil || !servesTablet { + return nil, errors.Errorf("Tablet not found for attribute: %v", err) + } + } + + var update pb.SchemaUpdate + if err := update.Unmarshal(val); err != nil { + return nil, err + } + return toSchema(attr, &update), nil +} + +func TypeExportKv(attr string, val []byte) (*bpb.KV, error) { + var update pb.TypeUpdate + if err := update.Unmarshal(val); err != nil { + return nil, err + } + return toType(attr, update), nil } // Export request is used to trigger exports for the request list of groups. diff --git a/worker/export_test.go b/worker/export_test.go index e7414df3e4f..a5af3fd8841 100644 --- a/worker/export_test.go +++ b/worker/export_test.go @@ -32,11 +32,7 @@ import ( "testing" "time" - "github.com/golang/protobuf/proto" - "github.com/stretchr/testify/require" - "github.com/dgraph-io/dgo/v210/protos/api" - "github.com/dgraph-io/dgraph/chunker" "github.com/dgraph-io/dgraph/dql" "github.com/dgraph-io/dgraph/lex" @@ -47,6 +43,9 @@ import ( "github.com/dgraph-io/dgraph/types" "github.com/dgraph-io/dgraph/types/facets" "github.com/dgraph-io/dgraph/x" + + "github.com/golang/protobuf/proto" + "github.com/stretchr/testify/require" ) const ( diff --git a/worker/file_handler.go b/worker/file_handler.go deleted file mode 100644 index 5817a9f1aee..00000000000 --- a/worker/file_handler.go +++ /dev/null @@ -1,305 +0,0 @@ -// +build !oss - -/* - * Copyright 2022 Dgraph Labs, Inc. and Contributors - * - * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt - */ - -package worker - -import ( - "encoding/json" - "fmt" - "io/ioutil" - "net/url" - "os" - "path/filepath" - "sort" - "strings" - - "github.com/dgraph-io/dgraph/protos/pb" - "github.com/dgraph-io/dgraph/x" - - "github.com/golang/glog" - "github.com/pkg/errors" -) - -// fileHandler is used for 'file:' URI scheme. -type fileHandler struct { - fp *os.File - numWritten int -} - -// readManifest reads a manifest file at path using the handler. -// Returns nil on success, otherwise an error. -func (h *fileHandler) readManifest(path string, m *Manifest) error { - b, err := ioutil.ReadFile(path) - if err != nil { - return err - } - return json.Unmarshal(b, m) -} - -// readMasterManifest reads the master manifest file at path using the handler. -// Returns nil on success, otherwise an error. -func (h *fileHandler) readMasterManifest(path string, m *MasterManifest) error { - b, err := ioutil.ReadFile(path) - if err != nil { - return err - } - return json.Unmarshal(b, m) -} - -func (h *fileHandler) createFiles(uri *url.URL, req *pb.BackupRequest, fileName string) error { - var dir, path string - - dir = filepath.Join(uri.Path, fmt.Sprintf(backupPathFmt, req.UnixTs)) - err := os.Mkdir(dir, 0700) - if err != nil && !os.IsExist(err) { - return err - } - - path = filepath.Join(dir, fileName) - h.fp, err = os.Create(path) - if err != nil { - return err - } - glog.V(2).Infof("Using file path: %q", path) - return nil -} - -// GetLatestManifest reads the manifests at the given URL and returns the -// latest manifest. -func (h *fileHandler) GetLatestManifest(uri *url.URL) (*Manifest, error) { - if err := createIfNotExists(uri.Path); err != nil { - return nil, errors.Wrap(err, "Get latest manifest failed:") - } - - manifest, err := h.getConsolidatedManifest(uri) - if err != nil { - return nil, errors.Wrap(err, "Get latest manifest failed while consolidation: ") - } - if len(manifest.Manifests) == 0 { - return &Manifest{}, nil - } - return manifest.Manifests[len(manifest.Manifests)-1], nil -} - -func createIfNotExists(path string) error { - if pathExist(path) { - return nil - } - if err := os.MkdirAll(path, 0755); err != nil { - return errors.Errorf("The path %q does not exist or it is inaccessible."+ - " While trying to create it, got error: %v", path, err) - } - return nil -} - -// CreateBackupFile prepares the a path to save the backup file. -func (h *fileHandler) CreateBackupFile(uri *url.URL, req *pb.BackupRequest) error { - if err := createIfNotExists(uri.Path); err != nil { - return errors.Errorf("while CreateBackupFile: %v", err) - } - - fileName := backupName(req.ReadTs, req.GroupId) - return h.createFiles(uri, req, fileName) -} - -// CreateManifest completes the backup by writing the manifest to a file. -func (h *fileHandler) CreateManifest(uri *url.URL, manifest *MasterManifest) error { - var err error - if err = createIfNotExists(uri.Path); err != nil { - return errors.Errorf("while WriteManifest: %v", err) - } - - tmpPath := filepath.Join(uri.Path, tmpManifest) - if h.fp, err = os.Create(tmpPath); err != nil { - return err - } - if err = json.NewEncoder(h).Encode(manifest); err != nil { - return err - } - - // Move the tmpManifest to backupManifest - path := filepath.Join(uri.Path, backupManifest) - return os.Rename(tmpPath, path) -} - -// GetManifest returns the master manifest, if the directory doesn't contain -// a master manifest, then it will try to return a master manifest by consolidating -// the manifests. -func (h *fileHandler) GetManifest(uri *url.URL) (*MasterManifest, error) { - if err := createIfNotExists(uri.Path); err != nil { - return nil, errors.Errorf("while GetLatestManifest: %v", err) - } - manifest, err := h.getConsolidatedManifest(uri) - if err != nil { - return manifest, errors.Wrap(err, "GetManifest failed to get consolidated manifest: ") - } - return manifest, nil -} - -func (h *fileHandler) GetManifests(uri *url.URL, backupId string, - backupNum uint64) ([]*Manifest, error) { - if err := createIfNotExists(uri.Path); err != nil { - return nil, errors.Errorf("while GetManifests: %v", err) - } - - manifest, err := h.getConsolidatedManifest(uri) - if err != nil { - return manifest.Manifests, errors.Wrap(err, "GetManifests failed to get consolidated manifest: ") - } - - var filtered []*Manifest - for _, m := range manifest.Manifests { - path := filepath.Join(uri.Path, m.Path) - if pathExist(path) { - filtered = append(filtered, m) - } - } - - return getManifests(filtered, backupId, backupNum) -} - -// Load uses tries to load any backup files found. -// Returns the maximum value of Since on success, error otherwise. -func (h *fileHandler) Load(uri *url.URL, backupId string, backupNum uint64, fn loadFn) LoadResult { - manifests, err := h.GetManifests(uri, backupId, backupNum) - if err != nil { - return LoadResult{Err: errors.Wrapf(err, "cannot retrieve manifests")} - } - - // Process each manifest, first check that they are valid and then confirm the - // backup files for each group exist. Each group in manifest must have a backup file, - // otherwise this is a failure and the user must remedy. - var since uint64 - var maxUid, maxNsId uint64 - for i, manifest := range manifests { - if manifest.ValidReadTs() == 0 || len(manifest.Groups) == 0 { - continue - } - - path := filepath.Join(uri.Path, manifests[i].Path) - for gid := range manifest.Groups { - file := filepath.Join(path, backupName(manifest.ValidReadTs(), gid)) - fp, err := os.Open(file) - if err != nil { - return LoadResult{Err: errors.Wrapf(err, "Failed to open %q", file)} - } - defer fp.Close() - - // Only restore the predicates that were assigned to this group at the time - // of the last backup. - predSet := manifests[len(manifests)-1].getPredsInGroup(gid) - - groupMaxUid, groupMaxNsId, err := fn(gid, - &loadBackupInput{ - r: fp, - preds: predSet, - dropOperations: manifest.DropOperations, - isOld: manifest.Version == 0, - compression: manifest.Compression, - }) - if err != nil { - return LoadResult{Err: err} - } - maxUid = x.Max(maxUid, groupMaxUid) - maxNsId = x.Max(maxNsId, groupMaxNsId) - } - since = manifest.ValidReadTs() - } - - return LoadResult{Version: since, MaxLeaseUid: maxUid, MaxLeaseNsId: maxNsId} -} - -// Verify performs basic checks to decide whether the specified backup can be restored -// to a live cluster. -func (h *fileHandler) Verify(uri *url.URL, req *pb.RestoreRequest, currentGroups []uint32) error { - manifests, err := h.GetManifests(uri, req.GetBackupId(), req.GetBackupNum()) - if err != nil { - return errors.Wrapf(err, "while retrieving manifests") - } - return verifyRequest(req, manifests, currentGroups) -} - -func (h *fileHandler) Close() error { - if h.fp == nil { - return nil - } - if err := h.fp.Sync(); err != nil { - glog.Errorf("While closing file: %s. Error: %v", h.fp.Name(), err) - x.Ignore(h.fp.Close()) - return err - } - return h.fp.Close() -} - -func (h *fileHandler) Write(b []byte) (int, error) { - n, err := h.fp.Write(b) - h.numWritten += n - return n, err -} - -func (h *fileHandler) BytesWritten() int { - return h.numWritten -} - -// pathExist checks if a path (file or dir) is found at target. -// Returns true if found, false otherwise. -func pathExist(path string) bool { - _, err := os.Stat(path) - if err == nil { - return true - } - return !os.IsNotExist(err) && !os.IsPermission(err) -} - -// getConsolidatedManifest walks over all the backup directories and generates a master manifest. -func (h *fileHandler) getConsolidatedManifest(uri *url.URL) (*MasterManifest, error) { - if err := createIfNotExists(uri.Path); err != nil { - return nil, errors.Wrap(err, "While GetLatestManifest") - } - - var manifest MasterManifest - - // If there is a master manifest already, we just return it. - path := filepath.Join(uri.Path, backupManifest) - if pathExist(path) { - if err := h.readMasterManifest(path, &manifest); err != nil { - return nil, errors.Wrap(err, "Get latest manifest failed to read master manifest: ") - } - return &manifest, nil - } - - // Otherwise, we create a master manifest by going through all the backup directories. - var paths []string - suffix := filepath.Join(string(filepath.Separator), backupManifest) - _ = x.WalkPathFunc(uri.Path, func(path string, isdir bool) bool { - if !isdir && strings.HasSuffix(path, suffix) { - paths = append(paths, path) - } - return false - }) - - sort.Strings(paths) - var mlist []*Manifest - - for _, path := range paths { - var m Manifest - if err := h.readManifest(path, &m); err != nil { - return nil, errors.Wrap(err, "While Getting latest manifest") - } - path = filepath.Dir(path) - _, path = filepath.Split(path) - m.Path = path - mlist = append(mlist, &m) - } - manifest.Manifests = mlist - return &manifest, nil -} diff --git a/worker/graphql_schema.go b/worker/graphql_schema.go index 4c7fb79ed03..26439ede976 100644 --- a/worker/graphql_schema.go +++ b/worker/graphql_schema.go @@ -22,15 +22,14 @@ import ( "sync" "time" - "google.golang.org/grpc/metadata" - - "github.com/golang/glog" - "github.com/dgraph-io/dgraph/conn" "github.com/dgraph-io/dgraph/protos/pb" "github.com/dgraph-io/dgraph/schema" "github.com/dgraph-io/dgraph/x" + + "github.com/golang/glog" "github.com/pkg/errors" + "google.golang.org/grpc/metadata" ) const ( @@ -48,8 +47,55 @@ var ( errUpdatingGraphQLSchemaOnNonGroupOneLeader = errors.New( "while updating GraphQL schema: this server isn't group-1 leader, please retry") ErrMultipleGraphQLSchemaNodes = errors.New("found multiple nodes for GraphQL schema") + gqlSchemaStore *GQLSchemaStore ) +type GqlSchema struct { + ID string `json:"id,omitempty"` + Schema string `json:"schema,omitempty"` + Version uint64 + GeneratedSchema string + Loaded bool // This indicate whether the schema has been loaded into graphql server + // or not +} + +type GQLSchemaStore struct { + mux sync.RWMutex + schema map[uint64]*GqlSchema +} + +func NewGQLSchemaStore() *GQLSchemaStore { + gqlSchemaStore = &GQLSchemaStore{ + mux: sync.RWMutex{}, + schema: make(map[uint64]*GqlSchema), + } + return gqlSchemaStore +} + +func (gs *GQLSchemaStore) Set(ns uint64, sch *GqlSchema) { + gs.mux.Lock() + defer gs.mux.Unlock() + gs.schema[ns] = sch +} + +func (gs *GQLSchemaStore) GetCurrent(ns uint64) (*GqlSchema, bool) { + gs.mux.RLock() + defer gs.mux.RUnlock() + sch, ok := gs.schema[ns] + return sch, ok +} + +func (gs *GQLSchemaStore) resetGQLSchema() { + gs.mux.Lock() + defer gs.mux.Unlock() + + gs.schema = make(map[uint64]*GqlSchema) +} + +func ResetGQLSchemaStore() { + gqlSchemaStore.resetGQLSchema() +} + // UpdateGQLSchemaOverNetwork sends the request to the group one leader for execution. func UpdateGQLSchemaOverNetwork(ctx context.Context, req *pb.UpdateGraphQLSchemaRequest) (*pb. UpdateGraphQLSchemaResponse, error) { diff --git a/worker/groups.go b/worker/groups.go index f5cad33bb61..378029bfec8 100644 --- a/worker/groups.go +++ b/worker/groups.go @@ -34,6 +34,7 @@ import ( "github.com/dgraph-io/dgraph/schema" "github.com/dgraph-io/dgraph/x" "github.com/dgraph-io/ristretto/z" + "github.com/golang/glog" "github.com/golang/protobuf/proto" "github.com/pkg/errors" diff --git a/worker/online_restore.go b/worker/online_restore.go index a36e5231af4..63e44b351e6 100644 --- a/worker/online_restore.go +++ b/worker/online_restore.go @@ -1,43 +1,523 @@ -// +build oss +//go:build !oss +// +build !oss /* * Copyright 2022 Dgraph Labs, Inc. and Contributors * - * 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 + * Licensed under the Dgraph 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 * - * 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. + * https://github.com/dgraph-io/dgraph/blob/master/licenses/DCL.txt */ package worker import ( "context" + "fmt" + "io/ioutil" + "math" + "net/url" + "os" + "path/filepath" + "strings" "sync" + "time" + "github.com/dgraph-io/badger/v3" + "github.com/dgraph-io/badger/v3/options" + "github.com/dgraph-io/dgraph/conn" + "github.com/dgraph-io/dgraph/ee" + "github.com/dgraph-io/dgraph/posting" "github.com/dgraph-io/dgraph/protos/pb" + "github.com/dgraph-io/dgraph/schema" "github.com/dgraph-io/dgraph/x" + "github.com/golang/glog" + "github.com/golang/protobuf/proto" + "github.com/pkg/errors" + "github.com/spf13/pflag" + "github.com/spf13/viper" ) +const ( + errRestoreProposal = "cannot propose restore request" +) + +// verifyRequest verifies that the manifest satisfies the requirements to process the given +// restore request. +func verifyRequest(h UriHandler, uri *url.URL, req *pb.RestoreRequest, + currentGroups []uint32) error { + + manifests, err := getManifestsToRestore(h, uri, req) + if err != nil { + return errors.Wrapf(err, "while retrieving manifests") + } + if len(manifests) == 0 { + return errors.Errorf("No backups with the specified backup ID %s", req.GetBackupId()) + } + + // TODO(Ahsan): Do we need to verify the manifests again here? + if err := verifyManifests(manifests); err != nil { + return err + } + + lastManifest := manifests[0] + if len(currentGroups) != len(lastManifest.Groups) { + return errors.Errorf("groups in cluster and latest backup manifest differ") + } + + for _, group := range currentGroups { + if _, ok := lastManifest.Groups[group]; !ok { + return errors.Errorf("groups in cluster and latest backup manifest differ") + } + } + return nil +} + +// VerifyBackup will access the backup location and verify that the specified backup can +// be restored to the cluster. +func VerifyBackup(req *pb.RestoreRequest, creds *x.MinioCredentials, currentGroups []uint32) error { + uri, err := url.Parse(req.GetLocation()) + if err != nil { + return err + } + + h, err := NewUriHandler(uri, creds) + if err != nil { + return errors.Wrap(err, "VerifyBackup") + } + + return verifyRequest(h, uri, req, currentGroups) +} + +// ProcessRestoreRequest verifies the backup data and sends a restore proposal to each group. func ProcessRestoreRequest(ctx context.Context, req *pb.RestoreRequest, wg *sync.WaitGroup) error { - glog.Warningf("Restore failed: %v", x.ErrNotSupported) - return x.ErrNotSupported + if req == nil { + return errors.Errorf("restore request cannot be nil") + } + + if err := UpdateMembershipState(ctx); err != nil { + return errors.Wrapf(err, "cannot update membership state before restore") + } + memState := GetMembershipState() + + currentGroups := make([]uint32, 0) + for gid := range memState.GetGroups() { + currentGroups = append(currentGroups, gid) + } + + creds := x.MinioCredentials{ + AccessKey: req.AccessKey, + SecretKey: req.SecretKey, + SessionToken: req.SessionToken, + Anonymous: req.Anonymous, + } + if err := VerifyBackup(req, &creds, currentGroups); err != nil { + return errors.Wrapf(err, "failed to verify backup") + } + if err := FillRestoreCredentials(req.Location, req); err != nil { + return errors.Wrapf(err, "cannot fill restore proposal with the right credentials") + } + + // This check if any restore operation running on the node. + // Operation initiated on other nodes doesn't have record in the record tracker. + // This keeps track if there is an already running restore operation return the error. + // IMP: This introduces few corner cases. + // Like two concurrent restore operation on different nodes. + // Considering Restore as admin operation, solving all those complexities has low gains + // than to sacrifice the simplicity. + isRestoreRunning := func() bool { + tasks := GetOngoingTasks() + for _, t := range tasks { + if t == opRestore.String() { + return true + } + } + return false + } + if isRestoreRunning() { + return errors.Errorf("another restore operation is already running. " + + "Please retry later.") + } + + req.RestoreTs = State.GetTimestamp(false) + + // TODO: prevent partial restores when proposeRestoreOrSend only sends the restore + // request to a subset of groups. + errCh := make(chan error, len(currentGroups)) + for _, gid := range currentGroups { + reqCopy := proto.Clone(req).(*pb.RestoreRequest) + reqCopy.GroupId = gid + wg.Add(1) + go func() { + errCh <- tryRestoreProposal(ctx, reqCopy) + }() + } + + go func() { + for range currentGroups { + if err := <-errCh; err != nil { + glog.Errorf("Error while restoring %v", err) + } + wg.Done() + } + }() + + return nil +} + +func proposeRestoreOrSend(ctx context.Context, req *pb.RestoreRequest) error { + if groups().ServesGroup(req.GetGroupId()) && groups().Node.AmLeader() { + _, err := (&grpcWorker{}).Restore(ctx, req) + return err + } + + pl := groups().Leader(req.GetGroupId()) + if pl == nil { + return conn.ErrNoConnection + } + con := pl.Get() + c := pb.NewWorkerClient(con) + + _, err := c.Restore(ctx, req) + return err +} + +func retriableRestoreError(err error) bool { + switch { + case err == conn.ErrNoConnection: + // Try to recover from temporary connection issues. + return true + case strings.Contains(err.Error(), "Raft isn't initialized yet"): + // Try to recover if raft has not been initialized. + return true + case strings.Contains(err.Error(), errRestoreProposal): + // Do not try to recover from other errors when sending the proposal. + return false + default: + // Try to recover from other errors (e.g wrong group, waiting for timestamp, etc). + return true + } +} + +func tryRestoreProposal(ctx context.Context, req *pb.RestoreRequest) error { + var err error + for i := 0; i < 10; i++ { + err = proposeRestoreOrSend(ctx, req) + if err == nil { + return nil + } + + if retriableRestoreError(err) { + time.Sleep(time.Second) + continue + } + return err + } + return err } // Restore implements the Worker interface. func (w *grpcWorker) Restore(ctx context.Context, req *pb.RestoreRequest) (*pb.Status, error) { - glog.Warningf("Restore failed: %v", x.ErrNotSupported) - return &pb.Status{}, x.ErrNotSupported + var emptyRes pb.Status + if !groups().ServesGroup(req.GroupId) { + return &emptyRes, errors.Errorf("this server doesn't serve group id: %v", req.GroupId) + } + + // We should wait to ensure that we have seen all the updates until the StartTs + // of this restore transaction. + if err := posting.Oracle().WaitForTs(ctx, req.RestoreTs); err != nil { + return nil, errors.Wrapf(err, "cannot wait for restore ts %d", req.RestoreTs) + } + + err := groups().Node.proposeAndWait(ctx, &pb.Proposal{Restore: req}) + if err != nil { + return &emptyRes, errors.Wrapf(err, errRestoreProposal) + } + + return &emptyRes, nil } +// TODO(DGRAPH-1232): Ensure all groups receive the restore proposal. func handleRestoreProposal(ctx context.Context, req *pb.RestoreRequest, pidx uint64) error { + if req == nil { + return errors.Errorf("nil restore request") + } + + // Drop all the current data. This also cancels all existing transactions. + dropProposal := pb.Proposal{ + Mutations: &pb.Mutations{ + GroupId: req.GroupId, + StartTs: req.RestoreTs, + DropOp: pb.Mutations_ALL, + }, + } + if err := groups().Node.applyMutations(ctx, &dropProposal); err != nil { + return err + } + + // TODO: after the drop, the tablets for the predicates stored in this group's + // backup could be in a different group. The tablets need to be moved. + + // Reset tablets and set correct tablets to match the restored backup. + creds := &x.MinioCredentials{ + AccessKey: req.AccessKey, + SecretKey: req.SecretKey, + SessionToken: req.SessionToken, + Anonymous: req.Anonymous, + } + uri, err := url.Parse(req.Location) + if err != nil { + return errors.Wrapf(err, "cannot parse backup location") + } + handler, err := NewUriHandler(uri, creds) + if err != nil { + return errors.Wrapf(err, "cannot create backup handler") + } + + manifests, err := getManifestsToRestore(handler, uri, req) + if err != nil { + return errors.Wrapf(err, "cannot get backup manifests") + } + if len(manifests) == 0 { + return errors.Errorf("no backup manifests found at location %s", req.Location) + } + + lastManifest := manifests[0] + preds, ok := lastManifest.Groups[req.GroupId] + + // Version is 0 if the backup was taken on an old version (v20.11). + if lastManifest.Version == 0 { + tmp := make([]string, 0, len(preds)) + for _, pred := range preds { + tmp = append(tmp, x.GalaxyAttr(pred)) + } + preds = tmp + } + + if !ok { + return errors.Errorf("backup manifest does not contain information for group ID %d", + req.GroupId) + } + for _, pred := range preds { + // Force the tablet to be moved to this group, even if it's currently being served + // by another group. + if tablet, err := groups().ForceTablet(pred); err != nil { + return errors.Wrapf(err, "cannot create tablet for restored predicate %s", pred) + } else if tablet.GetGroupId() != req.GroupId { + return errors.Errorf("cannot assign tablet for pred %s to group %d", pred, req.GroupId) + } + } + + mapDir, err := ioutil.TempDir(x.WorkerConfig.TmpDir, "restore-map") + x.Check(err) + defer os.RemoveAll(mapDir) + glog.Infof("Created temporary map directory: %s\n", mapDir) + + // Map the backup. + mapRes, err := RunMapper(req, mapDir) + if err != nil { + return errors.Wrapf(err, "Failed to map the backup files") + } + glog.Infof("Backup map phase is complete. Map result is: %+v\n", mapRes) + + // Reduce the map to pstore using stream writer. + sw := pstore.NewStreamWriter() + if err := sw.Prepare(); err != nil { + return errors.Wrapf(err, "while preparing DB") + } + if err := RunReducer(sw, mapDir); err != nil { + return errors.Wrap(err, "failed to reduce restore map") + } + if err := sw.Flush(); err != nil { + return errors.Wrap(err, "while stream writer flush") + } + + // Bump the UID and NsId lease after restore. + if err := bumpLease(ctx, mapRes); err != nil { + return errors.Wrap(err, "While bumping the leases after restore") + } + + // Load schema back. + if err := schema.LoadFromDb(); err != nil { + return errors.Wrapf(err, "cannot load schema after restore") + } + + ResetAclCache() + + // reset gql schema + glog.Info("reseting local gql schema store") + ResetGQLSchemaStore() + + // Propose a snapshot immediately after all the work is done to prevent the restore + // from being replayed. + go func(idx uint64) { + n := groups().Node + if !n.AmLeader() { + return + } + if err := n.Applied.WaitForMark(context.Background(), idx); err != nil { + glog.Errorf("Error waiting for mark for index %d: %+v", idx, err) + return + } + if err := n.proposeSnapshot(); err != nil { + glog.Errorf("cannot propose snapshot after processing restore proposal %+v", err) + } + }(pidx) + + // Update the membership state to re-compute the group checksums. + if err := UpdateMembershipState(ctx); err != nil { + return errors.Wrapf(err, "cannot update membership state after restore") + } + return nil +} + +func bumpLease(ctx context.Context, mr *mapResult) error { + pl := groups().connToZeroLeader() + if pl == nil { + return errors.Errorf("cannot update lease due to no connection to zero leader") + } + + zc := pb.NewZeroClient(pl.Get()) + bump := func(val uint64, typ pb.NumLeaseType) error { + _, err := zc.AssignIds(ctx, &pb.Num{Val: val, Type: typ, Bump: true}) + if err != nil && strings.Contains(err.Error(), "Nothing to be leased") { + return nil + } + return err + } + + if err := bump(mr.maxUid, pb.Num_UID); err != nil { + return errors.Wrapf(err, "cannot update max uid lease after restore.") + } + if err := bump(mr.maxNs, pb.Num_NS_ID); err != nil { + return errors.Wrapf(err, "cannot update max namespace lease after restore.") + } return nil } + +// create a config object from the request for use with enc package. +func getEncConfig(req *pb.RestoreRequest) (*viper.Viper, error) { + config := viper.New() + flags := &pflag.FlagSet{} + ee.RegisterEncFlag(flags) + if err := config.BindPFlags(flags); err != nil { + return nil, errors.Wrapf(err, "bad config bind") + } + + // Copy from the request. + config.Set("encryption", ee.BuildEncFlag(req.EncryptionKeyFile)) + + vaultBuilder := new(strings.Builder) + if req.VaultRoleidFile != "" { + fmt.Fprintf(vaultBuilder, "role-id-file=%s;", req.VaultRoleidFile) + } + if req.VaultSecretidFile != "" { + fmt.Fprintf(vaultBuilder, "secret-id-file=%s;", req.VaultSecretidFile) + } + if req.VaultAddr != "" { + fmt.Fprintf(vaultBuilder, "addr=%s;", req.VaultAddr) + } + if req.VaultPath != "" { + fmt.Fprintf(vaultBuilder, "path=%s;", req.VaultPath) + } + if req.VaultField != "" { + fmt.Fprintf(vaultBuilder, "field=%s;", req.VaultField) + } + if req.VaultFormat != "" { + fmt.Fprintf(vaultBuilder, "format=%s;", req.VaultFormat) + } + if vaultConfig := vaultBuilder.String(); vaultConfig != "" { + config.Set("vault", vaultConfig) + } + + return config, nil +} + +func getCredentialsFromRestoreRequest(req *pb.RestoreRequest) *x.MinioCredentials { + return &x.MinioCredentials{ + AccessKey: req.AccessKey, + SecretKey: req.SecretKey, + SessionToken: req.SessionToken, + Anonymous: req.Anonymous, + } +} + +// RunOfflineRestore creates required DBs and streams the backups to them. +func RunOfflineRestore(dir, location, backupId, keyFile string, key x.Sensitive, + ctype options.CompressionType, clevel int) LoadResult { + + // Create the pdir if it doesn't exist. + if err := os.MkdirAll(dir, 0700); err != nil { + return LoadResult{Err: err} + } + + uri, err := url.Parse(location) + if err != nil { + return LoadResult{Err: err} + } + + h, err := NewUriHandler(uri, nil) + if err != nil { + return LoadResult{Err: errors.Errorf("Unsupported URI: %v", uri)} + } + manifest, err := GetLatestManifest(h, uri) + if err != nil { + return LoadResult{Err: errors.Wrapf(err, "cannot retrieve manifests")} + } + if len(keyFile) > 0 { + key, err = ioutil.ReadFile(keyFile) + if err != nil { + return LoadResult{Err: errors.Wrapf(err, "RunRestore failed to read enc-key")} + } + } + + mapDir, err := ioutil.TempDir(x.WorkerConfig.TmpDir, "restore-map") + x.Check(err) + defer os.RemoveAll(mapDir) + + for gid := range manifest.Groups { + req := &pb.RestoreRequest{ + Location: location, + GroupId: gid, + BackupId: backupId, + EncryptionKeyFile: keyFile, + RestoreTs: 1, + } + if _, err := RunMapper(req, mapDir); err != nil { + return LoadResult{Err: errors.Wrap(err, "RunRestore failed to map")} + } + pdir := filepath.Join(dir, fmt.Sprintf("p%d", gid)) + db, err := badger.OpenManaged(badger.DefaultOptions(pdir). + WithCompression(ctype). + WithZSTDCompressionLevel(clevel). + WithSyncWrites(false). + WithBlockCacheSize(100 * (1 << 20)). + WithIndexCacheSize(100 * (1 << 20)). + WithNumVersionsToKeep(math.MaxInt32). + WithEncryptionKey(key). + WithNamespaceOffset(x.NamespaceOffset)) + if err != nil { + return LoadResult{Err: errors.Wrap(err, "RunRestore failed to open DB")} + } + defer db.Close() + + sw := db.NewStreamWriter() + if err := sw.Prepare(); err != nil { + return LoadResult{Err: errors.Wrap(err, "while preparing DB")} + } + if err := RunReducer(sw, mapDir); err != nil { + return LoadResult{Err: errors.Wrap(err, "RunRestore failed to reduce")} + } + if err := sw.Flush(); err != nil { + return LoadResult{Err: errors.Wrap(err, "while stream writer flush")} + } + if err := x.WriteGroupIdFile(pdir, uint32(gid)); err != nil { + return LoadResult{Err: errors.Wrap(err, "RunRestore failed to write group id file")} + } + } + // TODO: Fix this return value. + return LoadResult{Version: manifest.ValidReadTs()} +} diff --git a/worker/online_restore_ee.go b/worker/online_restore_ee.go deleted file mode 100644 index b29a8dfb9e9..00000000000 --- a/worker/online_restore_ee.go +++ /dev/null @@ -1,412 +0,0 @@ -// +build !oss - -/* - * Copyright 2022 Dgraph Labs, Inc. and Contributors - * - * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt - */ - -package worker - -import ( - "context" - "fmt" - "net/url" - "strings" - "sync" - "time" - - "github.com/golang/glog" - - "github.com/dgraph-io/dgraph/conn" - "github.com/dgraph-io/dgraph/ee" - "github.com/dgraph-io/dgraph/posting" - "github.com/dgraph-io/dgraph/protos/pb" - "github.com/dgraph-io/dgraph/schema" - "github.com/dgraph-io/dgraph/x" - - "github.com/golang/protobuf/proto" - "github.com/pkg/errors" - "github.com/spf13/pflag" - "github.com/spf13/viper" -) - -const ( - errRestoreProposal = "cannot propose restore request" -) - -// ProcessRestoreRequest verifies the backup data and sends a restore proposal to each group. -func ProcessRestoreRequest(ctx context.Context, req *pb.RestoreRequest, wg *sync.WaitGroup) error { - if req == nil { - return errors.Errorf("restore request cannot be nil") - } - - if err := UpdateMembershipState(ctx); err != nil { - return errors.Wrapf(err, "cannot update membership state before restore") - } - memState := GetMembershipState() - - currentGroups := make([]uint32, 0) - for gid := range memState.GetGroups() { - currentGroups = append(currentGroups, gid) - } - - creds := x.MinioCredentials{ - AccessKey: req.AccessKey, - SecretKey: req.SecretKey, - SessionToken: req.SessionToken, - Anonymous: req.Anonymous, - } - if err := VerifyBackup(req, &creds, currentGroups); err != nil { - return errors.Wrapf(err, "failed to verify backup") - } - if err := FillRestoreCredentials(req.Location, req); err != nil { - return errors.Wrapf(err, "cannot fill restore proposal with the right credentials") - } - - // This check if any restore operation running on the node. - // Operation initiated on other nodes doesn't have record in the record tracker. - // This keeps track if there is an already running restore operation return the error. - // IMP: This introduces few corner cases. - // Like two concurrent restore operation on different nodes. - // Considering Restore as admin operation, solving all those complexities has low gains - // than to sacrifice the simplicity. - isRestoreRunning := func() bool { - tasks := GetOngoingTasks() - for _, t := range tasks { - if t == opRestore.String() { - return true - } - } - return false - } - if isRestoreRunning() { - return errors.Errorf("another restore operation is already running. " + - "Please retry later.") - } - - req.RestoreTs = State.GetTimestamp(false) - - // TODO: prevent partial restores when proposeRestoreOrSend only sends the restore - // request to a subset of groups. - errCh := make(chan error, len(currentGroups)) - for _, gid := range currentGroups { - reqCopy := proto.Clone(req).(*pb.RestoreRequest) - reqCopy.GroupId = gid - wg.Add(1) - go func() { - errCh <- tryRestoreProposal(ctx, reqCopy) - }() - } - - go func() { - for range currentGroups { - if err := <-errCh; err != nil { - glog.Errorf("Error while restoring %v", err) - } - wg.Done() - } - }() - - return nil -} - -func proposeRestoreOrSend(ctx context.Context, req *pb.RestoreRequest) error { - if groups().ServesGroup(req.GetGroupId()) && groups().Node.AmLeader() { - _, err := (&grpcWorker{}).Restore(ctx, req) - return err - } - - pl := groups().Leader(req.GetGroupId()) - if pl == nil { - return conn.ErrNoConnection - } - con := pl.Get() - c := pb.NewWorkerClient(con) - - _, err := c.Restore(ctx, req) - return err -} - -func retriableRestoreError(err error) bool { - switch { - case err == conn.ErrNoConnection: - // Try to recover from temporary connection issues. - return true - case strings.Contains(err.Error(), "Raft isn't initialized yet"): - // Try to recover if raft has not been initialized. - return true - case strings.Contains(err.Error(), errRestoreProposal): - // Do not try to recover from other errors when sending the proposal. - return false - default: - // Try to recover from other errors (e.g wrong group, waiting for timestamp, etc). - return true - } -} - -func tryRestoreProposal(ctx context.Context, req *pb.RestoreRequest) error { - var err error - for i := 0; i < 10; i++ { - err = proposeRestoreOrSend(ctx, req) - if err == nil { - return nil - } - - if retriableRestoreError(err) { - time.Sleep(time.Second) - continue - } - return err - } - return err -} - -// Restore implements the Worker interface. -func (w *grpcWorker) Restore(ctx context.Context, req *pb.RestoreRequest) (*pb.Status, error) { - var emptyRes pb.Status - if !groups().ServesGroup(req.GroupId) { - return &emptyRes, errors.Errorf("this server doesn't serve group id: %v", req.GroupId) - } - - // We should wait to ensure that we have seen all the updates until the StartTs - // of this restore transaction. - if err := posting.Oracle().WaitForTs(ctx, req.RestoreTs); err != nil { - return nil, errors.Wrapf(err, "cannot wait for restore ts %d", req.RestoreTs) - } - - err := groups().Node.proposeAndWait(ctx, &pb.Proposal{Restore: req}) - if err != nil { - return &emptyRes, errors.Wrapf(err, errRestoreProposal) - } - - return &emptyRes, nil -} - -// TODO(DGRAPH-1232): Ensure all groups receive the restore proposal. -func handleRestoreProposal(ctx context.Context, req *pb.RestoreRequest, pidx uint64) error { - if req == nil { - return errors.Errorf("nil restore request") - } - - // Drop all the current data. This also cancels all existing transactions. - dropProposal := pb.Proposal{ - Mutations: &pb.Mutations{ - GroupId: req.GroupId, - StartTs: req.RestoreTs, - DropOp: pb.Mutations_ALL, - }, - } - if err := groups().Node.applyMutations(ctx, &dropProposal); err != nil { - return err - } - - // TODO: after the drop, the tablets for the predicates stored in this group's - // backup could be in a different group. The tablets need to be moved. - - // Reset tablets and set correct tablets to match the restored backup. - creds := &x.MinioCredentials{ - AccessKey: req.AccessKey, - SecretKey: req.SecretKey, - SessionToken: req.SessionToken, - Anonymous: req.Anonymous, - } - uri, err := url.Parse(req.Location) - if err != nil { - return errors.Wrapf(err, "cannot parse backup location") - } - handler, err := NewUriHandler(uri, creds) - if err != nil { - return errors.Wrapf(err, "cannot create backup handler") - } - - manifests, err := handler.GetManifests(uri, req.BackupId, req.BackupNum) - if err != nil { - return errors.Wrapf(err, "cannot get backup manifests") - } - if len(manifests) == 0 { - return errors.Errorf("no backup manifests found at location %s", req.Location) - } - - lastManifest := manifests[len(manifests)-1] - preds, ok := lastManifest.Groups[req.GroupId] - - // Version is 0 if the backup was taken on an old version (v20.11). - if lastManifest.Version == 0 { - tmp := make([]string, 0, len(preds)) - for _, pred := range preds { - tmp = append(tmp, x.GalaxyAttr(pred)) - } - preds = tmp - } - - if !ok { - return errors.Errorf("backup manifest does not contain information for group ID %d", - req.GroupId) - } - for _, pred := range preds { - // Force the tablet to be moved to this group, even if it's currently being served - // by another group. - if tablet, err := groups().ForceTablet(pred); err != nil { - return errors.Wrapf(err, "cannot create tablet for restored predicate %s", pred) - } else if tablet.GetGroupId() != req.GroupId { - return errors.Errorf("cannot assign tablet for pred %s to group %d", pred, req.GroupId) - } - } - - // Write restored values to disk and update the UID lease. - if err := writeBackup(ctx, req); err != nil { - return errors.Wrapf(err, "cannot write backup") - } - - // Load schema back. - if err := schema.LoadFromDb(); err != nil { - return errors.Wrapf(err, "cannot load schema after restore") - } - - ResetAclCache() - // Propose a snapshot immediately after all the work is done to prevent the restore - // from being replayed. - go func(idx uint64) { - n := groups().Node - if !n.AmLeader() { - return - } - if err := n.Applied.WaitForMark(context.Background(), idx); err != nil { - glog.Errorf("Error waiting for mark for index %d: %+v", idx, err) - return - } - if err := n.proposeSnapshot(); err != nil { - glog.Errorf("cannot propose snapshot after processing restore proposal %+v", err) - } - }(pidx) - - // Update the membership state to re-compute the group checksums. - if err := UpdateMembershipState(ctx); err != nil { - return errors.Wrapf(err, "cannot update membership state after restore") - } - return nil -} - -// create a config object from the request for use with enc package. -func getEncConfig(req *pb.RestoreRequest) (*viper.Viper, error) { - config := viper.New() - flags := &pflag.FlagSet{} - ee.RegisterEncFlag(flags) - if err := config.BindPFlags(flags); err != nil { - return nil, errors.Wrapf(err, "bad config bind") - } - - // Copy from the request. - config.Set("encryption", ee.BuildEncFlag(req.EncryptionKeyFile)) - - vaultBuilder := new(strings.Builder) - if req.VaultRoleidFile != "" { - fmt.Fprintf(vaultBuilder, "role-id-file=%s;", req.VaultRoleidFile) - } - if req.VaultSecretidFile != "" { - fmt.Fprintf(vaultBuilder, "secret-id-file=%s;", req.VaultSecretidFile) - } - if req.VaultAddr != "" { - fmt.Fprintf(vaultBuilder, "addr=%s;", req.VaultAddr) - } - if req.VaultPath != "" { - fmt.Fprintf(vaultBuilder, "path=%s;", req.VaultPath) - } - if req.VaultField != "" { - fmt.Fprintf(vaultBuilder, "field=%s;", req.VaultField) - } - if req.VaultFormat != "" { - fmt.Fprintf(vaultBuilder, "format=%s;", req.VaultFormat) - } - if vaultConfig := vaultBuilder.String(); vaultConfig != "" { - config.Set("vault", vaultConfig) - } - - return config, nil -} - -func getCredentialsFromRestoreRequest(req *pb.RestoreRequest) *x.MinioCredentials { - return &x.MinioCredentials{ - AccessKey: req.AccessKey, - SecretKey: req.SecretKey, - SessionToken: req.SessionToken, - Anonymous: req.Anonymous, - } -} - -func writeBackup(ctx context.Context, req *pb.RestoreRequest) error { - res := LoadBackup(req.Location, req.BackupId, req.BackupNum, - getCredentialsFromRestoreRequest(req), - func(groupId uint32, in *loadBackupInput) (uint64, uint64, error) { - if groupId != req.GroupId { - // LoadBackup will try to call the backup function for every group. - // Exit here if the group is not the one indicated by the request. - return 0, 0, nil - } - - cfg, err := getEncConfig(req) - if err != nil { - return 0, 0, errors.Wrapf(err, "unable to get encryption config") - } - keys, err := ee.GetKeys(cfg) - if err != nil { - return 0, 0, err - } - bReader, err := in.getReader(keys.EncKey) - if err != nil { - return 0, 0, errors.Wrap(err, "failed to getReader for restore") - } - - maxUid, maxNsId, err := loadFromBackup(pstore, &loadBackupInput{ - r: bReader, - restoreTs: req.RestoreTs, - preds: in.preds, - dropOperations: in.dropOperations, - isOld: in.isOld, - }) - if err != nil { - return 0, 0, errors.Wrapf(err, "cannot write backup") - } - - if maxUid == 0 { - // No need to update the lease, return here. - return 0, 0, nil - } - - // Use the value of maxUid to update the uid lease. - pl := groups().connToZeroLeader() - if pl == nil { - return 0, 0, errors.Errorf( - "cannot update uid lease due to no connection to zero leader") - } - - zc := pb.NewZeroClient(pl.Get()) - leaseID := func(val uint64, typ pb.NumLeaseType) error { - if val == 0 { - return nil - } - _, err := zc.AssignIds(ctx, &pb.Num{Val: val, Type: typ}) - return err - } - - if err := leaseID(maxUid, pb.Num_UID); err != nil { - return 0, 0, errors.Wrapf(err, "cannot update max uid lease after restore.") - } - if err := leaseID(maxNsId, pb.Num_NS_ID); err != nil { - return 0, 0, errors.Wrapf(err, "cannot update max namespace lease after restore.") - } - - // We return the maxUid/maxNsId to enforce the signature of the method but it will - // be ignored as the uid lease was updated above. - return maxUid, maxNsId, nil - }) - if res.Err != nil { - return errors.Wrapf(res.Err, "cannot write backup") - } - return nil -} diff --git a/worker/online_restore_oss.go b/worker/online_restore_oss.go new file mode 100644 index 00000000000..46b907feb36 --- /dev/null +++ b/worker/online_restore_oss.go @@ -0,0 +1,43 @@ +// +build oss + +/* + * Copyright 2020 Dgraph Labs, Inc. and Contributors + * + * 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 worker + +import ( + "context" + "sync" + + "github.com/dgraph-io/dgraph/protos/pb" + "github.com/dgraph-io/dgraph/x" + "github.com/golang/glog" +) + +func ProcessRestoreRequest(ctx context.Context, req *pb.RestoreRequest, wg *sync.WaitGroup) error { + glog.Warningf("Restore failed: %v", x.ErrNotSupported) + return x.ErrNotSupported +} + +// Restore implements the Worker interface. +func (w *grpcWorker) Restore(ctx context.Context, req *pb.RestoreRequest) (*pb.Status, error) { + glog.Warningf("Restore failed: %v", x.ErrNotSupported) + return &pb.Status{}, x.ErrNotSupported +} + +func handleRestoreProposal(ctx context.Context, req *pb.RestoreRequest) error { + return nil +} diff --git a/worker/restore.go b/worker/restore.go deleted file mode 100644 index 17d8a944dad..00000000000 --- a/worker/restore.go +++ /dev/null @@ -1,317 +0,0 @@ -//go:build !oss -// +build !oss - -/* - * Copyright 2022 Dgraph Labs, Inc. and Contributors - * - * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt - */ - -package worker - -import ( - "bufio" - "compress/gzip" - "encoding/binary" - "encoding/hex" - "fmt" - "io" - "math" - "os" - "path/filepath" - "strconv" - - "github.com/dgraph-io/badger/v3" - "github.com/dgraph-io/badger/v3/options" - bpb "github.com/dgraph-io/badger/v3/pb" - "github.com/golang/glog" - "github.com/golang/snappy" - "github.com/pkg/errors" - - "github.com/dgraph-io/dgraph/codec" - "github.com/dgraph-io/dgraph/ee/enc" - "github.com/dgraph-io/dgraph/posting" - "github.com/dgraph-io/dgraph/protos/pb" - "github.com/dgraph-io/dgraph/x" -) - -// RunRestore calls badger.Load and tries to load data into a new DB. -func RunRestore(pdir, location, backupId string, key x.Sensitive, - ctype options.CompressionType, clevel int) LoadResult { - // Create the pdir if it doesn't exist. - if err := os.MkdirAll(pdir, 0700); err != nil { - return LoadResult{Err: err} - } - - // Scan location for backup files and load them. Each file represents a node group, - // and we create a new p dir for each. - return LoadBackup(location, backupId, 0, nil, - func(groupId uint32, in *loadBackupInput) (uint64, uint64, error) { - bReader, err := in.getReader(key) - if err != nil { - return 0, 0, errors.Wrap(err, "failed to get reader for restore") - } - dir := filepath.Join(pdir, fmt.Sprintf("p%d", groupId)) - if !pathExist(dir) { - fmt.Println("Creating new db:", dir) - } - // The badger DB should be opened only after creating the backup - // file reader and verifying the encryption in the backup file. - db, err := badger.OpenManaged(badger.DefaultOptions(dir). - WithCompression(ctype). - WithZSTDCompressionLevel(clevel). - WithSyncWrites(false). - WithBlockCacheSize(100 * (1 << 20)). - WithIndexCacheSize(100 * (1 << 20)). - WithNumVersionsToKeep(math.MaxInt32). - WithEncryptionKey(key). - WithNamespaceOffset(x.NamespaceOffset)) - if err != nil { - return 0, 0, err - } - defer db.Close() - maxUid, maxNsId, err := loadFromBackup(db, &loadBackupInput{ - r: bReader, - restoreTs: 0, - preds: in.preds, - dropOperations: in.dropOperations, - isOld: in.isOld, - compression: in.compression, - }) - if err != nil { - return 0, 0, errors.Wrap(err, "loadFromBackup failed") - } - return maxUid, maxNsId, x.WriteGroupIdFile(dir, uint32(groupId)) - }) -} - -type loadBackupInput struct { - r io.Reader - restoreTs uint64 - preds predicateSet - dropOperations []*pb.DropOperation - isOld bool - compression string -} - -func (l *loadBackupInput) getReader(key x.Sensitive) (io.Reader, error) { - r, err := enc.GetReader(key, l.r) - if err != nil { - return nil, err - } - switch l.compression { - case "": - gzReader, err := gzip.NewReader(r) - if err != nil && len(key) != 0 { - err = errors.Wrap(err, - "Unable to read the backup. Ensure the encryption key is correct.") - } - return gzReader, err - case "snappy": - // Snappy doesn't return an error. If the data is encrypted, we will - // get an error while reading it. - return snappy.NewReader(r), nil - default: - return nil, errors.Errorf("Invalid compression in backup %q", l.compression) - } -} - -// loadFromBackup reads the backup, converts the keys and values to the required format, -// and loads them to the given badger DB. The set of predicates is used to avoid restoring -// values from predicates no longer assigned to this group. -// If restoreTs is greater than zero, the key-value pairs will be written with that timestamp. -// Otherwise, the original value is used. -// TODO(DGRAPH-1234): Check whether restoreTs can be removed. -func loadFromBackup(db *badger.DB, in *loadBackupInput) (uint64, uint64, error) { - br := bufio.NewReaderSize(in.r, 16<<10) - unmarshalBuf := make([]byte, 1<<10) - - // if there were any DROP operations that need to be applied before loading the backup into - // the db, then apply them here - if err := applyDropOperationsBeforeRestore(db, in.dropOperations, in.isOld); err != nil { - return 0, 0, errors.Wrapf(err, "cannot apply DROP operations while loading backup") - } - - // Delete schemas and types. Each backup file should have a complete copy of the schema. - if err := db.DropPrefix([]byte{x.ByteSchema}); err != nil { - return 0, 0, err - } - if err := db.DropPrefix([]byte{x.ByteType}); err != nil { - return 0, 0, err - } - - loader := db.NewKVLoader(16) - var maxUid, maxNsId uint64 - for { - var sz uint64 - err := binary.Read(br, binary.LittleEndian, &sz) - if err == io.EOF { - break - } else if err != nil { - return 0, 0, errors.Wrap(err, "read failed") - } - - if cap(unmarshalBuf) < int(sz) { - unmarshalBuf = make([]byte, sz) - } - - if _, err = io.ReadFull(br, unmarshalBuf[:sz]); err != nil { - return 0, 0, err - } - - list := &bpb.KVList{} - if err := list.Unmarshal(unmarshalBuf[:sz]); err != nil { - return 0, 0, err - } - - for _, kv := range list.Kv { - if len(kv.GetUserMeta()) != 1 { - return 0, 0, errors.Errorf( - "Unexpected meta: %v for key: %s", kv.UserMeta, hex.Dump(kv.Key)) - } - - restoreKey, namespace, err := fromBackupKey(kv.Key) - if err != nil { - return 0, 0, err - } - - // Filter keys using the preds set. Do not do this filtering for type keys - // as they are meant to be in every group and their Attr value does not - // match a predicate name. - parsedKey, err := x.Parse(restoreKey) - if err != nil { - return 0, 0, errors.Wrapf(err, "could not parse key %s", hex.Dump(restoreKey)) - } - if _, ok := in.preds[parsedKey.Attr]; !parsedKey.IsType() && !ok { - continue - } - - // Update the max uid and namespace id that has been seen while restoring this backup. - maxUid = x.Max(maxUid, parsedKey.Uid) - maxNsId = x.Max(maxNsId, namespace) - - // Override the version if requested. Should not be done for type and schema predicates, - // which always have their version set to 1. - if in.restoreTs > 0 && !parsedKey.IsSchema() && !parsedKey.IsType() { - kv.Version = in.restoreTs - } - - switch kv.GetUserMeta()[0] { - case posting.BitEmptyPosting, posting.BitCompletePosting, posting.BitDeltaPosting: - backupPl := &pb.BackupPostingList{} - if err := backupPl.Unmarshal(kv.Value); err != nil { - return 0, 0, errors.Wrapf(err, "while reading backup posting list") - } - pl := posting.FromBackupPostingList(backupPl) - shouldSplit := pl.Size() >= (1<<20)/2 && len(pl.Pack.Blocks) > 1 - - if !shouldSplit || parsedKey.HasStartUid || len(pl.GetSplits()) > 0 { - // This covers two cases. - // 1. The list is not big enough to be split. - // 2. This key is storing part of a multi-part list. Write each individual - // part without rolling the key first. This part is here for backwards - // compatibility. New backups are not affected because there was a change - // to roll up lists into a single one. - newKv := posting.MarshalPostingList(pl, nil) - codec.FreePack(pl.Pack) - newKv.Key = restoreKey - // Use the version of the KV before we marshalled the - // posting list. The MarshalPostingList function returns KV - // with a zero version. - newKv.Version = kv.Version - if err := loader.Set(newKv); err != nil { - return 0, 0, err - } - } else { - // This is a complete list. It should be rolled up to avoid writing - // a list that is too big to be read back from disk. - // Rollup will take ownership of the Pack and will free the memory. - l := posting.NewList(restoreKey, pl, kv.Version) - kvs, err := l.Rollup(nil) - if err != nil { - // TODO: wrap errors in this file for easier debugging. - return 0, 0, err - } - for _, kv := range kvs { - if err := loader.Set(kv); err != nil { - return 0, 0, err - } - } - } - - case posting.BitSchemaPosting: - appendNamespace := func() error { - // If the backup was taken on old version, we need to append the namespace to - // the fields of TypeUpdate. - var update pb.TypeUpdate - if err := update.Unmarshal(kv.Value); err != nil { - return err - } - update.TypeName = x.GalaxyAttr(update.TypeName) - for _, sch := range update.Fields { - sch.Predicate = x.GalaxyAttr(sch.Predicate) - } - kv.Value, err = update.Marshal() - return err - } - if in.isOld && parsedKey.IsType() { - if err := appendNamespace(); err != nil { - glog.Errorf("Unable to (un)marshal type: %+v. Err=%v\n", parsedKey, err) - continue - } - } - // Schema and type keys are not stored in an intermediate format so their - // value can be written as is. - kv.Key = restoreKey - if err := loader.Set(kv); err != nil { - return 0, 0, err - } - - default: - return 0, 0, errors.Errorf( - "Unexpected meta %d for key %s", kv.UserMeta[0], hex.Dump(kv.Key)) - } - } - } - - if err := loader.Finish(); err != nil { - return 0, 0, err - } - - return maxUid, maxNsId, nil -} - -func applyDropOperationsBeforeRestore( - db *badger.DB, dropOperations []*pb.DropOperation, isOld bool) error { - for _, operation := range dropOperations { - switch operation.DropOp { - case pb.DropOperation_ALL: - return db.DropAll() - case pb.DropOperation_DATA: - return db.DropPrefix([]byte{x.DefaultPrefix}) - case pb.DropOperation_ATTR: - attr := operation.DropValue - if isOld { - attr = x.GalaxyAttr(operation.DropValue) - } - return db.DropPrefix(x.PredicatePrefix(attr)) - case pb.DropOperation_NS: - ns, err := strconv.ParseUint(operation.DropValue, 0, 64) - x.Check(err) - return db.BanNamespace(ns) - } - } - return nil -} - -func fromBackupKey(key []byte) ([]byte, uint64, error) { - backupKey := &pb.BackupKey{} - if err := backupKey.Unmarshal(key); err != nil { - return nil, 0, errors.Wrapf(err, "while reading backup key %s", hex.Dump(key)) - } - return x.FromBackupKey(backupKey), backupKey.Namespace, nil -} diff --git a/worker/restore_map.go b/worker/restore_map.go new file mode 100644 index 00000000000..a553529c906 --- /dev/null +++ b/worker/restore_map.go @@ -0,0 +1,708 @@ +//go:build !oss +// +build !oss + +/* + * Copyright 2021 Dgraph Labs, Inc. and Contributors + * + * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt + */ + +package worker + +import ( + "bufio" + "bytes" + "compress/gzip" + "context" + "encoding/binary" + "encoding/hex" + "fmt" + "io" + "net/url" + "os" + "path/filepath" + "strconv" + "sync" + "sync/atomic" + "time" + + bpb "github.com/dgraph-io/badger/v3/pb" + "github.com/dgraph-io/badger/v3/y" + "github.com/dgraph-io/dgraph/codec" + "github.com/dgraph-io/dgraph/ee" + "github.com/dgraph-io/dgraph/ee/enc" + "github.com/dgraph-io/dgraph/posting" + "github.com/dgraph-io/dgraph/protos/pb" + "github.com/dgraph-io/dgraph/x" + "github.com/dgraph-io/ristretto/z" + + "github.com/dustin/go-humanize" + "github.com/golang/glog" + "github.com/golang/snappy" + "github.com/pkg/errors" + "golang.org/x/sync/errgroup" +) + +type backupReader struct { + toClose []io.Closer + r io.Reader + err error + once sync.Once +} + +func readerFrom(h UriHandler, file string) *backupReader { + br := &backupReader{} + reader, err := h.Stream(file) + br.setErr(err) + br.toClose = append(br.toClose, reader) + br.r = reader + return br +} +func (br *backupReader) Read(p []byte) (n int, err error) { + return br.r.Read(p) +} +func (br *backupReader) Close() (rerr error) { + br.once.Do(func() { + // Close in reverse order. + for i := len(br.toClose) - 1; i >= 0; i-- { + if err := br.toClose[i].Close(); err != nil { + rerr = err + } + } + }) + return rerr +} +func (br *backupReader) setErr(err error) { + if br.err == nil { + br.err = err + } +} +func (br *backupReader) WithEncryption(encKey x.Sensitive) *backupReader { + if len(encKey) == 0 { + return br + } + r, err := enc.GetReader(encKey, br.r) + br.setErr(err) + br.r = r + return br +} +func (br *backupReader) WithCompression(comp string) *backupReader { + switch comp { + case "snappy": + br.r = snappy.NewReader(br.r) + case "gzip", "": + r, err := gzip.NewReader(br.r) + br.setErr(err) + br.r = r + br.toClose = append(br.toClose, r) + default: + br.setErr(fmt.Errorf("Unknown compression for backup: %s", comp)) + } + return br +} + +type loadBackupInput struct { + restoreTs uint64 + preds predicateSet + dropOperations []*pb.DropOperation + isOld bool + keepSchema bool + compression string +} + +type listReq struct { + lbuf *z.Buffer + in *loadBackupInput +} + +// mapEntry stores uint16 (2 bytes), which store the length of the key, followed by the key itself. +// The rest of the mapEntry stores the marshalled KV. +// We store the key alongside the protobuf, to make it easier to parse for comparison. +type mapEntry []byte + +func (me mapEntry) Key() []byte { + sz := binary.BigEndian.Uint16(me[0:2]) + return me[2 : 2+sz] +} +func (me mapEntry) Data() []byte { + sz := binary.BigEndian.Uint16(me[0:2]) + return me[2+sz:] +} + +type mapper struct { + once sync.Once + nextId uint32 + thr *y.Throttle + + bytesProcessed uint64 + bytesRead uint64 + closer *z.Closer + + buf *z.Buffer + bufLock *sync.Mutex + restoreTs uint64 + + mapDir string + reqCh chan listReq + szHist *z.HistogramData + + maxUid uint64 + maxNs uint64 +} + +func (mw *mapper) newMapFile() (*os.File, error) { + fileNum := atomic.AddUint32(&mw.nextId, 1) + filename := filepath.Join(mw.mapDir, fmt.Sprintf("%06d.map", fileNum)) + x.Check(os.MkdirAll(filepath.Dir(filename), 0750)) + + return os.OpenFile(filename, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, 0600) +} + +func (m *mapper) writeToDisk(buf *z.Buffer) error { + defer buf.Release() + if buf.IsEmpty() { + return nil + } + + f, err := m.newMapFile() + if err != nil { + return errors.Wrap(err, "openOutputFile") + } + defer f.Close() + + // Create partition keys for the map file. + header := &pb.MapHeader{PartitionKeys: [][]byte{}} + var bufSize int + buf.SliceIterate(func(slice []byte) error { + bufSize += 4 + len(slice) + if bufSize < partitionBufSz { + return nil + } + sz := len(header.PartitionKeys) + me := mapEntry(slice) + if sz > 0 && bytes.Equal(me.Key(), header.PartitionKeys[sz-1]) { + // We already have this key. + return nil + } + header.PartitionKeys = append(header.PartitionKeys, me.Key()) + bufSize = 0 + return nil + }) + + // Write the header to the map file. + headerBuf, err := header.Marshal() + x.Check(err) + var lenBuf [4]byte + binary.BigEndian.PutUint32(lenBuf[:], uint32(len(headerBuf))) + + w := snappy.NewBufferedWriter(f) + x.Check2(w.Write(lenBuf[:])) + x.Check2(w.Write(headerBuf)) + x.Check(err) + + sizeBuf := make([]byte, binary.MaxVarintLen64) + err = buf.SliceIterate(func(slice []byte) error { + n := binary.PutUvarint(sizeBuf, uint64(len(slice))) + _, err := w.Write(sizeBuf[:n]) + x.Check(err) + + _, err = w.Write(slice) + return err + }) + if err != nil { + return errors.Wrap(err, "sliceIterate") + } + if err := w.Close(); err != nil { + return errors.Wrap(err, "writer.Close") + } + if err := f.Sync(); err != nil { + return errors.Wrap(err, "file.Sync") + } + if fi, err := f.Stat(); err == nil { + glog.Infof("Created new backup map file: %s of size: %s\n", + fi.Name(), humanize.IBytes(uint64(fi.Size()))) + } + return f.Close() +} + +func (mw *mapper) sendForWriting() error { + if mw.buf.IsEmpty() { + return nil + } + mw.buf.SortSlice(func(ls, rs []byte) bool { + lme := mapEntry(ls) + rme := mapEntry(rs) + return y.CompareKeys(lme.Key(), rme.Key()) < 0 + }) + + if err := mw.thr.Do(); err != nil { + return err + } + go func(buf *z.Buffer) { + err := mw.writeToDisk(buf) + mw.thr.Done(err) + }(mw.buf) + mw.buf = z.NewBuffer(mapFileSz, "Restore.Buffer") + return nil +} + +func (mw *mapper) Flush() error { + cl := func() error { + if err := mw.sendForWriting(); err != nil { + return err + } + if err := mw.thr.Finish(); err != nil { + return err + } + return mw.buf.Release() + } + + var rerr error + mw.once.Do(func() { + rerr = cl() + }) + return rerr +} + +func fromBackupKey(key []byte) ([]byte, uint64, error) { + backupKey := &pb.BackupKey{} + if err := backupKey.Unmarshal(key); err != nil { + return nil, 0, errors.Wrapf(err, "while reading backup key %s", hex.Dump(key)) + } + return x.FromBackupKey(backupKey), backupKey.Namespace, nil +} + +func (m *mapper) processReqCh(ctx context.Context) error { + buf := z.NewBuffer(20<<20, "processKVList") + defer buf.Release() + + maxNs := uint64(0) + maxUid := uint64(0) + + toBuffer := func(kv *bpb.KV, version uint64) error { + key := y.KeyWithTs(kv.Key, version) + sz := kv.Size() + buf := buf.SliceAllocate(2 + len(key) + sz) + + binary.BigEndian.PutUint16(buf[0:2], uint16(len(key))) + x.AssertTrue(copy(buf[2:], key) == len(key)) + _, err := kv.MarshalToSizedBuffer(buf[2+len(key):]) + return err + } + + processKV := func(in *loadBackupInput, kv *bpb.KV) error { + if len(kv.GetUserMeta()) != 1 { + return errors.Errorf( + "Unexpected meta: %v for key: %s", kv.UserMeta, hex.Dump(kv.Key)) + } + + restoreKey, ns, err := fromBackupKey(kv.Key) + if err != nil { + return errors.Wrap(err, "fromBackupKey") + } + + // Filter keys using the preds set. Do not do this filtering for type keys + // as they are meant to be in every group and their Attr value does not + // match a predicate name. + parsedKey, err := x.Parse(restoreKey) + if err != nil { + return errors.Wrapf(err, "could not parse key %s", hex.Dump(restoreKey)) + } + + // Update the local max uid and max namespace values. + maxUid = x.Max(maxUid, parsedKey.Uid) + maxNs = x.Max(maxNs, ns) + + if !in.keepSchema && (parsedKey.IsSchema() || parsedKey.IsType()) { + return nil + } + if _, ok := in.preds[parsedKey.Attr]; !parsedKey.IsType() && !ok { + return nil + } + + switch kv.GetUserMeta()[0] { + case posting.BitEmptyPosting, posting.BitCompletePosting, posting.BitDeltaPosting: + backupPl := &pb.BackupPostingList{} + if err := backupPl.Unmarshal(kv.Value); err != nil { + return errors.Wrapf(err, "while reading backup posting list") + } + + pl := posting.FromBackupPostingList(backupPl) + defer codec.FreePack(pl.Pack) + + shouldSplit := pl.Size() >= (1<<20)/2 && len(pl.Pack.Blocks) > 1 + if !shouldSplit || parsedKey.HasStartUid || len(pl.GetSplits()) > 0 { + // This covers two cases. + // 1. The list is not big enough to be split. + // 2. This key is storing part of a multi-part list. Write each individual + // part without rolling the key first. This part is here for backwards + // compatibility. New backups are not affected because there was a change + // to roll up lists into a single one. + newKv := posting.MarshalPostingList(pl, nil) + newKv.Key = restoreKey + + // We are using kv.Version (from the key-value) to generate the key. But, using + // restoreTs to set the version of the KV. This way, when we sort the keys, we + // choose the latest key based on kv.Version. But, then set its version to + // restoreTs. + newKv.Version = m.restoreTs + if err := toBuffer(newKv, kv.Version); err != nil { + return err + } + } else { + // This is a complete list. It should be rolled up to avoid writing + // a list that is too big to be read back from disk. + // Rollup will take ownership of the Pack and will free the memory. + l := posting.NewList(restoreKey, pl, kv.Version) + kvs, err := l.Rollup(nil) + if err != nil { + // TODO: wrap errors in this file for easier debugging. + return err + } + for _, kv := range kvs { + if err := toBuffer(kv, kv.Version); err != nil { + return err + } + } + } + + case posting.BitSchemaPosting: + appendNamespace := func() error { + // If the backup was taken on old version, we need to append the namespace to + // the fields of TypeUpdate. + var update pb.TypeUpdate + if err := update.Unmarshal(kv.Value); err != nil { + return err + } + update.TypeName = x.GalaxyAttr(update.TypeName) + for _, sch := range update.Fields { + sch.Predicate = x.GalaxyAttr(sch.Predicate) + } + kv.Value, err = update.Marshal() + return err + } + if in.isOld && parsedKey.IsType() { + if err := appendNamespace(); err != nil { + glog.Errorf("Unable to (un)marshal type: %+v. Err=%v\n", parsedKey, err) + return nil + } + } + // Reset the StreamId to prevent ordering issues while writing to stream writer. + kv.StreamId = 0 + // Schema and type keys are not stored in an intermediate format so their + // value can be written as is. + kv.Key = restoreKey + if err := toBuffer(kv, kv.Version); err != nil { + return err + } + + default: + return errors.Errorf( + "Unexpected meta %d for key %s", kv.UserMeta[0], hex.Dump(kv.Key)) + } + return nil + } + + mergeBuffer := func() error { + if buf.IsEmpty() { + return nil + } + atomic.AddUint64(&m.bytesProcessed, uint64(buf.LenNoPadding())) + + m.bufLock.Lock() + defer m.bufLock.Unlock() + + x.Check2(m.buf.Write(buf.Bytes())) + buf.Reset() + + if m.buf.LenNoPadding() < mapFileSz { + return nil + } + return m.sendForWriting() + } + + var list bpb.KVList + process := func(req listReq) error { + defer req.lbuf.Release() + + if ctx.Err() != nil { + return ctx.Err() + } + return req.lbuf.SliceIterate(func(s []byte) error { + list.Reset() + if err := list.Unmarshal(s); err != nil { + return err + } + for _, kv := range list.GetKv() { + if err := processKV(req.in, kv); err != nil { + return err + } + if buf.LenNoPadding() > 16<<20 { + if err := mergeBuffer(); err != nil { + return err + } + } + } + return nil + }) + } + + for req := range m.reqCh { + if err := process(req); err != nil { + return err + } + } + if err := mergeBuffer(); err != nil { + return err + } + + // Update the global maxUid and maxNs. We need CAS here because mapping is + // being carried out concurrently. + for { + oldMaxUid := atomic.LoadUint64(&m.maxUid) + newMaxUid := x.Max(oldMaxUid, maxUid) + if swapped := atomic.CompareAndSwapUint64(&m.maxUid, oldMaxUid, newMaxUid); swapped { + break + } + } + for { + oldMaxNs := atomic.LoadUint64(&m.maxNs) + newMaxNs := x.Max(oldMaxNs, maxNs) + if swapped := atomic.CompareAndSwapUint64(&m.maxNs, oldMaxNs, newMaxNs); swapped { + break + } + } + + return nil +} + +func (m *mapper) Progress() { + defer m.closer.Done() + + ticker := time.NewTicker(time.Second) + defer ticker.Stop() + + start := time.Now() + update := func() { + read := atomic.LoadUint64(&m.bytesRead) + proc := atomic.LoadUint64(&m.bytesProcessed) + since := time.Since(start) + rate := uint64(float64(proc) / since.Seconds()) + glog.Infof("Restore MAP %s read: %s. output: %s. rate: %s/sec. jemalloc: %s.\n", + x.FixedDuration(since), humanize.IBytes(read), humanize.IBytes(proc), + humanize.IBytes(rate), humanize.IBytes(uint64(z.NumAllocBytes()))) + } + for { + select { + case <-m.closer.HasBeenClosed(): + update() + glog.Infof("Restore MAP Done in %s.\n", x.FixedDuration(time.Since(start))) + return + case <-ticker.C: + update() + } + } +} + +const bufSz = 64 << 20 +const bufSoftLimit = bufSz - 2<<20 + +// mapToDisk reads the backup, converts the keys and values to the required format, +// and loads them to the given badger DB. The set of predicates is used to avoid restoring +// values from predicates no longer assigned to this group. +// If restoreTs is greater than zero, the key-value pairs will be written with that timestamp. +// Otherwise, the original value is used. +// TODO(DGRAPH-1234): Check whether restoreTs can be removed. +func (m *mapper) Map(r io.Reader, in *loadBackupInput) error { + br := bufio.NewReaderSize(r, 16<<10) + zbuf := z.NewBuffer(bufSz, "Restore.Map") + + for { + var sz uint64 + err := binary.Read(br, binary.LittleEndian, &sz) + if err == io.EOF { + break + } else if err != nil { + return err + } + + m.szHist.Update(int64(sz)) + buf := zbuf.SliceAllocate(int(sz)) + if _, err = io.ReadFull(br, buf); err != nil { + return err + } + + if zbuf.LenNoPadding() > bufSoftLimit { + atomic.AddUint64(&m.bytesRead, uint64(zbuf.LenNoPadding())) + glog.Infof("Sending req of size: %s\n", humanize.IBytes(uint64(zbuf.LenNoPadding()))) + m.reqCh <- listReq{zbuf, in} + zbuf = z.NewBuffer(bufSz, "Restore.Map") + } + } + m.reqCh <- listReq{zbuf, in} + return nil +} + +type mapResult struct { + maxUid uint64 + maxNs uint64 +} + +// 1. RunMapper creates a mapper object +// 2. mapper.Map() -> +func RunMapper(req *pb.RestoreRequest, mapDir string) (*mapResult, error) { + uri, err := url.Parse(req.Location) + if err != nil { + return nil, err + } + if req.RestoreTs == 0 { + return nil, errors.New("RestoreRequest must have a valid restoreTs") + } + + creds := getCredentialsFromRestoreRequest(req) + h, err := NewUriHandler(uri, creds) + if err != nil { + return nil, err + } + + manifests, err := getManifestsToRestore(h, uri, req) + if err != nil { + return nil, errors.Wrapf(err, "cannot retrieve manifests") + } + glog.Infof("Got %d backups to restore ", len(manifests)) + + cfg, err := getEncConfig(req) + if err != nil { + return nil, errors.Wrapf(err, "unable to get encryption config") + } + keys, err := ee.GetKeys(cfg) + if err != nil { + return nil, errors.Wrapf(err, "unable to get encryption keys") + } + + mapper := &mapper{ + buf: z.NewBuffer(mapFileSz, "Restore.Buffer"), + thr: y.NewThrottle(3), + bufLock: &sync.Mutex{}, + closer: z.NewCloser(1), + reqCh: make(chan listReq, 3), + restoreTs: req.RestoreTs, + mapDir: mapDir, + szHist: z.NewHistogramData(z.HistogramBounds(10, 32)), + } + + numGo := 8 + g, ctx := errgroup.WithContext(mapper.closer.Ctx()) + for i := 0; i < numGo; i++ { + g.Go(func() error { + return mapper.processReqCh(ctx) + }) + } + go mapper.Progress() + defer func() { + mapper.Flush() + mapper.closer.SignalAndWait() + }() + + dropAll := false + dropAttr := make(map[string]struct{}) + + // manifests are ordered as: latest..full + for i, manifest := range manifests { + // A dropAll or DropData operation is encountered. No need to restore previous backups. + if dropAll { + break + } + if manifest.ValidReadTs() == 0 || len(manifest.Groups) == 0 { + continue + } + for gid := range manifest.Groups { + if gid != req.GroupId { + // LoadBackup will try to call the backup function for every group. + // Exit here if the group is not the one indicated by the request. + continue + } + + // Only restore the predicates that were assigned to this group at the time + // of the last backup. + file := filepath.Join(manifest.Path, backupName(manifest.ValidReadTs(), gid)) + br := readerFrom(h, file).WithEncryption(keys.EncKey).WithCompression(manifest.Compression) + if br.err != nil { + return nil, errors.Wrap(br.err, "newBackupReader") + } + defer br.Close() + + // Only map the predicates which haven't been dropped yet. + predSet := manifest.getPredsInGroup(gid) + for p := range predSet { + if _, ok := dropAttr[p]; ok { + delete(predSet, p) + } + } + in := &loadBackupInput{ + preds: predSet, + dropOperations: manifest.DropOperations, + isOld: manifest.Version == 0, + restoreTs: req.RestoreTs, + // Only map the schema keys corresponding to the latest backup. + keepSchema: i == 0, + compression: manifest.Compression, + } + + // This would stream the backups from the source, and map them in + // Dgraph compatible format on disk. + if err := mapper.Map(br, in); err != nil { + return nil, errors.Wrap(err, "mapper.Map") + } + if err := br.Close(); err != nil { + return nil, errors.Wrap(err, "br.Close") + } + } + for _, op := range manifest.DropOperations { + switch op.DropOp { + case pb.DropOperation_ALL: + dropAll = true + case pb.DropOperation_DATA: + dropAll = true + case pb.DropOperation_ATTR: + p := op.DropValue + if manifest.Version == 0 { + p = x.NamespaceAttr(x.GalaxyNamespace, p) + } + dropAttr[p] = struct{}{} + case pb.DropOperation_NS: + // pstore will be nil for export_backup tool. In that case we don't need to ban ns. + if pstore == nil { + continue + } + // If there is a drop namespace, we just ban the namespace in the pstore. + // TODO: We probably need to propose ban request. + ns, err := strconv.ParseUint(op.DropValue, 0, 64) + if err != nil { + return nil, errors.Wrapf(err, "Map phase failed to parse namespace") + } + if err := pstore.BanNamespace(ns); err != nil { + return nil, errors.Wrapf(err, "Map phase failed to ban namespace: %d", ns) + } + } + } + } // done with all the manifests. + + glog.Infof("Histogram of map input sizes:\n%s\n", mapper.szHist) + close(mapper.reqCh) + if err := g.Wait(); err != nil { + return nil, errors.Wrapf(err, "from processKVList") + } + if err := mapper.Flush(); err != nil { + return nil, errors.Wrap(err, "failed to flush the mapper") + } + mapRes := &mapResult{ + maxUid: mapper.maxUid, + maxNs: mapper.maxNs, + } + return mapRes, nil +} diff --git a/worker/restore_reduce.go b/worker/restore_reduce.go new file mode 100644 index 00000000000..a99751a1770 --- /dev/null +++ b/worker/restore_reduce.go @@ -0,0 +1,314 @@ +//go:build !oss +// +build !oss + +/* + * Copyright 2019 Dgraph Labs, Inc. and Contributors + * + * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt + */ + +package worker + +import ( + "bufio" + "encoding/binary" + "io" + "log" + "os" + "path/filepath" + "sort" + "strings" + "sync/atomic" + "time" + + "github.com/dgraph-io/badger/v3/y" + "github.com/dgraph-io/dgraph/protos/pb" + "github.com/dgraph-io/dgraph/x" + "github.com/dgraph-io/ristretto/z" + + "github.com/dustin/go-humanize" + "github.com/golang/glog" + "github.com/golang/snappy" +) + +const ( + mapFileSz int = 2 << 30 + partitionBufSz int = 4 << 20 +) + +type mapIterator struct { + fd *os.File + reader *bufio.Reader + meBuf []byte +} + +func (mi *mapIterator) Next(cbuf *z.Buffer, partitionKey []byte) error { + readMapEntry := func() error { + if len(mi.meBuf) > 0 { + return nil + } + r := mi.reader + sizeBuf, err := r.Peek(binary.MaxVarintLen64) + if err != nil { + return err + } + sz, n := binary.Uvarint(sizeBuf) + if n <= 0 { + log.Fatalf("Could not read uvarint: %d", n) + } + x.Check2(r.Discard(n)) + if cap(mi.meBuf) < int(sz) { + mi.meBuf = make([]byte, int(sz)) + } + mi.meBuf = mi.meBuf[:int(sz)] + x.Check2(io.ReadFull(r, mi.meBuf)) + return nil + } + for { + if err := readMapEntry(); err == io.EOF { + break + } else if err != nil { + return err + } + key := mapEntry(mi.meBuf).Key() + + if len(partitionKey) == 0 || y.CompareKeys(key, partitionKey) < 0 { + b := cbuf.SliceAllocate(len(mi.meBuf)) + copy(b, mi.meBuf) + mi.meBuf = mi.meBuf[:0] + // map entry is already part of cBuf. + continue + } + // Current key is not part of this batch so track that we have already read the key. + return nil + } + return nil +} + +func (mi *mapIterator) Close() error { + return mi.fd.Close() +} + +func newMapIterator(filename string) (*pb.MapHeader, *mapIterator) { + fd, err := os.Open(filename) + x.Check(err) + r := snappy.NewReader(fd) + + // Read the header size. + reader := bufio.NewReaderSize(r, 16<<10) + headerLenBuf := make([]byte, 4) + x.Check2(io.ReadFull(reader, headerLenBuf)) + headerLen := binary.BigEndian.Uint32(headerLenBuf) + // Reader the map header. + headerBuf := make([]byte, headerLen) + + x.Check2(io.ReadFull(reader, headerBuf)) + header := &pb.MapHeader{} + err = header.Unmarshal(headerBuf) + x.Check(err) + + itr := &mapIterator{ + fd: fd, + reader: reader, + } + return header, itr +} + +type reducer struct { + mapDir string + mapItrs []*mapIterator + partitionKeys [][]byte + bufferCh chan *z.Buffer + w Writer + + bytesProcessed uint64 + bytesRead uint64 +} + +type Writer interface { + Write(buf *z.Buffer) error +} + +func RunReducer(w Writer, mapDir string) error { + r := &reducer{ + w: w, + bufferCh: make(chan *z.Buffer, 10), + mapDir: mapDir, + } + closer := z.NewCloser(1) + defer closer.SignalAndWait() + go r.Progress(closer) + + return r.Reduce() +} + +func (r *reducer) Progress(closer *z.Closer) { + defer closer.Done() + + ticker := time.NewTicker(time.Second) + defer ticker.Stop() + + start := time.Now() + update := func() { + since := time.Since(start) + read := atomic.LoadUint64(&r.bytesRead) + proc := atomic.LoadUint64(&r.bytesProcessed) + pr := uint64(float64(proc) / since.Seconds()) + glog.Infof( + "Restore REDUCE %s read: %s. processed: %s. rate: %s/sec. jemalloc: %s.\n", + x.FixedDuration(since), humanize.IBytes(read), humanize.IBytes(proc), + humanize.IBytes(pr), humanize.IBytes(uint64(z.NumAllocBytes()))) + } + for { + select { + case <-closer.HasBeenClosed(): + update() + glog.Infof("Restore REDUCE Done in %s.\n", x.FixedDuration(time.Since(start))) + return + case <-ticker.C: + update() + } + } +} + +func (r *reducer) Reduce() error { + var files []string + + var total int64 + f := func(path string, info os.FileInfo, err error) error { + if err != nil { + return err + } + if strings.HasSuffix(info.Name(), ".map") { + files = append(files, path) + total += info.Size() + } + return nil + } + + if err := filepath.Walk(r.mapDir, f); err != nil { + return err + } + glog.Infof("Got %d map files of compressed size: %s.\n", + len(files), humanize.IBytes(uint64(total))) + + // Pick up map iterators and partition keys. + partitions := make(map[string]struct{}) + for _, fname := range files { + header, itr := newMapIterator(fname) + for _, k := range header.PartitionKeys { + if len(k) == 0 { + continue + } + partitions[string(k)] = struct{}{} + } + r.mapItrs = append(r.mapItrs, itr) + } + + keys := make([][]byte, 0, len(partitions)) + for k := range partitions { + keys = append(keys, []byte(k)) + } + sort.Slice(keys, func(i, j int) bool { + return y.CompareKeys(keys[i], keys[j]) < 0 + }) + // Append nil for the last entries. + keys = append(keys, nil) + r.partitionKeys = keys + + errCh := make(chan error, 2) + go func() { + errCh <- r.blockingRead() + }() + go func() { + errCh <- r.process() + }() + + for i := 0; i < 2; i++ { + if err := <-errCh; err != nil { + return err + } + } + return nil +} + +func (r *reducer) blockingRead() error { + cbuf := z.NewBuffer(64<<20, "Restore.GetBuf") + + sortAndPush := func(buf *z.Buffer) { + // Let's sort here. So, there's less work for processor. + buf.SortSlice(func(ls, rs []byte) bool { + lme := mapEntry(ls) + rme := mapEntry(rs) + return y.CompareKeys(lme.Key(), rme.Key()) < 0 + }) + atomic.AddUint64(&r.bytesRead, uint64(buf.LenNoPadding())) + r.bufferCh <- buf + } + for _, pkey := range r.partitionKeys { + for _, itr := range r.mapItrs { + if err := itr.Next(cbuf, pkey); err != nil { + cbuf.Release() + return err + } + } + if cbuf.LenNoPadding() < 256<<20 { + // Pick up more data. + continue + } + sortAndPush(cbuf) + cbuf = z.NewBuffer(64<<20, "Restore.GetBuf") + } + + if !cbuf.IsEmpty() { + sortAndPush(cbuf) + } else { + cbuf.Release() + } + close(r.bufferCh) + return nil +} + +func (r *reducer) process() error { + if r.w == nil { + return nil + } + writer := r.w + + kvBuf := z.NewBuffer(64<<20, "Restore.GetBuf") + defer func() { + kvBuf.Release() + }() + + var lastKey []byte + for cbuf := range r.bufferCh { + err := cbuf.SliceIterate(func(s []byte) error { + me := mapEntry(s) + key := me.Key() + + // Don't need to pick multiple versions of the same key. + if y.SameKey(key, lastKey) { + return nil + } + lastKey = append(lastKey[:0], key...) + + kvBuf.WriteSlice(me.Data()) + return nil + }) + if err != nil { + return err + } + + atomic.AddUint64(&r.bytesProcessed, uint64(cbuf.LenNoPadding())) + if err := writer.Write(kvBuf); err != nil { + return err + } + kvBuf.Reset() + cbuf.Release() + } // end loop for bufferCh + return nil +} diff --git a/worker/s3_handler.go b/worker/s3_handler.go deleted file mode 100644 index 13b3aa3e4c7..00000000000 --- a/worker/s3_handler.go +++ /dev/null @@ -1,405 +0,0 @@ -// +build !oss - -/* - * Copyright 2022 Dgraph Labs, Inc. and Contributors - * - * Licensed under the Dgraph 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/dgraph-io/dgraph/blob/master/licenses/DCL.txt - */ - -package worker - -import ( - "encoding/json" - "fmt" - "io" - "net/url" - "path/filepath" - "sort" - "strings" - "time" - - "github.com/dgraph-io/dgraph/x" - "github.com/dustin/go-humanize" - - "github.com/dgraph-io/dgraph/protos/pb" - "github.com/golang/glog" - minio "github.com/minio/minio-go/v6" - "github.com/minio/minio-go/v6/pkg/credentials" - "github.com/pkg/errors" -) - -// FillRestoreCredentials fills the empty values with the default credentials so that -// a restore request is sent to all the groups with the same credentials. -func FillRestoreCredentials(location string, req *pb.RestoreRequest) error { - uri, err := url.Parse(location) - if err != nil { - return err - } - - defaultCreds := credentials.Value{ - AccessKeyID: req.AccessKey, - SecretAccessKey: req.SecretKey, - SessionToken: req.SessionToken, - } - provider := x.MinioCredentialsProvider(uri.Scheme, defaultCreds) - - creds, _ := provider.Retrieve() // Error is always nil. - - req.AccessKey = creds.AccessKeyID - req.SecretKey = creds.SecretAccessKey - req.SessionToken = creds.SessionToken - - return nil -} - -// s3Handler is used for 's3:' and 'minio:' URI schemes. -type s3Handler struct { - bucketName, objectPrefix string - pwriter *io.PipeWriter - preader *io.PipeReader - cerr chan error - creds *x.MinioCredentials - uri *url.URL - mc *x.MinioClient - numWritten int -} - -// setup creates a new session, checks valid bucket at uri.Path, and configures a minio client. -// setup also fills in values used by the handler in subsequent calls. -// Returns a new S3 minio client, otherwise a nil client with an error. -func NewS3Handler(uri *url.URL, creds *x.MinioCredentials) (*s3Handler, error) { - h := &s3Handler{ - creds: creds, - uri: uri, - } - mc, err := x.NewMinioClient(uri, creds) - if err != nil { - return nil, err - } - h.mc = mc - h.bucketName, h.objectPrefix = mc.ParseBucketAndPrefix(uri.Path) - return h, nil -} - -func (h *s3Handler) createObject(mc *x.MinioClient, objectPath string) { - - // The backup object is: folder1...folderN/dgraph.20181106.0113/r110001-g1.backup - object := filepath.Join(h.objectPrefix, objectPath) - glog.V(2).Infof("Sending data to %s blob %q ...", h.uri.Scheme, object) - - h.cerr = make(chan error, 1) - h.preader, h.pwriter = io.Pipe() - go func() { - h.cerr <- h.upload(mc, object) - }() -} - -// GetLatestManifest reads the manifests at the given URL and returns the -// latest manifest. -func (h *s3Handler) GetLatestManifest(uri *url.URL) (*Manifest, error) { - manifest, err := h.getConsolidatedManifest() - if err != nil { - errors.Wrap(err, "GetLatestManifest failed to get consolidated manifests: ") - } - if len(manifest.Manifests) == 0 { - return &Manifest{}, nil - } - return manifest.Manifests[len(manifest.Manifests)-1], nil -} - -// CreateBackupFile creates a new session and prepares the data stream for the backup. -// URI formats: -// minio:///bucket/folder1.../folderN?secure=true|false -// minio:///bucket/folder1.../folderN?secure=true|false -// s3:///bucket/folder1.../folderN?secure=true|false -// s3:///bucket/folder1.../folderN?secure=true|false (use default S3 endpoint) -func (h *s3Handler) CreateBackupFile(uri *url.URL, req *pb.BackupRequest) error { - glog.V(2).Infof("S3Handler got uri: %+v. Host: %s. Path: %s\n", uri, uri.Host, uri.Path) - - fileName := backupName(req.ReadTs, req.GroupId) - objectPath := filepath.Join(fmt.Sprintf(backupPathFmt, req.UnixTs), fileName) - h.createObject(h.mc, objectPath) - return nil -} - -// CreateManifest finishes a backup by creating an object to store the manifest. -func (h *s3Handler) CreateManifest(uri *url.URL, manifest *MasterManifest) error { - glog.V(2).Infof("S3Handler got uri: %+v. Host: %s. Path: %s\n", uri, uri.Host, uri.Path) - - // If there is already a consolidated manifest, write the manifest to a temp file, which - // will be used to replace original manifest. - objectPath := filepath.Join(h.objectPrefix, backupManifest) - if h.objectExists(objectPath) { - h.createObject(h.mc, tmpManifest) - if err := json.NewEncoder(h).Encode(manifest); err != nil { - return err - } - if err := h.flush(); err != nil { - return errors.Wrap(err, "CreateManifest failed to flush the handler") - } - - // At this point, a temporary manifest is successfully created, we need to replace the - // original manifest with this temporary manifest. - object := filepath.Join(h.objectPrefix, backupManifest) - tmpObject := filepath.Join(h.objectPrefix, tmpManifest) - src := minio.NewSourceInfo(h.bucketName, tmpObject, nil) - dst, err := minio.NewDestinationInfo(h.bucketName, object, nil, nil) - if err != nil { - return errors.Wrap(err, "CreateManifest failed to create dstInfo") - } - - // We try copying 100 times, if it still fails, the user should manually copy the - // tmpManifest to the original manifest. - err = x.RetryUntilSuccess(100, time.Second, func() error { - if err := h.mc.CopyObject(dst, src); err != nil { - return errors.Wrapf(err, "COPYING TEMPORARY MANIFEST TO MAIN MANIFEST FAILED!!!\n"+ - "It is possible that the manifest would have been corrupted. You must copy "+ - "the file: %s to: %s (present in the backup s3 bucket), in order to "+ - "fix the backup manifest.", tmpManifest, backupManifest) - } - return nil - }) - if err != nil { - return err - } - - err = h.mc.RemoveObject(h.bucketName, tmpObject) - return errors.Wrap(err, "CreateManifest failed to remove temporary manifest") - } - h.createObject(h.mc, backupManifest) - err := json.NewEncoder(h).Encode(manifest) - return errors.Wrap(err, "CreateManifest failed to create a new master manifest") -} - -// GetManifest returns the master manifest, if the directory doesn't contain -// a master manifest, then it will try to return a master manifest by consolidating -// the manifests. -func (h *s3Handler) GetManifest(uri *url.URL) (*MasterManifest, error) { - manifest, err := h.getConsolidatedManifest() - if err != nil { - return manifest, errors.Wrap(err, "GetManifest failed to get consolidated manifests: ") - } - return manifest, nil - -} - -func (h *s3Handler) GetManifests(uri *url.URL, backupId string, - backupNum uint64) ([]*Manifest, error) { - manifest, err := h.getConsolidatedManifest() - if err != nil { - return manifest.Manifests, errors.Wrap(err, "GetManifest failed to get consolidated manifests: ") - } - - var filtered []*Manifest - for _, m := range manifest.Manifests { - path := filepath.Join(uri.Path, m.Path) - if h.objectExists(path) { - filtered = append(filtered, m) - } - } - return getManifests(manifest.Manifests, backupId, backupNum) -} - -// Load creates a new session, scans for backup objects in a bucket, then tries to -// load any backup objects found. -// Returns nil and the maximum Since value on success, error otherwise. -func (h *s3Handler) Load(uri *url.URL, backupId string, backupNum uint64, fn loadFn) LoadResult { - manifests, err := h.GetManifests(uri, backupId, backupNum) - if err != nil { - return LoadResult{Err: errors.Wrapf(err, "while retrieving manifests")} - } - // since is returned with the max manifest Since value found. - var since uint64 - - // Process each manifest, first check that they are valid and then confirm the - // backup manifests for each group exist. Each group in manifest must have a backup file, - // otherwise this is a failure and the user must remedy. - var maxUid, maxNsId uint64 - for i, manifest := range manifests { - if manifest.ValidReadTs() == 0 || len(manifest.Groups) == 0 { - continue - } - - path := filepath.Join(h.objectPrefix, manifests[i].Path) - for gid := range manifest.Groups { - object := filepath.Join(path, backupName(manifest.ValidReadTs(), gid)) - reader, err := h.mc.GetObject(h.bucketName, object, minio.GetObjectOptions{}) - if err != nil { - return LoadResult{Err: errors.Wrapf(err, "Failed to get %q", object)} - } - defer reader.Close() - - st, err := reader.Stat() - if err != nil { - return LoadResult{Err: errors.Wrapf(err, "Stat failed %q", object)} - } - if st.Size <= 0 { - return LoadResult{Err: errors.Errorf("Remote object is empty or inaccessible: %s", - object)} - } - - // Only restore the predicates that were assigned to this group at the time - // of the last backup. - predSet := manifests[len(manifests)-1].getPredsInGroup(gid) - - groupMaxUid, groupMaxNsId, err := fn(gid, - &loadBackupInput{ - r: reader, - preds: predSet, - dropOperations: manifest.DropOperations, - isOld: manifest.Version == 0, - compression: manifest.Compression, - }) - if err != nil { - return LoadResult{Err: err} - } - maxUid = x.Max(maxUid, groupMaxUid) - maxNsId = x.Max(maxNsId, groupMaxNsId) - } - since = manifest.ValidReadTs() - } - - return LoadResult{Version: since, MaxLeaseUid: maxUid, MaxLeaseNsId: maxNsId} -} - -// Verify performs basic checks to decide whether the specified backup can be restored -// to a live cluster. -func (h *s3Handler) Verify(uri *url.URL, req *pb.RestoreRequest, currentGroups []uint32) error { - manifests, err := h.GetManifests(uri, req.GetBackupId(), req.GetBackupNum()) - if err != nil { - return errors.Wrapf(err, "while retrieving manifests") - } - return verifyRequest(req, manifests, currentGroups) -} - -// upload will block until it's done or an error occurs. -func (h *s3Handler) upload(mc *x.MinioClient, object string) error { - start := time.Now() - - // We don't need to have a progress object, because we're using a Pipe. A write to Pipe would - // block until it can be fully read. So, the rate of the writes here would be equal to the rate - // of upload. We're already tracking progress of the writes in stream.Lists, so no need to track - // the progress of read. By definition, it must be the same. - n, err := mc.PutObject(h.bucketName, object, h.preader, -1, minio.PutObjectOptions{}) - glog.V(2).Infof("Backup sent data of size %s. Time elapsed: %s", - humanize.IBytes(uint64(n)), time.Since(start).Round(time.Second)) - - if err != nil { - // This should cause Write to fail as well. - glog.Errorf("Backup: Closing RW pipe due to error: %v", err) - if err := h.pwriter.Close(); err != nil { - return err - } - if err := h.preader.Close(); err != nil { - return err - } - } - return err -} - -func (h *s3Handler) Close() error { - // Done buffering, send EOF. - if h.pwriter == nil { - return nil - } - return h.flush() -} - -func (h *s3Handler) flush() error { - if err := h.pwriter.CloseWithError(nil); err != nil && err != io.EOF { - glog.Errorf("Unexpected error when closing pipe: %v", err) - } - glog.V(2).Infof("Backup waiting for upload to complete.") - // We are setting this to nil, so that closing the handler after flushing is a no-op. - h.pwriter = nil - return <-h.cerr - -} - -func (h *s3Handler) Write(b []byte) (int, error) { - n, err := h.pwriter.Write(b) - h.numWritten += n - return n, err -} - -func (h *s3Handler) BytesWritten() int { - return h.numWritten -} - -func (h *s3Handler) objectExists(objectPath string) bool { - _, err := h.mc.StatObject(h.bucketName, objectPath, minio.StatObjectOptions{}) - if err != nil { - errResponse := minio.ToErrorResponse(err) - if errResponse.Code == "NoSuchKey" { - return false - } else { - glog.Errorf("Failed to verify object %s existance: %s", objectPath, errResponse.Code) - return false - } - } - return true -} - -func (h *s3Handler) readManifest(path string, m *Manifest) error { - reader, err := h.mc.GetObject(h.bucketName, path, minio.GetObjectOptions{}) - if err != nil { - return err - } - defer reader.Close() - return json.NewDecoder(reader).Decode(m) -} - -func (h *s3Handler) readMasterManifest(m *MasterManifest) error { - path := filepath.Join(h.objectPrefix, backupManifest) - reader, err := h.mc.GetObject(h.bucketName, path, minio.GetObjectOptions{}) - if err != nil { - return err - } - defer reader.Close() - return json.NewDecoder(reader).Decode(m) -} - -// getConsolidatedManifest walks over all the backup directories and generates a master manifest. -func (h *s3Handler) getConsolidatedManifest() (*MasterManifest, error) { - var manifest MasterManifest - - // If there is a master manifest already, we just return it. - objectPath := filepath.Join(h.objectPrefix, backupManifest) - if h.objectExists(objectPath) { - if err := h.readMasterManifest(&manifest); err != nil { - return nil, err - } - return &manifest, nil - } - - // Otherwise, we consolidate the manifests to make a master manifest. - var paths []string - done := make(chan struct{}) - defer close(done) - suffix := "/" + backupManifest - for object := range h.mc.ListObjects(h.bucketName, h.objectPrefix, true, done) { - if strings.HasSuffix(object.Key, suffix) { - paths = append(paths, object.Key) - } - } - - sort.Strings(paths) - var mlist []*Manifest - - for _, path := range paths { - var m Manifest - if err := h.readManifest(path, &m); err != nil { - return nil, errors.Wrap(err, "While Getting latest manifest") - } - path = filepath.Dir(path) - _, path = filepath.Split(path) - m.Path = path - mlist = append(mlist, &m) - } - manifest.Manifests = mlist - return &manifest, nil -}