From 93f9ae0e37825d357bab62928e150f0f0aa5bf26 Mon Sep 17 00:00:00 2001 From: wayblink Date: Thu, 23 May 2024 20:25:58 +0800 Subject: [PATCH] Define meta v2 (#349) --- core/backup_context.go | 53 +--- core/backup_impl_create_backup.go | 424 ++++++++++++------------- core/backup_impl_restore_backup.go | 4 +- core/backup_meta.go | 72 ----- core/backup_meta_manager.go | 462 ++++++++++++++++++++++++++++ core/proto/backup.proto | 3 +- core/proto/backuppb/backup.pb.go | 378 ++++++++++++----------- core/storage/minio_chunk_manager.go | 4 +- 8 files changed, 888 insertions(+), 512 deletions(-) create mode 100644 core/backup_meta_manager.go diff --git a/core/backup_context.go b/core/backup_context.go index b005f996..61401340 100644 --- a/core/backup_context.go +++ b/core/backup_context.go @@ -24,7 +24,6 @@ const ( BULKINSERT_SLEEP_INTERVAL = 5 BACKUP_NAME = "BACKUP_NAME" COLLECTION_RENAME_SUFFIX = "COLLECTION_RENAME_SUFFIX" - WORKER_NUM = 100 RPS = 1000 ) @@ -48,18 +47,11 @@ type BackupContext struct { milvusRootPath string backupRootPath string - backupNameIdDict sync.Map //map[string]string - backupTasks sync.Map //map[string]*backuppb.BackupInfo - - backupTasksCache sync.Map //map[string]*LeveledBackupInfo - updateMu sync.Mutex - - restoreTasks map[string]*backuppb.RestoreBackupTask + meta *MetaManager backupCollectionWorkerPool *common.WorkerPool backupCopyDataWorkerPool *common.WorkerPool - //bulkinsertWorkerPool *common.WorkerPool - bulkinsertWorkerPools map[string]*common.WorkerPool + bulkinsertWorkerPools map[string]*common.WorkerPool } func CreateMilvusClient(ctx context.Context, params paramtable.BackupParams) (gomilvus.Client, error) { @@ -97,12 +89,7 @@ func CreateStorageClient(ctx context.Context, params paramtable.BackupParams) (s } func (b *BackupContext) Start() error { - //b.backupTasks = sync.Map{} - b.backupTasksCache = sync.Map{} - b.backupNameIdDict = sync.Map{} - b.restoreTasks = make(map[string]*backuppb.RestoreBackupTask) b.started = true - b.updateMu = sync.Mutex{} log.Info(fmt.Sprintf("%+v", b.params.BackupCfg)) log.Info(fmt.Sprintf("%+v", b.params.HTTPCfg)) return nil @@ -126,6 +113,7 @@ func CreateBackupContext(ctx context.Context, params paramtable.BackupParams) *B milvusRootPath: params.MinioCfg.RootPath, backupRootPath: params.MinioCfg.BackupRootPath, bulkinsertWorkerPools: make(map[string]*common.WorkerPool), + meta: newMetaManager(), } } @@ -229,31 +217,17 @@ func (b *BackupContext) GetBackup(ctx context.Context, request *backuppb.GetBack resp.Code = backuppb.ResponseCode_Parameter_Error resp.Msg = "empty backup name and backup id, please set a backup name or id" } else if request.GetBackupId() != "" { - if value, ok := b.backupTasksCache.Load(request.GetBackupId()); ok { - backupInfo, err := levelToTree(value.(*LeveledBackupInfo)) - if err != nil { - resp.Code = backuppb.ResponseCode_Fail - resp.Msg = err.Error() - } else { - resp.Code = backuppb.ResponseCode_Success - resp.Msg = "success" - resp.Data = backupInfo - } - } + backupInfo := b.meta.GetFullMeta(request.GetBackupId()) + resp.Code = backuppb.ResponseCode_Success + resp.Msg = "success" + resp.Data = backupInfo } else if request.GetBackupName() != "" { - if id, ok := b.backupNameIdDict.Load(request.GetBackupName()); ok { + backupInfo := b.meta.GetBackupByName(request.GetBackupName()) + if backupInfo != nil { + fullBackupInfo := b.meta.GetFullMeta(backupInfo.Id) resp.Code = backuppb.ResponseCode_Success resp.Msg = "success" - backup, ok := b.backupTasksCache.Load(id) - if ok { - backupInfo, err := levelToTree(backup.(*LeveledBackupInfo)) - if err != nil { - resp.Code = backuppb.ResponseCode_Fail - resp.Msg = err.Error() - } else { - resp.Data = backupInfo - } - } + resp.Data = fullBackupInfo } else { var backupBucketName string var backupPath string @@ -529,10 +503,11 @@ func (b *BackupContext) GetRestore(ctx context.Context, request *backuppb.GetRes return resp } - if value, ok := b.restoreTasks[request.GetId()]; ok { + task := b.meta.GetRestoreTask(request.GetId()) + if task != nil { resp.Code = backuppb.ResponseCode_Success resp.Msg = "success" - resp.Data = UpdateRestoreBackupTask(value) + resp.Data = UpdateRestoreBackupTask(task) return resp } else { resp.Code = backuppb.ResponseCode_Fail diff --git a/core/backup_impl_create_backup.go b/core/backup_impl_create_backup.go index e2fa1c9d..331af44d 100644 --- a/core/backup_impl_create_backup.go +++ b/core/backup_impl_create_backup.go @@ -11,7 +11,6 @@ import ( "sort" "strconv" "strings" - "sync" "time" jsoniter "github.com/json-iterator/go" @@ -80,8 +79,6 @@ func (b *BackupContext) CreateBackup(ctx context.Context, request *backuppb.Crea return resp } - var name string = request.BackupName - milvusVersion, err := b.getMilvusClient().GetVersion(b.ctx) if err != nil { log.Error("fail to get milvus version", zap.Error(err)) @@ -94,15 +91,16 @@ func (b *BackupContext) CreateBackup(ctx context.Context, request *backuppb.Crea Id: request.GetRequestId(), StateCode: backuppb.BackupTaskStateCode_BACKUP_INITIAL, StartTime: time.Now().UnixNano() / int64(time.Millisecond), - Name: name, + Name: request.BackupName, MilvusVersion: milvusVersion, } - levelBackupInfo := NewLeveledBackupInfo(backup) - b.backupTasksCache.Store(request.GetRequestId(), levelBackupInfo) - b.backupNameIdDict.Store(name, request.GetRequestId()) + b.meta.AddBackup(backup) + //levelBackupInfo := NewLeveledBackupInfo(backup) + //b.backupTasksCache.Store(request.GetRequestId(), levelBackupInfo) + //b.backupNameIdDict.Store(name, request.GetRequestId()) if request.Async { - go b.executeCreateBackup(ctx, request, levelBackupInfo) + go b.executeCreateBackup(ctx, request, backup) asyncResp := &backuppb.BackupInfoResponse{ RequestId: request.GetRequestId(), Code: backuppb.ResponseCode_Success, @@ -111,8 +109,8 @@ func (b *BackupContext) CreateBackup(ctx context.Context, request *backuppb.Crea } return asyncResp } else { - task, err := b.executeCreateBackup(ctx, request, levelBackupInfo) - resp.Data = task + err := b.executeCreateBackup(ctx, request, backup) + resp.Data = b.meta.GetBackup(backup.GetId()) if err != nil { resp.Code = backuppb.ResponseCode_Fail resp.Msg = err.Error() @@ -214,13 +212,13 @@ func (b *BackupContext) parseBackupCollections(request *backuppb.CreateBackupReq return toBackupCollections, nil } -func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo *LeveledBackupInfo, collection collectionStruct, force bool) (*backuppb.CollectionBackupInfo, error) { +func (b *BackupContext) backupCollectionPrepare(ctx context.Context, backupInfo *backuppb.BackupInfo, collection collectionStruct, force bool) error { log.Info("start backup collection", zap.String("db", collection.db), zap.String("collection", collection.collectionName)) // list collection result is not complete completeCollection, err := b.getMilvusClient().DescribeCollection(b.ctx, collection.db, collection.collectionName) if err != nil { log.Error("fail in DescribeCollection", zap.Error(err)) - return nil, err + return err } fields := make([]*backuppb.FieldSchema, 0) for _, field := range completeCollection.Schema.Fields { @@ -265,7 +263,7 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * continue } else { log.Error("fail in DescribeIndex", zap.Error(err)) - return nil, err + return err } } log.Info("field index", @@ -289,7 +287,7 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * } collectionBackup := &backuppb.CollectionBackupInfo{ - Id: utils.UUID(), + Id: backupInfo.Id, StateCode: backuppb.BackupTaskStateCode_BACKUP_INITIAL, StartTime: time.Now().Unix(), CollectionId: completeCollection.ID, @@ -301,19 +299,21 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * HasIndex: len(indexInfos) > 0, IndexInfos: indexInfos, } - b.updateCollection(levelInfo, collectionBackup) + b.meta.AddCollection(collectionBackup) + + //b.updateCollection(levelInfo, collectionBackup) partitionBackupInfos := make([]*backuppb.PartitionBackupInfo, 0) partitions, err := b.getMilvusClient().ShowPartitions(b.ctx, collectionBackup.GetDbName(), collectionBackup.GetCollectionName()) if err != nil { log.Error("fail to ShowPartitions", zap.Error(err)) - return nil, err + return err } // use GetLoadingProgress currently, GetLoadState is a new interface @20230104 milvus pr#21515 collectionLoadProgress, err := b.getMilvusClient().GetLoadingProgress(ctx, collectionBackup.GetDbName(), collectionBackup.GetCollectionName(), []string{}) if err != nil { log.Error("fail to GetLoadingProgress of collection", zap.Error(err)) - return nil, err + return err } var collectionLoadState string @@ -334,7 +334,7 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * loadProgress, err := b.getMilvusClient().GetLoadingProgress(ctx, collectionBackup.GetDbName(), collectionBackup.GetCollectionName(), []string{partition.Name}) if err != nil { log.Error("fail to GetLoadingProgress of partition", zap.Error(err)) - return nil, err + return err } if loadProgress == 0 { partitionLoadStates[partition.Name] = LoadState_NotLoad @@ -352,7 +352,7 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * // Flush segmentEntitiesBeforeFlush, err := b.getMilvusClient().GetPersistentSegmentInfo(ctx, collectionBackup.GetDbName(), collectionBackup.GetCollectionName()) if err != nil { - return nil, err + return err } log.Info("GetPersistentSegmentInfo before flush from milvus", zap.String("databaseName", collectionBackup.GetDbName()), @@ -364,11 +364,10 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * zap.String("databaseName", collectionBackup.GetDbName()), zap.String("collectionName", collectionBackup.GetCollectionName()), zap.Error(err)) - return nil, err + return err } //collectionBackup.BackupTimestamp = utils.ComposeTS(timeOfSeal, 0) - collectionBackup.BackupPhysicalTimestamp = uint64(timeOfSeal) channelCheckpoints := make(map[string]string, 0) var maxChannelBackupTimeStamp uint64 = 0 for vch, checkpoint := range channelCPs { @@ -379,8 +378,13 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * maxChannelBackupTimeStamp = checkpoint.GetTimestamp() } } - collectionBackup.ChannelCheckpoints = channelCheckpoints - collectionBackup.BackupTimestamp = maxChannelBackupTimeStamp + //collectionBackup.ChannelCheckpoints = channelCheckpoints + //collectionBackup.BackupTimestamp = maxChannelBackupTimeStamp + //collectionBackup.BackupPhysicalTimestamp = uint64(timeOfSeal) + b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, + setCollectionChannelCheckpoints(channelCheckpoints), + setCollectionBackupTimestamp(maxChannelBackupTimeStamp), + setCollectionBackupPhysicalTimestamp(uint64(timeOfSeal))) log.Info("flush segments", zap.String("databaseName", collectionBackup.GetDbName()), zap.String("collectionName", collectionBackup.GetCollectionName()), @@ -393,7 +397,7 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * flushSegmentIDs := append(newSealedSegmentIDs, flushedSegmentIDs...) segmentEntitiesAfterFlush, err := b.getMilvusClient().GetPersistentSegmentInfo(ctx, collectionBackup.GetDbName(), collectionBackup.GetCollectionName()) if err != nil { - return nil, err + return err } segmentIDsEntitiesBeforeFlush := lo.Map(segmentEntitiesBeforeFlush, func(segment *entity.Segment, _ int) int64 { return segment.ID }) @@ -434,7 +438,7 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * segmentEntitiesBeforeFlush, err := b.getMilvusClient().GetPersistentSegmentInfo(ctx, collectionBackup.GetDbName(), collectionBackup.GetCollectionName()) if err != nil { log.Error(fmt.Sprintf("fail to flush the collection: %s", collectionBackup.GetCollectionName()), zap.Error(err)) - return nil, err + return err } log.Info("GetPersistentSegmentInfo from milvus", zap.String("databaseName", collectionBackup.GetDbName()), @@ -446,9 +450,7 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * } if err != nil { - collectionBackup.StateCode = backuppb.BackupTaskStateCode_BACKUP_FAIL - collectionBackup.ErrorMessage = err.Error() - return nil, err + return err } newSegIDs := lo.Map(unfilledSegments, func(segment *entity.Segment, _ int) int64 { return segment.ID }) @@ -466,7 +468,8 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * PartitionId: segment.ParititionID, NumOfRows: segment.NumRows, } - b.updateSegment(levelInfo, segmentInfo) + b.meta.AddSegment(segmentInfo) + //b.updateSegment(levelInfo, segmentInfo) partSegInfoMap[segment.ParititionID] = append(partSegInfoMap[segment.ParititionID], segmentInfo) } @@ -484,13 +487,15 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * Size: size, LoadState: partitionLoadStates[partition.Name], } - b.updatePartition(levelInfo, partitionBackupInfo) + b.meta.AddPartition(partitionBackupInfo) + //b.updatePartition(levelInfo, partitionBackupInfo) //partitionBackupInfos = append(partitionBackupInfos, partitionBackupInfo) } - collectionBackup.PartitionBackups = partitionBackupInfos - collectionBackup.LoadState = collectionLoadState - b.updateCollection(levelInfo, collectionBackup) + //collectionBackup.PartitionBackups = partitionBackupInfos + //collectionBackup.LoadState = collectionLoadState + b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setCollectionLoadState(collectionLoadState)) + //b.updateCollection(levelInfo, collectionBackup) log.Info("finish build partition info", zap.String("collectionName", collectionBackup.GetCollectionName()), @@ -501,17 +506,18 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, levelInfo * collectionBackupSize += part.GetSize() } - collectionBackup.Size = collectionBackupSize - b.updateCollection(levelInfo, collectionBackup) - return collectionBackup, nil + b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setCollectionSize(collectionBackupSize)) + //collectionBackup.Size = collectionBackupSize + //b.updateCollection(levelInfo, collectionBackup) + return nil } -func (b *BackupContext) backupCollectionExecute(ctx context.Context, levelInfo *LeveledBackupInfo, collectionBackup *backuppb.CollectionBackupInfo) error { +func (b *BackupContext) backupCollectionExecute(ctx context.Context, collectionBackup *backuppb.CollectionBackupInfo) error { log.Info("backupCollectionExecute", zap.Any("collectionMeta", collectionBackup.String())) var segmentBackupInfos []*backuppb.SegmentBackupInfo - for _, seg := range levelInfo.segmentLevel.Infos { - if seg.CollectionId == collectionBackup.GetCollectionId() { - segmentBackupInfos = append(segmentBackupInfos, seg) + for _, partition := range b.meta.GetPartitions(collectionBackup.CollectionId) { + for _, segment := range b.meta.GetSegments(partition.GetPartitionId()) { + segmentBackupInfos = append(segmentBackupInfos, segment) } } log.Info("Begin copy data", @@ -522,13 +528,13 @@ func (b *BackupContext) backupCollectionExecute(ctx context.Context, levelInfo * sort.SliceStable(segmentBackupInfos, func(i, j int) bool { return segmentBackupInfos[i].Size < segmentBackupInfos[j].Size }) - err := b.copySegments(ctx, segmentBackupInfos, levelInfo) + err := b.copySegments(ctx, segmentBackupInfos) if err != nil { return err } - collectionBackup.EndTime = time.Now().Unix() - b.updateCollection(levelInfo, collectionBackup) + //collectionBackup.EndTime = time.Now().Unix() + b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setCollectionEndTime(time.Now().Unix())) log.Info("Finish copy data", zap.String("dbName", collectionBackup.GetDbName()), @@ -577,7 +583,7 @@ func (b *BackupContext) resumeMilvusGC(ctx context.Context, gcAddress string) { log.Info("Resume Milvus GC response", zap.String("response", string(body)), zap.String("address", gcAddress)) } -func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backuppb.CreateBackupRequest, levelInfo *LeveledBackupInfo) (*backuppb.BackupInfo, error) { +func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backuppb.CreateBackupRequest, backupInfo *backuppb.BackupInfo) error { b.mu.Lock() defer b.mu.Unlock() @@ -599,19 +605,18 @@ func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backup defer b.resumeMilvusGC(ctx, gcAddress) } - backupInfo := levelInfo.backupLevel - backupInfo.BackupTimestamp = uint64(time.Now().UnixNano() / int64(time.Millisecond)) - backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_EXECUTING - - defer b.updateBackup(levelInfo, backupInfo) + // not important + //backupInfo.BackupTimestamp = uint64(time.Now().UnixNano() / int64(time.Millisecond)) + //backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_EXECUTING + //b.meta.UpdateBackup(backupInfo.Id, setStateCode(backuppb.BackupTaskStateCode_BACKUP_EXECUTING)) + //defer b.updateBackup(levelInfo, backupInfo) // 1, get collection level meta toBackupCollections, err := b.parseBackupCollections(request) if err != nil { log.Error("parse backup collections from request failed", zap.Error(err)) - backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_FAIL - backupInfo.ErrorMessage = err.Error() - return backupInfo, err + b.meta.UpdateBackup(backupInfo.Id, setStateCode(backuppb.BackupTaskStateCode_BACKUP_FAIL), setErrorMessage(err.Error())) + return err } collectionNames := make([]string, len(toBackupCollections)) for i, coll := range toBackupCollections { @@ -620,19 +625,19 @@ func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backup log.Info("collections to backup", zap.Strings("collections", collectionNames)) jobIds := make([]int64, 0) - toBackupCollectionInfos := make([]*backuppb.CollectionBackupInfo, 0) - toBackupCollectionInfosMutex := sync.Mutex{} + //toBackupCollectionInfos := make([]*backuppb.CollectionBackupInfo, 0) + //toBackupCollectionInfosMutex := sync.Mutex{} for _, collection := range toBackupCollections { collectionClone := collection job := func(ctx context.Context) error { err := retry.Do(ctx, func() error { - coll, err := b.backupCollectionPrepare(ctx, levelInfo, collectionClone, request.GetForce()) + err := b.backupCollectionPrepare(ctx, backupInfo, collectionClone, request.GetForce()) if err != nil { return err } - toBackupCollectionInfosMutex.Lock() - defer toBackupCollectionInfosMutex.Unlock() - toBackupCollectionInfos = append(toBackupCollectionInfos, coll) + //toBackupCollectionInfosMutex.Lock() + //defer toBackupCollectionInfosMutex.Unlock() + //toBackupCollectionInfos = append(toBackupCollectionInfos, coll) return nil }, retry.Sleep(120*time.Second), retry.Attempts(128)) return err @@ -641,20 +646,20 @@ func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backup jobIds = append(jobIds, jobId) } err = b.getBackupCollectionWorkerPool().WaitJobs(jobIds) - if err != nil { - backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_FAIL - backupInfo.ErrorMessage = err.Error() - return backupInfo, err + //backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_FAIL + //backupInfo.ErrorMessage = err.Error() + b.meta.UpdateBackup(backupInfo.Id, setStateCode(backuppb.BackupTaskStateCode_BACKUP_FAIL), setErrorMessage(err.Error())) + return err } - log.Info("Finish flush all collections") + log.Info("Finish prepare all collections meta") if !request.GetMetaOnly() { - for _, collection := range toBackupCollectionInfos { + for collectionID, collection := range b.meta.GetCollections(backupInfo.GetId()) { collectionClone := collection - log.Info("before backupCollectionExecute", zap.String("collection", collection.CollectionName)) + log.Info("before backupCollectionExecute", zap.Int64("collectionID", collectionID), zap.String("collection", collection.CollectionName)) job := func(ctx context.Context) error { - err := b.backupCollectionExecute(ctx, levelInfo, collectionClone) + err := b.backupCollectionExecute(ctx, collectionClone) return err } jobId := b.getBackupCollectionWorkerPool().SubmitWithId(job) @@ -663,41 +668,38 @@ func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backup err = b.getBackupCollectionWorkerPool().WaitJobs(jobIds) if err != nil { - backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_FAIL - backupInfo.ErrorMessage = err.Error() - return backupInfo, err + b.meta.UpdateBackup(backupInfo.Id, setStateCode(backuppb.BackupTaskStateCode_BACKUP_FAIL), setErrorMessage(err.Error())) + return err } - _, err := treeToLevel(backupInfo) - if err != nil { - backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_FAIL - backupInfo.ErrorMessage = err.Error() - return backupInfo, err - } - backupInfo.EndTime = time.Now().UnixNano() / int64(time.Millisecond) - backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_SUCCESS + //_, err := treeToLevel(backupInfo) + //if err != nil { + // b.meta.UpdateBackup(backupInfo.Id, setStateCode(backuppb.BackupTaskStateCode_BACKUP_FAIL), setErrorMessage(err.Error())) + // return err + //} } else { log.Info("skip copy data because it is a metaOnly backup request") } - b.updateBackup(levelInfo, backupInfo) + backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_SUCCESS + b.meta.UpdateBackup(backupInfo.Id, setStateCode(backuppb.BackupTaskStateCode_BACKUP_SUCCESS), setEndTime(time.Now().UnixNano()/int64(time.Millisecond))) // 7, write meta data - err = b.writeBackupInfoMeta(ctx, levelInfo, backupInfo.GetName()) + err = b.writeBackupInfoMeta(ctx, backupInfo.GetId()) if err != nil { backupInfo.StateCode = backuppb.BackupTaskStateCode_BACKUP_FAIL backupInfo.ErrorMessage = err.Error() - return backupInfo, err + return err } log.Info("finish executeCreateBackup", zap.String("requestId", request.GetRequestId()), zap.String("backupName", request.GetBackupName()), zap.Strings("collections", request.GetCollectionNames()), zap.Bool("async", request.GetAsync())) - return backupInfo, nil + return nil } -func (b *BackupContext) writeBackupInfoMeta(ctx context.Context, levelBackupInfo *LeveledBackupInfo, path string) error { - backupInfo, _ := levelToTree(levelBackupInfo) +func (b *BackupContext) writeBackupInfoMeta(ctx context.Context, id string) error { + backupInfo := b.meta.GetFullMeta(id) log.Info("Final backupInfo", zap.String("backupInfo", backupInfo.String())) output, _ := serialize(backupInfo) log.Debug("backup meta", zap.String("value", string(output.BackupMetaBytes))) @@ -724,23 +726,49 @@ func (b *BackupContext) writeBackupInfoMeta(ctx context.Context, levelBackupInfo } log.Debug("channel cp meta", zap.String("value", string(channelCPsBytes))) - b.getStorageClient().Write(ctx, b.backupBucketName, BackupMetaPath(b.backupRootPath, path), output.BackupMetaBytes) - b.getStorageClient().Write(ctx, b.backupBucketName, CollectionMetaPath(b.backupRootPath, path), output.CollectionMetaBytes) - b.getStorageClient().Write(ctx, b.backupBucketName, PartitionMetaPath(b.backupRootPath, path), output.PartitionMetaBytes) - b.getStorageClient().Write(ctx, b.backupBucketName, SegmentMetaPath(b.backupRootPath, path), output.SegmentMetaBytes) - b.getStorageClient().Write(ctx, b.backupBucketName, FullMetaPath(b.backupRootPath, path), output.FullMetaBytes) + b.getStorageClient().Write(ctx, b.backupBucketName, BackupMetaPath(b.backupRootPath, backupInfo.GetName()), output.BackupMetaBytes) + b.getStorageClient().Write(ctx, b.backupBucketName, CollectionMetaPath(b.backupRootPath, backupInfo.GetName()), output.CollectionMetaBytes) + b.getStorageClient().Write(ctx, b.backupBucketName, PartitionMetaPath(b.backupRootPath, backupInfo.GetName()), output.PartitionMetaBytes) + b.getStorageClient().Write(ctx, b.backupBucketName, SegmentMetaPath(b.backupRootPath, backupInfo.GetName()), output.SegmentMetaBytes) + b.getStorageClient().Write(ctx, b.backupBucketName, FullMetaPath(b.backupRootPath, backupInfo.GetName()), output.FullMetaBytes) b.getStorageClient().Write(ctx, b.backupBucketName, ChannelCPMetaPath(b.backupRootPath, backupInfo.GetName()), channelCPsBytes) log.Info("finish writeBackupInfoMeta", - zap.String("path", path), + zap.String("path", BackupDirPath(b.backupRootPath, backupInfo.GetName())), zap.String("backupName", backupInfo.GetName()), zap.String("backup meta", string(output.BackupMetaBytes))) return nil } -func (b *BackupContext) copySegments(ctx context.Context, segments []*backuppb.SegmentBackupInfo, levelInfo *LeveledBackupInfo) error { - dstPath := BackupBinlogDirPath(b.backupRootPath, levelInfo.backupLevel.GetName()) +func (b *BackupContext) copySegments(ctx context.Context, segments []*backuppb.SegmentBackupInfo) error { + jobIds := make([]int64, 0) + for _, v := range segments { + segment := v + log := log.With(zap.Int64("collection_id", segment.GetCollectionId()), + zap.Int64("partition_id", segment.GetPartitionId()), + zap.Int64("segment_id", segment.GetSegmentId()), + zap.Int64("group_id", segment.GetGroupId())) + log.Debug("copy segment") + err := b.fillSegmentBackupInfo(ctx, segment) + if err != nil { + log.Error("Fail to fill segment backup info", zap.Error(err)) + return err + } + job := func(ctx context.Context) error { + return b.copySegment(ctx, segment.GetSegmentId()) + } + jobId := b.getCopyDataWorkerPool().SubmitWithId(job) + jobIds = append(jobIds, jobId) + } + err := b.getCopyDataWorkerPool().WaitJobs(jobIds) + return err +} + +func (b *BackupContext) copySegment(ctx context.Context, segmentID int64) error { + segment := b.meta.GetSegment(segmentID) + backupInfo := b.meta.GetBackupBySegmentID(segmentID) + dstPath := BackupBinlogDirPath(b.backupRootPath, backupInfo.GetName()) // generate target path // milvus_rootpath/insert_log/collection_id/partition_id/segment_id/ => // backup_rootpath/backup_name/binlog/insert_log/collection_id/partition_id/group_id/segment_id @@ -751,128 +779,102 @@ func (b *BackupContext) copySegments(ctx context.Context, segments []*backuppb.S return strings.Replace(binlogPath, rootPath, dstPath, 1) } } + // insert log + for _, binlogs := range segment.GetBinlogs() { + for _, binlog := range binlogs.GetBinlogs() { + targetPath := backupPathFunc(binlog.GetLogPath(), b.milvusRootPath, dstPath) + // use segmentID as group id + segment.GroupId = segment.SegmentId + if segment.GetGroupId() != 0 { + targetPath = strings.Replace(targetPath, + strconv.FormatInt(segment.GetPartitionId(), 10), + strconv.FormatInt(segment.GetPartitionId(), 10)+"/"+strconv.FormatInt(segment.GetGroupId(), 10), + 1) + } + if targetPath == binlog.GetLogPath() { + return errors.New(fmt.Sprintf("copy src path and dst path can not be the same, src: %s dst: %s", binlog.GetLogPath(), targetPath)) + } - jobIds := make([]int64, 0) - for _, segment := range segments { - log := log.With(zap.Int64("collection_id", segment.GetCollectionId()), - zap.Int64("partition_id", segment.GetPartitionId()), - zap.Int64("segment_id", segment.GetSegmentId()), - zap.Int64("group_id", segment.GetGroupId())) - log.Debug("copy segment") - _, err := b.fillSegmentBackupInfo(ctx, levelInfo, segment) - if err != nil { - log.Error("Fail to fill segment backup info", zap.Error(err)) - return err - } - // insert log - for _, binlogs := range segment.GetBinlogs() { - for _, binlog := range binlogs.GetBinlogs() { - targetPath := backupPathFunc(binlog.GetLogPath(), b.milvusRootPath, dstPath) - // use segmentID as group id - segment.GroupId = segment.SegmentId - if segment.GetGroupId() != 0 { - targetPath = strings.Replace(targetPath, - strconv.FormatInt(segment.GetPartitionId(), 10), - strconv.FormatInt(segment.GetPartitionId(), 10)+"/"+strconv.FormatInt(segment.GetGroupId(), 10), - 1) - } - if targetPath == binlog.GetLogPath() { - return errors.New(fmt.Sprintf("copy src path and dst path can not be the same, src: %s dst: %s", binlog.GetLogPath(), targetPath)) - } + //binlog := binlog + exist, err := b.getStorageClient().Exist(ctx, b.milvusBucketName, binlog.GetLogPath()) + if err != nil { + log.Info("Fail to check file exist", + zap.Error(err), + zap.String("file", binlog.GetLogPath())) + return err + } + if !exist { + log.Error("Binlog file not exist", + zap.Error(err), + zap.String("file", binlog.GetLogPath())) + return err + } - binlog := binlog - job := func(ctx context.Context) error { - exist, err := b.getStorageClient().Exist(ctx, b.milvusBucketName, binlog.GetLogPath()) - if err != nil { - log.Info("Fail to check file exist", - zap.Error(err), - zap.String("file", binlog.GetLogPath())) - return err - } - if !exist { - log.Error("Binlog file not exist", - zap.Error(err), - zap.String("file", binlog.GetLogPath())) - return err - } - - err = retry.Do(ctx, func() error { - return b.getStorageClient().Copy(ctx, b.milvusBucketName, b.backupBucketName, binlog.GetLogPath(), targetPath) - }, retry.Sleep(2*time.Second), retry.Attempts(5)) - if err != nil { - log.Info("Fail to copy file after retry", - zap.Error(err), - zap.String("from", binlog.GetLogPath()), - zap.String("to", targetPath)) - return err - } else { - log.Debug("Successfully copy file", - zap.String("from", binlog.GetLogPath()), - zap.String("to", targetPath)) - } - - return nil - } - jobId := b.getCopyDataWorkerPool().SubmitWithId(job) - jobIds = append(jobIds, jobId) + err = retry.Do(ctx, func() error { + return b.getStorageClient().Copy(ctx, b.milvusBucketName, b.backupBucketName, binlog.GetLogPath(), targetPath) + }, retry.Sleep(2*time.Second), retry.Attempts(5)) + if err != nil { + log.Info("Fail to copy file after retry", + zap.Error(err), + zap.String("from", binlog.GetLogPath()), + zap.String("to", targetPath)) + return err + } else { + log.Debug("Successfully copy file", + zap.String("from", binlog.GetLogPath()), + zap.String("to", targetPath)) } } - // delta log - for _, binlogs := range segment.GetDeltalogs() { - for _, binlog := range binlogs.GetBinlogs() { - targetPath := backupPathFunc(binlog.GetLogPath(), b.milvusRootPath, dstPath) - if segment.GetGroupId() != 0 { - targetPath = strings.Replace(targetPath, - strconv.FormatInt(segment.GetPartitionId(), 10), - strconv.FormatInt(segment.GetPartitionId(), 10)+"/"+strconv.FormatInt(segment.GetGroupId(), 10), - 1) - } - if targetPath == binlog.GetLogPath() { - return errors.New(fmt.Sprintf("copy src path and dst path can not be the same, src: %s dst: %s", binlog.GetLogPath(), targetPath)) - } + } + // delta log + for _, binlogs := range segment.GetDeltalogs() { + for _, binlog := range binlogs.GetBinlogs() { + targetPath := backupPathFunc(binlog.GetLogPath(), b.milvusRootPath, dstPath) + if segment.GetGroupId() != 0 { + targetPath = strings.Replace(targetPath, + strconv.FormatInt(segment.GetPartitionId(), 10), + strconv.FormatInt(segment.GetPartitionId(), 10)+"/"+strconv.FormatInt(segment.GetGroupId(), 10), + 1) + } + if targetPath == binlog.GetLogPath() { + return errors.New(fmt.Sprintf("copy src path and dst path can not be the same, src: %s dst: %s", binlog.GetLogPath(), targetPath)) + } - binlog := binlog - job := func(ctx context.Context) error { - exist, err := b.getStorageClient().Exist(ctx, b.milvusBucketName, binlog.GetLogPath()) - if err != nil { - log.Info("Fail to check file exist", - zap.Error(err), - zap.String("file", binlog.GetLogPath())) - return err - } - if !exist { - log.Error("Binlog file not exist", - zap.Error(err), - zap.String("file", binlog.GetLogPath())) - return errors.New("Binlog file not exist " + binlog.GetLogPath()) - } - err = retry.Do(ctx, func() error { - return b.getStorageClient().Copy(ctx, b.milvusBucketName, b.backupBucketName, binlog.GetLogPath(), targetPath) - }, retry.Sleep(2*time.Second), retry.Attempts(5)) - if err != nil { - log.Info("Fail to copy file after retry", - zap.Error(err), - zap.String("from", binlog.GetLogPath()), - zap.String("to", targetPath)) - return err - } else { - log.Debug("Successfully copy file", - zap.String("from", binlog.GetLogPath()), - zap.String("to", targetPath)) - } - return err - } - jobId := b.getCopyDataWorkerPool().SubmitWithId(job) - jobIds = append(jobIds, jobId) + //binlog := binlog + exist, err := b.getStorageClient().Exist(ctx, b.milvusBucketName, binlog.GetLogPath()) + if err != nil { + log.Info("Fail to check file exist", + zap.Error(err), + zap.String("file", binlog.GetLogPath())) + return err + } + if !exist { + log.Error("Binlog file not exist", + zap.Error(err), + zap.String("file", binlog.GetLogPath())) + return errors.New("Binlog file not exist " + binlog.GetLogPath()) + } + err = retry.Do(ctx, func() error { + return b.getStorageClient().Copy(ctx, b.milvusBucketName, b.backupBucketName, binlog.GetLogPath(), targetPath) + }, retry.Sleep(2*time.Second), retry.Attempts(5)) + if err != nil { + log.Info("Fail to copy file after retry", + zap.Error(err), + zap.String("from", binlog.GetLogPath()), + zap.String("to", targetPath)) + return err + } else { + log.Debug("Successfully copy file", + zap.String("from", binlog.GetLogPath()), + zap.String("to", targetPath)) } } } - - err := b.getCopyDataWorkerPool().WaitJobs(jobIds) - return err + segment.Backuped = true + return nil } -func (b *BackupContext) fillSegmentBackupInfo(ctx context.Context, levelInfo *LeveledBackupInfo, segmentBackupInfo *backuppb.SegmentBackupInfo) (*backuppb.SegmentBackupInfo, error) { +func (b *BackupContext) fillSegmentBackupInfo(ctx context.Context, segmentBackupInfo *backuppb.SegmentBackupInfo) error { var size int64 = 0 var rootPath string @@ -887,11 +889,11 @@ func (b *BackupContext) fillSegmentBackupInfo(ctx context.Context, levelInfo *Le fieldsLogDir, _, err := b.getStorageClient().ListWithPrefix(ctx, b.milvusBucketName, insertPath, false) if len(fieldsLogDir) == 0 { msg := fmt.Sprintf("Get empty input path, but segment should not be empty, %s", insertPath) - return segmentBackupInfo, errors.New(msg) + return errors.New(msg) } if err != nil { log.Error("Fail to list segment path", zap.String("insertPath", insertPath), zap.Error(err)) - return segmentBackupInfo, err + return err } log.Debug("fieldsLogDir", zap.String("bucket", b.milvusBucketName), zap.Any("fieldsLogDir", fieldsLogDir)) insertLogs := make([]*backuppb.FieldBinlog, 0) @@ -959,10 +961,10 @@ func (b *BackupContext) fillSegmentBackupInfo(ctx context.Context, levelInfo *Le // }) //} - segmentBackupInfo.Binlogs = insertLogs - segmentBackupInfo.Deltalogs = deltaLogs - segmentBackupInfo.Size = size - b.updateSegment(levelInfo, segmentBackupInfo) - log.Debug("fill segment info", zap.Int64("segId", segmentBackupInfo.GetSegmentId()), zap.Int64("size", segmentBackupInfo.GetSize())) - return segmentBackupInfo, nil + //segmentBackupInfo.Binlogs = insertLogs + //segmentBackupInfo.Deltalogs = deltaLogs + //segmentBackupInfo.Size = size + b.meta.UpdateSegment(segmentBackupInfo.GetPartitionId(), segmentBackupInfo.GetSegmentId(), setSegmentBinlogs(insertLogs), setSegmentDeltaBinlogs(deltaLogs), setSegmentSize(size)) + log.Debug("fill segment info", zap.Int64("segId", segmentBackupInfo.GetSegmentId()), zap.Int64("size", size)) + return nil } diff --git a/core/backup_impl_restore_backup.go b/core/backup_impl_restore_backup.go index 8e3c8cd9..4fff4084 100644 --- a/core/backup_impl_restore_backup.go +++ b/core/backup_impl_restore_backup.go @@ -345,7 +345,7 @@ func (b *BackupContext) executeRestoreBackupTask(ctx context.Context, backupBuck log.Info("Start collection level restore pool", zap.Int("parallelism", b.params.BackupCfg.RestoreParallelism)) id := task.GetId() - b.restoreTasks[id] = task + b.meta.AddRestoreTask(task) task.StateCode = backuppb.RestoreTaskStateCode_EXECUTING log.Info("executeRestoreBackupTask start", @@ -353,7 +353,7 @@ func (b *BackupContext) executeRestoreBackupTask(ctx context.Context, backupBuck zap.String("backupBucketName", backupBucketName), zap.String("backupPath", backupPath)) updateRestoreTaskFunc := func(id string, task *backuppb.RestoreBackupTask) { - b.restoreTasks[id] = task + b.meta.AddRestoreTask(task) } defer updateRestoreTaskFunc(id, task) diff --git a/core/backup_meta.go b/core/backup_meta.go index 3f9084f0..802319ac 100644 --- a/core/backup_meta.go +++ b/core/backup_meta.go @@ -46,21 +46,6 @@ type LeveledBackupInfo struct { segmentLevel *backuppb.SegmentLevelBackupInfo } -func NewLeveledBackupInfo(backupInfo *backuppb.BackupInfo) *LeveledBackupInfo { - return &LeveledBackupInfo{ - backupLevel: backupInfo, - collectionLevel: &backuppb.CollectionLevelBackupInfo{ - Infos: make([]*backuppb.CollectionBackupInfo, 0), - }, - partitionLevel: &backuppb.PartitionLevelBackupInfo{ - Infos: make([]*backuppb.PartitionBackupInfo, 0), - }, - segmentLevel: &backuppb.SegmentLevelBackupInfo{ - Infos: make([]*backuppb.SegmentBackupInfo, 0), - }, - } -} - // treeToLevel parse BackupInfo into backup-collection-partition-segment 4-level structure func treeToLevel(backup *backuppb.BackupInfo) (LeveledBackupInfo, error) { collections := make([]*backuppb.CollectionBackupInfo, 0) @@ -394,60 +379,3 @@ func UpdateRestoreBackupTask(input *backuppb.RestoreBackupTask) *backuppb.Restor } type DbCollections = map[string][]string - -func (b *BackupContext) updateBackup(leveledBackupInfo *LeveledBackupInfo, backupInfo *backuppb.BackupInfo) { - b.updateMu.Lock() - defer b.updateMu.Unlock() - leveledBackupInfo.backupLevel = backupInfo -} - -func (b *BackupContext) updateCollection(leveledBackupInfo *LeveledBackupInfo, collection *backuppb.CollectionBackupInfo) { - b.updateMu.Lock() - defer b.updateMu.Unlock() - - newCollectionLevel := make([]*backuppb.CollectionBackupInfo, 0) - newCollectionLevel = append(newCollectionLevel, collection) - for _, collectionInfo := range leveledBackupInfo.collectionLevel.Infos { - if collectionInfo.GetDbName() == collection.GetDbName() && collectionInfo.GetCollectionName() == collection.GetCollectionName() { - //newCollectionLevel = append(newCollectionLevel, collection) - } else { - newCollectionLevel = append(newCollectionLevel, collectionInfo) - } - } - leveledBackupInfo.collectionLevel = &backuppb.CollectionLevelBackupInfo{ - Infos: newCollectionLevel, - } -} - -func (b *BackupContext) updatePartition(leveledBackupInfo *LeveledBackupInfo, partition *backuppb.PartitionBackupInfo) { - b.updateMu.Lock() - defer b.updateMu.Unlock() - - newPartitionLevel := make([]*backuppb.PartitionBackupInfo, 0) - newPartitionLevel = append(newPartitionLevel, partition) - for _, partitionInfo := range leveledBackupInfo.partitionLevel.Infos { - if partitionInfo.GetPartitionId() != partition.GetPartitionId() { - newPartitionLevel = append(newPartitionLevel, partitionInfo) - } - } - - leveledBackupInfo.partitionLevel = &backuppb.PartitionLevelBackupInfo{ - Infos: newPartitionLevel, - } -} - -func (b *BackupContext) updateSegment(leveledBackupInfo *LeveledBackupInfo, segment *backuppb.SegmentBackupInfo) { - b.updateMu.Lock() - defer b.updateMu.Unlock() - - newSegmentLevel := make([]*backuppb.SegmentBackupInfo, 0) - newSegmentLevel = append(newSegmentLevel, segment) - for _, segmentInfo := range leveledBackupInfo.segmentLevel.Infos { - if segmentInfo.GetSegmentId() != segment.GetSegmentId() { - newSegmentLevel = append(newSegmentLevel, segmentInfo) - } - } - leveledBackupInfo.segmentLevel = &backuppb.SegmentLevelBackupInfo{ - Infos: newSegmentLevel, - } -} diff --git a/core/backup_meta_manager.go b/core/backup_meta_manager.go new file mode 100644 index 00000000..7e1c8b17 --- /dev/null +++ b/core/backup_meta_manager.go @@ -0,0 +1,462 @@ +package core + +import ( + "sync" + + "github.com/golang/protobuf/proto" + + "github.com/zilliztech/milvus-backup/core/proto/backuppb" +) + +type MetaManager struct { + backups map[string]*backuppb.BackupInfo // backupId -> BackupInfo + collections map[string]map[int64]*backuppb.CollectionBackupInfo // backupId -> collectionID -> collection + partitions map[int64]map[int64]*backuppb.PartitionBackupInfo // collectionID -> partitionID -> partition + segments map[int64]map[int64]*backuppb.SegmentBackupInfo // partitionID -> segmentID -> segment + segmentPartitionReverse map[int64]int64 // segmentID -> partitionID + partitionCollectionReverse map[int64]int64 // partitionID -> collectionID + collectionBackupReverse map[int64]string // collectionID -> backupId + backupNameToIdDict map[string]string + restoreTasks map[string]*backuppb.RestoreBackupTask + mu sync.Mutex +} + +func newMetaManager() *MetaManager { + return &MetaManager{ + backups: make(map[string]*backuppb.BackupInfo, 0), + collections: make(map[string]map[int64]*backuppb.CollectionBackupInfo, 0), + partitions: make(map[int64]map[int64]*backuppb.PartitionBackupInfo, 0), + segments: make(map[int64]map[int64]*backuppb.SegmentBackupInfo, 0), + segmentPartitionReverse: make(map[int64]int64, 0), + partitionCollectionReverse: make(map[int64]int64, 0), + collectionBackupReverse: make(map[int64]string, 0), + backupNameToIdDict: make(map[string]string, 0), + restoreTasks: make(map[string]*backuppb.RestoreBackupTask, 0), + mu: sync.Mutex{}, + } +} + +func (meta *MetaManager) GetBackup(id string) *backuppb.BackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + backup, exist := meta.backups[id] + if !exist { + return nil + } + return backup +} + +func (meta *MetaManager) GetBackupByName(name string) *backuppb.BackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + id, exist := meta.backupNameToIdDict[name] + if !exist { + return nil + } + backup, exist := meta.backups[id] + if !exist { + return nil + } + return backup +} + +func (meta *MetaManager) AddBackup(backup *backuppb.BackupInfo) { + meta.mu.Lock() + defer meta.mu.Unlock() + meta.backups[backup.Id] = backup +} + +func (meta *MetaManager) AddCollection(collection *backuppb.CollectionBackupInfo) { + meta.mu.Lock() + defer meta.mu.Unlock() + if _, exist := meta.collections[collection.Id]; !exist { + meta.collections[collection.Id] = make(map[int64]*backuppb.CollectionBackupInfo, 0) + } + meta.collections[collection.Id][collection.GetCollectionId()] = collection + meta.collectionBackupReverse[collection.GetCollectionId()] = collection.Id +} + +func (meta *MetaManager) AddPartition(partition *backuppb.PartitionBackupInfo) { + meta.mu.Lock() + defer meta.mu.Unlock() + if _, exist := meta.partitions[partition.GetCollectionId()]; !exist { + meta.partitions[partition.GetCollectionId()] = make(map[int64]*backuppb.PartitionBackupInfo, 0) + } + meta.partitions[partition.GetCollectionId()][partition.GetPartitionId()] = partition + meta.partitionCollectionReverse[partition.GetPartitionId()] = partition.GetCollectionId() +} + +func (meta *MetaManager) AddSegment(segment *backuppb.SegmentBackupInfo) { + meta.mu.Lock() + defer meta.mu.Unlock() + if _, exist := meta.segments[segment.GetPartitionId()]; !exist { + meta.segments[segment.GetPartitionId()] = make(map[int64]*backuppb.SegmentBackupInfo, 0) + } + meta.segments[segment.GetPartitionId()][segment.GetSegmentId()] = segment + meta.segmentPartitionReverse[segment.GetSegmentId()] = segment.GetPartitionId() +} + +type BackupOpt func(backup *backuppb.BackupInfo) + +func setStateCode(stateCode backuppb.BackupTaskStateCode) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.StateCode = stateCode + } +} + +func setErrorMessage(errorMessage string) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.ErrorMessage = errorMessage + } +} +func setStartTime(startTime int64) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.StartTime = startTime + } +} + +func setEndTime(endTime int64) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.EndTime = endTime + } +} + +func setProgress(progress int32) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.Progress = progress + } +} + +func setName(name string) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.Name = name + } +} + +// backup timestamp +func setBackupTimestamp(backupTimestamp uint64) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.BackupTimestamp = backupTimestamp + } +} + +// array of collection backup +//repeated CollectionBackupInfo collection_backups = 9; + +func setSize(size int64) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.Size = size + } +} + +func incSize(size int64) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.Size = backup.Size + size + } +} + +func setMilvusVersion(milvusVersion string) BackupOpt { + return func(backup *backuppb.BackupInfo) { + backup.MilvusVersion = milvusVersion + } +} + +func (meta *MetaManager) UpdateBackup(backupID string, opts ...BackupOpt) { + meta.mu.Lock() + defer meta.mu.Unlock() + backup := meta.backups[backupID] + cBackup := proto.Clone(backup).(*backuppb.BackupInfo) + for _, opt := range opts { + opt(cBackup) + } + meta.backups[backup.Id] = cBackup +} + +type CollectionOpt func(collection *backuppb.CollectionBackupInfo) + +func setCollectionStateCode(stateCode backuppb.BackupTaskStateCode) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.StateCode = stateCode + } +} + +func setCollectionErrorMessage(errorMessage string) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.ErrorMessage = errorMessage + } +} + +func setCollectionStartTime(startTime int64) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.StartTime = startTime + } +} + +func setCollectionEndTime(endTime int64) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.EndTime = endTime + } +} + +func setCollectionProgress(progress int32) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.Progress = progress + } +} + +// backup timestamp +func setCollectionBackupTimestamp(backupTimestamp uint64) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.BackupTimestamp = backupTimestamp + } +} + +func setCollectionSize(size int64) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.Size = size + } +} + +func incCollectionSize(size int64) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.Size = collection.Size + size + } +} + +func setCollectionDbName(dbName string) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.DbName = dbName + } +} + +func setCollectionName(collectionName string) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.CollectionName = collectionName + } +} + +func setCollectionSchema(schema *backuppb.CollectionSchema) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.Schema = schema + } +} + +func setCollectionShardNum(shardsNum int32) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.ShardsNum = shardsNum + } +} + +func setCollectionConsistencyLevel(consistencyLevel backuppb.ConsistencyLevel) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.ConsistencyLevel = consistencyLevel + } +} + +func setCollectionHasIndex(hasIndex bool) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.HasIndex = hasIndex + } +} + +func setCollectionIndexInfos(indexInfos []*backuppb.IndexInfo) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.IndexInfos = indexInfos + } +} + +func setCollectionLoadState(loadState string) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.LoadState = loadState + } +} + +func setCollectionBackupPhysicalTimestamp(backupPhysicalTimestamp uint64) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.BackupPhysicalTimestamp = backupPhysicalTimestamp + } +} + +func setCollectionChannelCheckpoints(channelCheckpoints map[string]string) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.ChannelCheckpoints = channelCheckpoints + } +} + +func (meta *MetaManager) GetCollections(backupID string) map[int64]*backuppb.CollectionBackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + return meta.collections[backupID] +} + +func (meta *MetaManager) UpdateCollection(backupID string, collectionID int64, opts ...CollectionOpt) { + meta.mu.Lock() + defer meta.mu.Unlock() + backup := meta.collections[backupID][collectionID] + cBackup := proto.Clone(backup).(*backuppb.CollectionBackupInfo) + for _, opt := range opts { + opt(cBackup) + } + meta.collections[backupID][collectionID] = cBackup +} + +type PartitionOpt func(partition *backuppb.PartitionBackupInfo) + +func setPartitionSize(size int64) PartitionOpt { + return func(partition *backuppb.PartitionBackupInfo) { + partition.Size = size + } +} + +func (meta *MetaManager) GetPartitions(collectionID int64) map[int64]*backuppb.PartitionBackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + + partitions, exist := meta.partitions[collectionID] + if !exist { + return make(map[int64]*backuppb.PartitionBackupInfo, 0) + } + return partitions +} + +func (meta *MetaManager) UpdatePartition(collectionID int64, partitionID int64, opts ...PartitionOpt) { + meta.mu.Lock() + defer meta.mu.Unlock() + backup := meta.partitions[collectionID][partitionID] + cBackup := proto.Clone(backup).(*backuppb.PartitionBackupInfo) + for _, opt := range opts { + opt(cBackup) + } + meta.partitions[collectionID][partitionID] = cBackup +} + +type SegmentOpt func(segment *backuppb.SegmentBackupInfo) + +func (meta *MetaManager) GetSegments(partitionID int64) map[int64]*backuppb.SegmentBackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + segments, exist := meta.segments[partitionID] + if !exist { + return make(map[int64]*backuppb.SegmentBackupInfo, 0) + } + return segments +} + +func (meta *MetaManager) GetSegment(segmentID int64) *backuppb.SegmentBackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + partitionID := meta.segmentPartitionReverse[segmentID] + return meta.segments[partitionID][segmentID] +} + +func setSegmentNumOfRows(numOfRows int64) SegmentOpt { + return func(segment *backuppb.SegmentBackupInfo) { + segment.NumOfRows = numOfRows + } +} + +func setSegmentSize(size int64) SegmentOpt { + return func(segment *backuppb.SegmentBackupInfo) { + segment.Size = size + } +} + +func setSegmentBinlogs(binlogs []*backuppb.FieldBinlog) SegmentOpt { + return func(segment *backuppb.SegmentBackupInfo) { + segment.Binlogs = binlogs + } +} + +func setSegmentStatsBinlogs(binlogs []*backuppb.FieldBinlog) SegmentOpt { + return func(segment *backuppb.SegmentBackupInfo) { + segment.Statslogs = binlogs + } +} + +func setSegmentDeltaBinlogs(binlogs []*backuppb.FieldBinlog) SegmentOpt { + return func(segment *backuppb.SegmentBackupInfo) { + segment.Deltalogs = binlogs + } +} + +func setSegmentGroupId(groupId int64) SegmentOpt { + return func(segment *backuppb.SegmentBackupInfo) { + segment.GroupId = groupId + } +} + +func setSegmentBackuped(backuped bool) SegmentOpt { + return func(segment *backuppb.SegmentBackupInfo) { + segment.Backuped = backuped + } +} + +func (meta *MetaManager) UpdateSegment(partitionID int64, segmentID int64, opts ...SegmentOpt) { + meta.mu.Lock() + defer meta.mu.Unlock() + backup := meta.segments[partitionID][segmentID] + cBackup := proto.Clone(backup).(*backuppb.SegmentBackupInfo) + for _, opt := range opts { + opt(cBackup) + } + meta.segments[partitionID][segmentID] = cBackup +} + +func (meta *MetaManager) GetBackupBySegmentID(segmentID int64) *backuppb.BackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + partitionID, exist := meta.segmentPartitionReverse[segmentID] + if !exist { + return nil + } + collectionID, exist := meta.partitionCollectionReverse[partitionID] + if !exist { + return nil + } + backupID, exist := meta.collectionBackupReverse[collectionID] + if !exist { + return nil + } + return meta.backups[backupID] +} + +func (meta *MetaManager) GetFullMeta(id string) *backuppb.BackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + backup, exist := meta.backups[id] + if !exist { + return nil + } + collections := meta.collections[id] + collectionBackups := make([]*backuppb.CollectionBackupInfo, 0) + for collectionID, collection := range collections { + partitionBackups := make([]*backuppb.PartitionBackupInfo, 0) + for partitionID, partition := range meta.partitions[collectionID] { + segmentBackups := make([]*backuppb.SegmentBackupInfo, 0) + for _, segment := range meta.segments[partitionID] { + segmentBackups = append(segmentBackups, proto.Clone(segment).(*backuppb.SegmentBackupInfo)) + } + partitionBackup := proto.Clone(partition).(*backuppb.PartitionBackupInfo) + partitionBackup.SegmentBackups = segmentBackups + partitionBackups = append(partitionBackups, partitionBackup) + } + collectionBackup := proto.Clone(collection).(*backuppb.CollectionBackupInfo) + collectionBackup.PartitionBackups = partitionBackups + collectionBackups = append(collectionBackups, collectionBackup) + } + cloneBackup := proto.Clone(backup).(*backuppb.BackupInfo) + cloneBackup.CollectionBackups = collectionBackups + return cloneBackup +} + +func (meta *MetaManager) AddRestoreTask(task *backuppb.RestoreBackupTask) { + meta.mu.Lock() + defer meta.mu.Unlock() + meta.restoreTasks[task.Id] = task +} + +func (meta *MetaManager) GetRestoreTask(taskID string) *backuppb.RestoreBackupTask { + meta.mu.Lock() + defer meta.mu.Unlock() + return meta.restoreTasks[taskID] +} diff --git a/core/proto/backup.proto b/core/proto/backup.proto index ece418f7..28557e72 100644 --- a/core/proto/backup.proto +++ b/core/proto/backup.proto @@ -91,7 +91,8 @@ message SegmentBackupInfo { // separate segments into multi groups by size, // segments in one group will be copied into one directory during backup // and will bulkinsert in one call during restore - int64 group_id = 9; + int64 group_id = 9; + bool backuped = 10; } /** diff --git a/core/proto/backuppb/backup.pb.go b/core/proto/backuppb/backup.pb.go index 43b8e8c4..7171fa24 100644 --- a/core/proto/backuppb/backup.pb.go +++ b/core/proto/backuppb/backup.pb.go @@ -620,6 +620,7 @@ type SegmentBackupInfo struct { // segments in one group will be copied into one directory during backup // and will bulkinsert in one call during restore GroupId int64 `protobuf:"varint,9,opt,name=group_id,json=groupId,proto3" json:"group_id,omitempty"` + Backuped bool `protobuf:"varint,10,opt,name=backuped,proto3" json:"backuped,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -713,6 +714,13 @@ func (m *SegmentBackupInfo) GetGroupId() int64 { return 0 } +func (m *SegmentBackupInfo) GetBackuped() bool { + if m != nil { + return m.Backuped + } + return false +} + // * // root of backup type BackupInfo struct { @@ -2883,195 +2891,195 @@ func init() { func init() { proto.RegisterFile("backup.proto", fileDescriptor_65240d19de191688) } var fileDescriptor_65240d19de191688 = []byte{ - // 2993 bytes of a gzipped FileDescriptorProto + // 3003 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x5a, 0xcb, 0x6f, 0x1c, 0xc7, 0xd1, 0xe7, 0xbe, 0x77, 0x6b, 0x1f, 0x1c, 0x36, 0x29, 0x6a, 0x45, 0x59, 0x16, 0xb5, 0x9f, 0x2d, 0x53, 0x32, 0x3e, 0xca, 0xa6, 0x6d, 0x7d, 0xb6, 0xf0, 0xf9, 0x21, 0x3e, 0x24, 0xad, 0x25, 0x51, 0xc4, 0x90, 0x12, 0x04, 0xe7, 0x31, 0x98, 0x9d, 0x69, 0x2e, 0x27, 0x9a, 0x9d, 0xde, 0x4c, 0xf7, - 0xca, 0x5a, 0x01, 0x09, 0x02, 0x04, 0x01, 0x72, 0xc8, 0x21, 0x87, 0x9c, 0xf2, 0x1f, 0xe4, 0x96, - 0x20, 0xf0, 0x25, 0xff, 0x41, 0x82, 0xfc, 0x11, 0xb9, 0x05, 0x39, 0xe5, 0x98, 0x6b, 0xd0, 0xd5, - 0x3d, 0x8f, 0x5d, 0x0e, 0xa9, 0x65, 0x60, 0xc4, 0x71, 0x6e, 0xd3, 0xbf, 0xae, 0xaa, 0xee, 0xae, - 0xae, 0xae, 0xaa, 0xae, 0x1e, 0x68, 0xf4, 0x6c, 0xe7, 0xd9, 0x68, 0xb8, 0x3e, 0x0c, 0x99, 0x60, - 0x64, 0x71, 0xe0, 0xf9, 0xcf, 0x47, 0x5c, 0xb5, 0xd6, 0x55, 0xd7, 0xca, 0x6b, 0x7d, 0xc6, 0xfa, - 0x3e, 0xbd, 0x81, 0x60, 0x6f, 0x74, 0x78, 0x83, 0x8b, 0x70, 0xe4, 0x08, 0x45, 0xd4, 0xf9, 0x6b, - 0x0e, 0x6a, 0xdd, 0xc0, 0xa5, 0x2f, 0xba, 0xc1, 0x21, 0x23, 0x97, 0x00, 0x0e, 0x3d, 0xea, 0xbb, - 0x56, 0x60, 0x0f, 0x68, 0x3b, 0xb7, 0x9a, 0x5b, 0xab, 0x99, 0x35, 0x44, 0x76, 0xed, 0x01, 0x95, - 0xdd, 0x9e, 0xa4, 0x55, 0xdd, 0x79, 0xd5, 0x8d, 0xc8, 0x64, 0xb7, 0x18, 0x0f, 0x69, 0xbb, 0x90, - 0xea, 0x3e, 0x18, 0x0f, 0x29, 0xd9, 0x84, 0xf2, 0xd0, 0x0e, 0xed, 0x01, 0x6f, 0x17, 0x57, 0x0b, - 0x6b, 0xf5, 0x8d, 0xeb, 0xeb, 0x19, 0xd3, 0x5d, 0x8f, 0x27, 0xb3, 0xbe, 0x87, 0xc4, 0x3b, 0x81, - 0x08, 0xc7, 0xa6, 0xe6, 0x5c, 0xf9, 0x08, 0xea, 0x29, 0x98, 0x18, 0x50, 0x78, 0x46, 0xc7, 0x7a, - 0xa2, 0xf2, 0x93, 0x2c, 0x41, 0xe9, 0xb9, 0xed, 0x8f, 0xa2, 0xd9, 0xa9, 0xc6, 0xad, 0xfc, 0x87, - 0xb9, 0xce, 0x2f, 0xaa, 0xb0, 0xb4, 0xc5, 0x7c, 0x9f, 0x3a, 0xc2, 0x63, 0xc1, 0x26, 0x8e, 0x86, - 0x8b, 0x6e, 0x41, 0xde, 0x73, 0xb5, 0x8c, 0xbc, 0xe7, 0x92, 0xbb, 0x00, 0x5c, 0xd8, 0x82, 0x5a, - 0x0e, 0x73, 0x95, 0x9c, 0xd6, 0xc6, 0x5a, 0xe6, 0x5c, 0x95, 0x90, 0x03, 0x9b, 0x3f, 0xdb, 0x97, - 0x0c, 0x5b, 0xcc, 0xa5, 0x66, 0x8d, 0x47, 0x9f, 0xa4, 0x03, 0x0d, 0x1a, 0x86, 0x2c, 0x7c, 0x48, - 0x39, 0xb7, 0xfb, 0x91, 0x46, 0x26, 0x30, 0xa9, 0x33, 0x2e, 0xec, 0x50, 0x58, 0xc2, 0x1b, 0xd0, - 0x76, 0x71, 0x35, 0xb7, 0x56, 0x40, 0x11, 0xa1, 0x38, 0xf0, 0x06, 0x94, 0x5c, 0x80, 0x2a, 0x0d, - 0x5c, 0xd5, 0x59, 0xc2, 0xce, 0x0a, 0x0d, 0x5c, 0xec, 0x5a, 0x81, 0xea, 0x30, 0x64, 0xfd, 0x90, - 0x72, 0xde, 0x2e, 0xaf, 0xe6, 0xd6, 0x4a, 0x66, 0xdc, 0x26, 0xff, 0x03, 0x4d, 0x27, 0x5e, 0xaa, - 0xe5, 0xb9, 0xed, 0x0a, 0xf2, 0x36, 0x12, 0xb0, 0xeb, 0x92, 0xf3, 0x50, 0x71, 0x7b, 0x6a, 0x2b, - 0xab, 0x38, 0xb3, 0xb2, 0xdb, 0xc3, 0x7d, 0x7c, 0x0b, 0xe6, 0x53, 0xdc, 0x48, 0x50, 0x43, 0x82, - 0x56, 0x02, 0x23, 0xe1, 0xc7, 0x50, 0xe6, 0xce, 0x11, 0x1d, 0xd8, 0x6d, 0x58, 0xcd, 0xad, 0xd5, - 0x37, 0xde, 0xcc, 0xd4, 0x52, 0xa2, 0xf4, 0x7d, 0x24, 0x36, 0x35, 0x13, 0xae, 0xfd, 0xc8, 0x0e, - 0x5d, 0x6e, 0x05, 0xa3, 0x41, 0xbb, 0x8e, 0x6b, 0xa8, 0x29, 0x64, 0x77, 0x34, 0x20, 0x26, 0x2c, - 0x38, 0x2c, 0xe0, 0x1e, 0x17, 0x34, 0x70, 0xc6, 0x96, 0x4f, 0x9f, 0x53, 0xbf, 0xdd, 0xc0, 0xed, - 0x38, 0x69, 0xa0, 0x98, 0xfa, 0x81, 0x24, 0x36, 0x0d, 0x67, 0x0a, 0x21, 0x8f, 0x61, 0x61, 0x68, - 0x87, 0xc2, 0xc3, 0x95, 0x29, 0x36, 0xde, 0x6e, 0xa2, 0x39, 0x66, 0x6f, 0xf1, 0x5e, 0x44, 0x9d, - 0x18, 0x8c, 0x69, 0x0c, 0x27, 0x41, 0x4e, 0xae, 0x81, 0xa1, 0xe8, 0x71, 0xa7, 0xb8, 0xb0, 0x07, - 0xc3, 0x76, 0x6b, 0x35, 0xb7, 0x56, 0x34, 0xe7, 0x15, 0x7e, 0x10, 0xc1, 0x84, 0x40, 0x91, 0x7b, - 0x2f, 0x69, 0x7b, 0x1e, 0x77, 0x04, 0xbf, 0xc9, 0x45, 0xa8, 0x1d, 0xd9, 0xdc, 0xc2, 0xa3, 0xd2, - 0x36, 0x56, 0x73, 0x6b, 0x55, 0xb3, 0x7a, 0x64, 0x73, 0x3c, 0x0a, 0xe4, 0x53, 0xa8, 0xab, 0x53, - 0xe5, 0x05, 0x87, 0x8c, 0xb7, 0x17, 0x70, 0xb2, 0xaf, 0x9f, 0x7e, 0x76, 0x4c, 0x75, 0x10, 0xe5, - 0x27, 0x97, 0x6a, 0xf6, 0x99, 0xed, 0x5a, 0x68, 0x98, 0x6d, 0xa2, 0x8e, 0xa5, 0x44, 0xd0, 0x68, - 0xc9, 0x2d, 0xb8, 0xa0, 0xe7, 0x3e, 0x3c, 0x1a, 0x73, 0xcf, 0xb1, 0xfd, 0xd4, 0x22, 0x16, 0x71, - 0x11, 0xe7, 0x15, 0xc1, 0x9e, 0xee, 0x4f, 0x16, 0x13, 0xc2, 0xa2, 0x73, 0x64, 0x07, 0x01, 0xf5, - 0x2d, 0xe7, 0x88, 0x3a, 0xcf, 0x86, 0xcc, 0x0b, 0x04, 0x6f, 0x2f, 0xe1, 0x1c, 0x6f, 0xbf, 0xc2, - 0x1a, 0x12, 0x8d, 0xae, 0x6f, 0x29, 0x21, 0x5b, 0x89, 0x0c, 0x75, 0xec, 0x89, 0x73, 0xac, 0x63, - 0x65, 0x07, 0xce, 0x9f, 0x40, 0x7e, 0x26, 0x77, 0xf0, 0xf3, 0x3c, 0x2c, 0x66, 0x6c, 0x2e, 0xb9, - 0x02, 0x8d, 0xc4, 0x42, 0xb4, 0x5f, 0x28, 0x98, 0xf5, 0x18, 0xeb, 0xba, 0xe4, 0x4d, 0x68, 0x25, - 0x24, 0x29, 0x57, 0xd8, 0x8c, 0x51, 0x3c, 0x1d, 0xc7, 0x0e, 0x61, 0x21, 0xe3, 0x10, 0x3e, 0x82, - 0x79, 0x4e, 0xfb, 0x03, 0x1a, 0x88, 0xd8, 0x1c, 0x95, 0x77, 0xbc, 0x9a, 0xa9, 0xbd, 0x7d, 0x45, - 0x9b, 0x32, 0xc6, 0x16, 0x4f, 0x43, 0x3c, 0xb6, 0xaf, 0x52, 0xca, 0xbe, 0x26, 0x2d, 0xa0, 0x3c, - 0x65, 0x01, 0x9d, 0x9f, 0x15, 0x60, 0xe1, 0x98, 0x60, 0x3c, 0x9d, 0x7a, 0x66, 0xb1, 0x1a, 0x6a, - 0x1a, 0xe9, 0xba, 0xc7, 0x57, 0x97, 0xcf, 0x58, 0xdd, 0xb4, 0x32, 0x0b, 0xc7, 0x95, 0xf9, 0x3a, - 0xd4, 0x83, 0xd1, 0xc0, 0x62, 0x87, 0x56, 0xc8, 0xbe, 0xe4, 0x91, 0x07, 0x0c, 0x46, 0x83, 0x47, - 0x87, 0x26, 0xfb, 0x92, 0x93, 0x5b, 0x50, 0xe9, 0x79, 0x81, 0xcf, 0xfa, 0xbc, 0x5d, 0x42, 0xc5, - 0xac, 0x66, 0x2a, 0xe6, 0x8e, 0x0c, 0x52, 0x9b, 0x48, 0x68, 0x46, 0x0c, 0xe4, 0x13, 0x40, 0x6f, - 0xcc, 0x91, 0xbb, 0x3c, 0x23, 0x77, 0xc2, 0x22, 0xf9, 0x5d, 0xea, 0x0b, 0x1b, 0xf9, 0x2b, 0xb3, - 0xf2, 0xc7, 0x2c, 0xf1, 0x5e, 0x54, 0x53, 0x7b, 0x71, 0x01, 0xaa, 0xfd, 0x90, 0x8d, 0x86, 0x52, - 0x1d, 0x35, 0xe5, 0xd1, 0xb1, 0xdd, 0x75, 0x3b, 0xbf, 0x2f, 0x00, 0xfc, 0x77, 0xc7, 0x25, 0x02, - 0x45, 0x3c, 0x2f, 0x15, 0x1c, 0x11, 0xbf, 0x33, 0x7d, 0x67, 0x35, 0xdb, 0x77, 0x3e, 0x05, 0x92, - 0xb2, 0xb9, 0xe8, 0xbc, 0xd4, 0x70, 0x63, 0xae, 0xcd, 0xec, 0x6d, 0xcc, 0x05, 0x67, 0x0a, 0x4d, - 0x76, 0x0a, 0x52, 0x3b, 0xf5, 0x26, 0xb4, 0x94, 0x48, 0xeb, 0x39, 0x0d, 0xb9, 0xc7, 0x02, 0x0c, - 0x51, 0x35, 0xb3, 0xa9, 0xd0, 0x27, 0x0a, 0xec, 0x7c, 0x17, 0x2e, 0x24, 0xa3, 0x60, 0x94, 0x49, - 0xed, 0xe1, 0xa7, 0x50, 0x52, 0x6e, 0x3b, 0x77, 0xd6, 0x49, 0x2a, 0xbe, 0xce, 0x17, 0xd0, 0x8e, - 0xbd, 0xd4, 0xb4, 0xf0, 0x4f, 0x26, 0x85, 0xcf, 0x1e, 0xc0, 0xb4, 0xec, 0x27, 0xb0, 0xac, 0x8f, - 0xfd, 0xb4, 0xe4, 0xff, 0x9f, 0x94, 0x3c, 0xab, 0x2f, 0xd2, 0x72, 0x7f, 0x5a, 0x80, 0xc5, 0xad, - 0x90, 0xda, 0x82, 0xaa, 0x3e, 0x93, 0xfe, 0x70, 0x44, 0xb9, 0x20, 0xaf, 0x41, 0x2d, 0x54, 0x9f, - 0xdd, 0xc8, 0xae, 0x13, 0x80, 0x5c, 0x86, 0xba, 0xb6, 0x83, 0x94, 0x4b, 0x05, 0x05, 0xed, 0x6a, - 0x43, 0x99, 0x4a, 0x4b, 0x78, 0xbb, 0xb0, 0x5a, 0x58, 0xab, 0x99, 0xf3, 0x93, 0x79, 0x09, 0x97, - 0x6e, 0xdf, 0xe6, 0xe3, 0xc0, 0x41, 0xc3, 0xad, 0x9a, 0xaa, 0x41, 0x3e, 0x86, 0x96, 0xdb, 0xb3, - 0x12, 0x5a, 0x8e, 0xa6, 0x5b, 0xdf, 0x58, 0x5e, 0x57, 0x29, 0xf2, 0x7a, 0x94, 0x22, 0xaf, 0x3f, - 0x91, 0x61, 0xc2, 0x6c, 0xba, 0xbd, 0x64, 0x6b, 0x50, 0xe8, 0x21, 0x0b, 0x1d, 0xe5, 0x40, 0xab, - 0xa6, 0x6a, 0xc8, 0xd8, 0x3d, 0xa0, 0xc2, 0xb6, 0x58, 0xe0, 0x8f, 0xd1, 0xae, 0xab, 0x66, 0x55, - 0x02, 0x8f, 0x02, 0x7f, 0x4c, 0xae, 0xc2, 0x7c, 0xdf, 0xb1, 0x86, 0xf6, 0x88, 0x53, 0x8b, 0x06, - 0x76, 0xcf, 0x57, 0xbe, 0xa0, 0x6a, 0x36, 0xfb, 0xce, 0x9e, 0x44, 0x77, 0x10, 0x24, 0x6b, 0x60, - 0xc4, 0x74, 0x9c, 0x3a, 0x2c, 0x70, 0x39, 0x3a, 0x87, 0x92, 0xd9, 0xd2, 0x84, 0xfb, 0x0a, 0x9d, - 0xa0, 0xb4, 0x5d, 0x17, 0x4f, 0x19, 0xa8, 0xe4, 0x4c, 0x53, 0xde, 0x56, 0x68, 0xe7, 0xb7, 0x39, - 0x20, 0xa9, 0xbd, 0xa1, 0x7c, 0xc8, 0x02, 0x4e, 0x5f, 0xb1, 0x09, 0x1f, 0x40, 0x31, 0xe5, 0x5d, - 0xae, 0x64, 0xee, 0x7b, 0x24, 0x0a, 0xdd, 0x0a, 0x92, 0xcb, 0xc0, 0x3b, 0xe0, 0x7d, 0xed, 0x48, - 0xe4, 0x27, 0x79, 0x0f, 0x8a, 0xae, 0x2d, 0x6c, 0xdc, 0x80, 0xfa, 0xc6, 0xe5, 0x53, 0xdc, 0x14, - 0xce, 0x0e, 0x89, 0x3b, 0x7f, 0xca, 0x81, 0x71, 0x97, 0x8a, 0xaf, 0xd5, 0x6a, 0x2e, 0x42, 0x4d, - 0x13, 0xe8, 0xf8, 0x53, 0x33, 0xab, 0x0a, 0xd0, 0xdc, 0x23, 0xe7, 0x19, 0x15, 0x8a, 0xbb, 0xa8, - 0xb9, 0x11, 0x42, 0x6e, 0x02, 0xc5, 0xa1, 0x2d, 0x8e, 0xd0, 0x50, 0x6a, 0x26, 0x7e, 0x4b, 0xbf, - 0xf0, 0xa5, 0x27, 0x8e, 0xd8, 0x48, 0x58, 0x2e, 0x15, 0xb6, 0xe7, 0x6b, 0x83, 0x68, 0x6a, 0x74, - 0x1b, 0xc1, 0xce, 0x77, 0x80, 0x3c, 0xf0, 0x78, 0x14, 0x97, 0x67, 0x5b, 0x4d, 0x46, 0xe6, 0x9d, - 0xcf, 0xca, 0xbc, 0x3b, 0xbf, 0xcb, 0xc1, 0xe2, 0x84, 0xf4, 0x6f, 0x6a, 0x77, 0x0b, 0xb3, 0xef, - 0xee, 0x01, 0x2c, 0x6e, 0x53, 0x9f, 0x7e, 0xbd, 0x5e, 0xa1, 0xf3, 0x23, 0x58, 0x9a, 0x94, 0xfa, - 0x6f, 0xd5, 0x44, 0xe7, 0x2f, 0x25, 0x58, 0x32, 0x29, 0x17, 0x2c, 0xfc, 0xc6, 0x9c, 0xdd, 0xdb, - 0x90, 0x0a, 0x68, 0x16, 0x1f, 0x1d, 0x1e, 0x7a, 0x2f, 0xb4, 0x29, 0xa7, 0x64, 0xec, 0x23, 0x4e, - 0xd8, 0x44, 0x08, 0x0d, 0xa9, 0x92, 0xac, 0x32, 0xab, 0xcf, 0x4e, 0x52, 0xc3, 0xb1, 0xd5, 0xa5, - 0x42, 0x96, 0xa9, 0x44, 0xa8, 0x7c, 0x3d, 0x35, 0x11, 0x8d, 0x27, 0xae, 0xb8, 0x9c, 0x76, 0xc5, - 0x53, 0x07, 0xaf, 0x72, 0xe2, 0xc1, 0xab, 0xa6, 0x0e, 0xde, 0x71, 0xff, 0x5d, 0x3b, 0x8b, 0xff, - 0x5e, 0x81, 0xd8, 0x31, 0xa3, 0xcb, 0x4c, 0x3b, 0xea, 0x0e, 0x34, 0x42, 0xb5, 0x4e, 0xbc, 0x43, - 0x61, 0xa4, 0xaf, 0x9a, 0x13, 0x98, 0xa4, 0x91, 0xee, 0x75, 0x24, 0x98, 0xa2, 0x69, 0x28, 0x9a, - 0x34, 0x46, 0xde, 0x81, 0x45, 0x37, 0x64, 0xc3, 0x9d, 0x17, 0x1e, 0x17, 0xc9, 0xd8, 0xed, 0x26, - 0x92, 0x66, 0x75, 0x91, 0xab, 0xd0, 0x8a, 0x61, 0x25, 0xb7, 0x85, 0xc4, 0x53, 0x28, 0xd9, 0x80, - 0x25, 0xfe, 0xcc, 0x1b, 0xaa, 0xb8, 0x9a, 0x12, 0x3d, 0x8f, 0xd4, 0x99, 0x7d, 0x2b, 0xdb, 0xb0, - 0x9c, 0xbd, 0x51, 0x67, 0xba, 0x29, 0x7d, 0x95, 0x8f, 0x4d, 0x3c, 0x4e, 0x26, 0x64, 0x6e, 0x79, - 0x2c, 0x41, 0xbd, 0x97, 0x91, 0xa0, 0x5e, 0x3b, 0xcd, 0xa6, 0xfe, 0x03, 0x33, 0xd4, 0x2e, 0xe0, - 0xed, 0x44, 0x27, 0x97, 0x68, 0x98, 0x67, 0xc9, 0xac, 0x40, 0x32, 0xab, 0x76, 0xe7, 0xab, 0x32, - 0x9c, 0xd3, 0x0b, 0x4d, 0x76, 0xe1, 0x5b, 0xad, 0xb8, 0xcf, 0xa1, 0x2e, 0x4f, 0x5f, 0xa4, 0x9c, - 0x32, 0x2a, 0xe7, 0x0c, 0x39, 0x2d, 0x48, 0x6e, 0xd5, 0x26, 0xef, 0xc3, 0xb2, 0xb0, 0xc3, 0x3e, - 0x15, 0xd6, 0x74, 0xc4, 0x53, 0xce, 0x60, 0x49, 0xf5, 0x6e, 0x4d, 0x56, 0x9c, 0x6c, 0x38, 0x9f, - 0x5c, 0x28, 0xf5, 0xe9, 0xb4, 0x84, 0xcd, 0x9f, 0xf1, 0x76, 0xf5, 0x94, 0x0c, 0x3b, 0xcb, 0x7c, - 0xcd, 0x73, 0xb1, 0xa4, 0x94, 0x56, 0xb1, 0x76, 0xa6, 0x05, 0xbb, 0x16, 0xde, 0x09, 0xd4, 0x2d, - 0x2d, 0xf2, 0x05, 0xee, 0xbe, 0xbc, 0x1b, 0x5c, 0x85, 0x79, 0xc1, 0xe2, 0x09, 0xa4, 0xae, 0x0e, - 0x4d, 0xc1, 0xb4, 0x34, 0xa4, 0x4b, 0x9b, 0x5a, 0x7d, 0xca, 0xd4, 0xde, 0x80, 0x96, 0xd6, 0x40, - 0x54, 0x86, 0x6b, 0xa8, 0xdd, 0x52, 0xe8, 0xb6, 0x2a, 0xc6, 0xa5, 0xbd, 0x56, 0xf3, 0x15, 0x5e, - 0xab, 0x35, 0x83, 0xd7, 0x9a, 0x9f, 0xdd, 0x6b, 0x19, 0x67, 0xf1, 0x5a, 0x0b, 0x67, 0xf2, 0x5a, - 0xe4, 0x64, 0xaf, 0xd5, 0xf9, 0x75, 0x01, 0x16, 0x26, 0x82, 0xce, 0xb7, 0xfa, 0xcc, 0xb8, 0xd0, - 0x9e, 0x08, 0xb8, 0x69, 0x93, 0x2d, 0x9f, 0x52, 0x07, 0xcf, 0xf4, 0x1c, 0xe6, 0x72, 0x3a, 0xc0, - 0x9e, 0x66, 0xb4, 0x95, 0xd9, 0x8c, 0xb6, 0xfa, 0x2a, 0xa3, 0xad, 0x4d, 0x1a, 0x6d, 0xe7, 0x0f, - 0xb9, 0xd8, 0xa9, 0x7d, 0x23, 0x09, 0x17, 0xb9, 0x35, 0x71, 0xb1, 0xb8, 0xfa, 0xea, 0x94, 0x05, - 0xf5, 0xa6, 0x32, 0xd0, 0x3b, 0xb0, 0x7c, 0x97, 0x8a, 0x68, 0xa9, 0xd2, 0x00, 0x66, 0xcb, 0xd6, - 0x94, 0xed, 0xe5, 0x23, 0xdb, 0xeb, 0x7c, 0x1f, 0xea, 0xa9, 0x8a, 0x0f, 0x69, 0x43, 0x05, 0xdf, - 0x48, 0xba, 0xdb, 0xba, 0x4c, 0x16, 0x35, 0xc9, 0x07, 0x49, 0xf1, 0x2a, 0x8f, 0x7b, 0x7d, 0x31, - 0x3b, 0x55, 0x9e, 0xac, 0x5b, 0x75, 0x7e, 0x93, 0x83, 0xb2, 0x96, 0x7d, 0x19, 0xea, 0x34, 0x10, - 0xa1, 0x47, 0x55, 0x91, 0x5c, 0xc9, 0x07, 0x0d, 0xed, 0x8e, 0x06, 0xf2, 0x36, 0x12, 0xd7, 0x4d, - 0xac, 0xc3, 0x90, 0x0d, 0x70, 0x9e, 0x45, 0xb3, 0x19, 0xa3, 0x77, 0x42, 0x36, 0x20, 0x57, 0xa0, - 0x91, 0x90, 0x09, 0x86, 0x1a, 0x2d, 0x9a, 0xf5, 0x18, 0x3b, 0x60, 0xd2, 0x88, 0x7d, 0xd6, 0xb7, - 0x30, 0xed, 0x52, 0xe9, 0x63, 0xc5, 0x67, 0xfd, 0x3d, 0x99, 0x79, 0xe9, 0xae, 0x54, 0x61, 0x51, - 0x76, 0x49, 0x63, 0xe9, 0xdc, 0x84, 0xc6, 0x7d, 0x3a, 0xc6, 0x84, 0x6b, 0xcf, 0xf6, 0xc2, 0x59, - 0x33, 0x8b, 0xce, 0x3f, 0x72, 0x00, 0xc8, 0x85, 0x9a, 0x24, 0x97, 0xa0, 0xd6, 0x63, 0xcc, 0xb7, - 0x70, 0x6f, 0x25, 0x73, 0xf5, 0xde, 0x9c, 0x59, 0x95, 0xd0, 0xb6, 0x2d, 0x6c, 0x72, 0x11, 0xaa, - 0x5e, 0x20, 0x54, 0xaf, 0x14, 0x53, 0xba, 0x37, 0x67, 0x56, 0xbc, 0x40, 0x60, 0xe7, 0x25, 0xa8, - 0xf9, 0x2c, 0xe8, 0xab, 0x5e, 0x2c, 0x31, 0x4a, 0x5e, 0x09, 0x61, 0xf7, 0x65, 0x80, 0x43, 0x9f, - 0xd9, 0x9a, 0x5b, 0xae, 0x2c, 0x7f, 0x6f, 0xce, 0xac, 0x21, 0x86, 0x04, 0x57, 0xa0, 0xee, 0xb2, - 0x51, 0xcf, 0xa7, 0x8a, 0x42, 0x2e, 0x30, 0x77, 0x6f, 0xce, 0x04, 0x05, 0x46, 0x24, 0x5c, 0x84, - 0x5e, 0x34, 0x08, 0x96, 0x50, 0x25, 0x89, 0x02, 0xa3, 0x61, 0x7a, 0x63, 0x41, 0xb9, 0xa2, 0x90, - 0xe7, 0xaf, 0x21, 0x87, 0x41, 0x4c, 0x12, 0x6c, 0x96, 0x95, 0xe5, 0x76, 0xfe, 0x56, 0xd4, 0xe6, - 0xa3, 0x9e, 0x43, 0x4e, 0x31, 0x9f, 0xa8, 0x5c, 0x96, 0x4f, 0x95, 0xcb, 0xde, 0x80, 0x96, 0xc7, - 0xad, 0x61, 0xe8, 0x0d, 0xec, 0x70, 0x6c, 0x49, 0x55, 0x17, 0x94, 0x47, 0xf7, 0xf8, 0x9e, 0x02, - 0xef, 0xd3, 0x31, 0x59, 0x85, 0xba, 0x4b, 0xb9, 0x13, 0x7a, 0x43, 0x74, 0xb7, 0x6a, 0x3b, 0xd3, - 0x10, 0xb9, 0x05, 0x35, 0x39, 0x1b, 0xf5, 0x56, 0x57, 0xc2, 0x53, 0x79, 0x29, 0xd3, 0x38, 0xe5, - 0xdc, 0x0f, 0xc6, 0x43, 0x6a, 0x56, 0x5d, 0xfd, 0x45, 0x36, 0xa1, 0x2e, 0xd9, 0x2c, 0xfd, 0x9c, - 0xa7, 0xdc, 0x58, 0xf6, 0x99, 0x4e, 0xdb, 0x86, 0x09, 0x92, 0x4b, 0xbd, 0xdf, 0x91, 0x6d, 0x68, - 0xa8, 0x67, 0x0d, 0x2d, 0xa4, 0x32, 0xab, 0x10, 0xf5, 0x1a, 0xa2, 0xa5, 0x2c, 0x43, 0xd9, 0x96, - 0x61, 0x6c, 0x5b, 0xd7, 0x55, 0x74, 0x8b, 0x7c, 0x00, 0x25, 0x55, 0xec, 0xae, 0xe1, 0xca, 0x2e, - 0x9f, 0x5c, 0xb5, 0x55, 0x6e, 0x40, 0x51, 0x93, 0xcf, 0xa0, 0x41, 0x7d, 0x8a, 0x35, 0x6f, 0xd4, - 0x0b, 0xcc, 0xa2, 0x97, 0xba, 0x66, 0x41, 0xd5, 0x6c, 0x43, 0xd3, 0xa5, 0x87, 0xf6, 0xc8, 0x17, - 0x96, 0x32, 0xfa, 0xfa, 0x29, 0x05, 0x90, 0xc4, 0xfe, 0xcd, 0x86, 0xe6, 0x42, 0x08, 0x5f, 0x52, - 0xb9, 0xe5, 0x8e, 0x03, 0x7b, 0xe0, 0x39, 0xfa, 0xa2, 0x51, 0xf3, 0xf8, 0xb6, 0x02, 0xc8, 0x1a, - 0x18, 0xd2, 0x06, 0xe2, 0x44, 0x48, 0x5a, 0x81, 0xca, 0x0d, 0x5a, 0x1e, 0x8f, 0x93, 0x9c, 0xfb, - 0x74, 0xdc, 0xf9, 0x73, 0x0e, 0x8c, 0xe9, 0xf7, 0xb7, 0xd8, 0xac, 0x72, 0x29, 0xb3, 0x9a, 0x32, - 0x98, 0xfc, 0x71, 0x83, 0x49, 0x54, 0x5d, 0x98, 0x50, 0xf5, 0x87, 0x50, 0x46, 0x7b, 0x8d, 0x1e, - 0x2e, 0x4e, 0xa9, 0x90, 0x47, 0xef, 0x7f, 0x8a, 0x9e, 0xbc, 0x03, 0x4b, 0xaa, 0x28, 0x16, 0xad, - 0xd4, 0xc2, 0x0e, 0xb4, 0xc6, 0xaa, 0x49, 0x54, 0x9f, 0x5e, 0x33, 0xf2, 0x77, 0x5a, 0xd0, 0xc0, - 0x47, 0x1f, 0xed, 0xb6, 0x3b, 0x4f, 0xa1, 0xa9, 0xdb, 0x3a, 0x08, 0x45, 0x61, 0x26, 0xf7, 0x2f, - 0x85, 0x99, 0x7c, 0x72, 0xaf, 0xff, 0x49, 0x0e, 0xea, 0x0f, 0x79, 0x7f, 0x8f, 0x71, 0xd4, 0xa5, - 0xf4, 0x9f, 0xd1, 0x4b, 0x57, 0x4a, 0x77, 0x75, 0x8d, 0x61, 0xa6, 0xb6, 0x04, 0xa5, 0x01, 0xef, - 0x77, 0xb7, 0x51, 0x4c, 0xc3, 0x54, 0x0d, 0xcc, 0xdf, 0x78, 0xff, 0x6e, 0xc8, 0x46, 0xc3, 0xa8, - 0xfc, 0x14, 0xb5, 0x65, 0xd4, 0x49, 0x6a, 0xde, 0x45, 0xf4, 0xc8, 0x09, 0xd0, 0xb9, 0x0d, 0xf3, - 0xfa, 0xa1, 0x2b, 0x9e, 0x45, 0xd6, 0xce, 0xc9, 0x68, 0xad, 0xfb, 0xf5, 0x02, 0xe2, 0xf6, 0xf5, - 0x1f, 0x43, 0x23, 0xbd, 0x5a, 0x52, 0x87, 0xca, 0xfe, 0xc8, 0x71, 0x28, 0xe7, 0xc6, 0x1c, 0x99, - 0x87, 0xfa, 0x2e, 0x13, 0xd6, 0xfe, 0x68, 0x38, 0x64, 0xa1, 0x30, 0x72, 0x64, 0x01, 0x9a, 0xbb, - 0xcc, 0xda, 0xa3, 0xe1, 0xc0, 0xe3, 0xdc, 0x63, 0x81, 0x91, 0x27, 0x55, 0x28, 0xde, 0xb1, 0x3d, - 0xdf, 0x28, 0x90, 0x25, 0x98, 0xc7, 0x33, 0x47, 0x05, 0x0d, 0xad, 0x1d, 0x99, 0x1b, 0x19, 0xbf, - 0x2c, 0x90, 0x4b, 0xd0, 0xd6, 0x7b, 0x61, 0x3d, 0xea, 0xfd, 0x80, 0x3a, 0xc2, 0x92, 0x22, 0xef, - 0xb0, 0x51, 0xe0, 0x1a, 0xbf, 0x2a, 0x5c, 0x7f, 0x01, 0x8b, 0x19, 0x6f, 0x11, 0x84, 0x40, 0x6b, - 0xf3, 0xf6, 0xd6, 0xfd, 0xc7, 0x7b, 0x56, 0x77, 0xb7, 0x7b, 0xd0, 0xbd, 0xfd, 0xc0, 0x98, 0x23, - 0x4b, 0x60, 0x68, 0x6c, 0xe7, 0xe9, 0xce, 0xd6, 0xe3, 0x83, 0xee, 0xee, 0x5d, 0x23, 0x97, 0xa2, - 0xdc, 0x7f, 0xbc, 0xb5, 0xb5, 0xb3, 0xbf, 0x6f, 0xe4, 0xe5, 0xbc, 0x35, 0x76, 0xe7, 0x76, 0xf7, - 0x81, 0x51, 0x48, 0x11, 0x1d, 0x74, 0x1f, 0xee, 0x3c, 0x7a, 0x7c, 0x60, 0x14, 0xaf, 0x3f, 0x89, - 0xef, 0xac, 0x93, 0x43, 0xd7, 0xa1, 0x92, 0x8c, 0xd9, 0x84, 0x5a, 0x7a, 0x30, 0xa9, 0x9d, 0x78, - 0x14, 0xb9, 0x72, 0x25, 0xbe, 0x0e, 0x95, 0x44, 0xee, 0x53, 0x79, 0x9e, 0xa6, 0x1e, 0x95, 0x01, - 0xca, 0xfb, 0x22, 0x64, 0x41, 0xdf, 0x98, 0x43, 0x19, 0x54, 0x69, 0x0f, 0x05, 0x6e, 0x4a, 0x55, - 0x50, 0xd7, 0xc8, 0x93, 0x16, 0xc0, 0xce, 0x73, 0x1a, 0x88, 0x91, 0xed, 0xfb, 0x63, 0xa3, 0x20, - 0xdb, 0x5b, 0x23, 0x2e, 0xd8, 0xc0, 0x7b, 0x49, 0x5d, 0xa3, 0x78, 0xfd, 0xef, 0x39, 0xa8, 0x46, - 0x3e, 0x45, 0x8e, 0xbe, 0xcb, 0x02, 0x6a, 0xcc, 0xc9, 0xaf, 0x4d, 0xc6, 0x7c, 0x23, 0x27, 0xbf, - 0xba, 0x81, 0xf8, 0xd0, 0xc8, 0x93, 0x1a, 0x94, 0xba, 0x81, 0x78, 0xf7, 0xa6, 0x51, 0xd0, 0x9f, - 0xef, 0x6d, 0x18, 0x45, 0xfd, 0x79, 0xf3, 0x7d, 0xa3, 0x24, 0x3f, 0xef, 0xc8, 0xf0, 0x66, 0x80, - 0x9c, 0xdc, 0x36, 0xc6, 0x31, 0xa3, 0xae, 0x27, 0xea, 0x05, 0x7d, 0x63, 0x49, 0xce, 0xed, 0x89, - 0x1d, 0x6e, 0x1d, 0xd9, 0xa1, 0x71, 0x4e, 0xd2, 0xdf, 0x0e, 0x43, 0x7b, 0x6c, 0x2c, 0xcb, 0x51, - 0x3e, 0xe7, 0x2c, 0x30, 0xce, 0x13, 0x03, 0x1a, 0x9b, 0x5e, 0x60, 0x87, 0xe3, 0x27, 0xd4, 0x11, - 0x2c, 0x34, 0x5c, 0xa9, 0x79, 0x14, 0xab, 0x01, 0x2a, 0x2d, 0x06, 0x81, 0x77, 0x6f, 0x6a, 0xe8, - 0x10, 0x37, 0x63, 0x12, 0xeb, 0x93, 0x73, 0xb0, 0xb0, 0x3f, 0xb4, 0x43, 0x4e, 0xd3, 0xdc, 0x47, - 0xd7, 0x9f, 0x00, 0x24, 0x2e, 0x58, 0x0e, 0x87, 0x2d, 0x75, 0x1f, 0x70, 0x8d, 0x39, 0x94, 0x1e, - 0x23, 0x72, 0xd6, 0xb9, 0x18, 0xda, 0x0e, 0xd9, 0x70, 0x28, 0xa1, 0x7c, 0xcc, 0x87, 0x10, 0x75, - 0x8d, 0xc2, 0xc6, 0x1f, 0x4b, 0xb0, 0xf8, 0x10, 0x0f, 0xbe, 0x32, 0xbe, 0x7d, 0x1a, 0x3e, 0xf7, - 0x1c, 0x4a, 0x1c, 0x68, 0xa4, 0x9f, 0x25, 0x48, 0xf6, 0xb5, 0x3e, 0xe3, 0xe5, 0x62, 0xe5, 0xad, - 0x57, 0x55, 0x38, 0xf5, 0x21, 0xeb, 0xcc, 0x91, 0xef, 0x41, 0x2d, 0x2e, 0x61, 0x93, 0xec, 0xff, - 0x14, 0xa6, 0x4b, 0xdc, 0x67, 0x11, 0xdf, 0x83, 0x7a, 0xaa, 0xee, 0x4b, 0xb2, 0x39, 0x8f, 0xd7, - 0x9d, 0x57, 0xd6, 0x5e, 0x4d, 0x18, 0x8f, 0x41, 0xa1, 0x91, 0x2e, 0xa9, 0x9e, 0xa0, 0xa7, 0x8c, - 0x5a, 0xee, 0xca, 0xb5, 0x19, 0x28, 0xe3, 0x61, 0x8e, 0xa0, 0x39, 0x91, 0xa8, 0x93, 0x6b, 0x33, - 0xd7, 0x1f, 0x57, 0xae, 0xcf, 0x42, 0x1a, 0x8f, 0xd4, 0x07, 0x48, 0xf2, 0x7e, 0xf2, 0xf6, 0x49, - 0x9b, 0x92, 0x71, 0x31, 0x38, 0xe3, 0x40, 0x7b, 0x50, 0xc2, 0x78, 0x44, 0xb2, 0x23, 0x4f, 0x3a, - 0x76, 0xad, 0x74, 0x4e, 0x23, 0x89, 0x24, 0x6e, 0x7e, 0xf4, 0xc5, 0xff, 0xf5, 0x3d, 0x71, 0x34, - 0xea, 0xad, 0x3b, 0x6c, 0x70, 0xe3, 0xa5, 0xe7, 0xfb, 0xde, 0x4b, 0x41, 0x9d, 0xa3, 0x1b, 0x8a, - 0xf9, 0x7f, 0x15, 0xdb, 0x0d, 0x87, 0x85, 0xfa, 0x0f, 0xaf, 0x1b, 0x0a, 0x19, 0xf6, 0x7a, 0x65, - 0x6c, 0xbf, 0xf7, 0xcf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xce, 0x03, 0xc7, 0x5b, 0x24, 0x26, 0x00, - 0x00, + 0xca, 0x5a, 0x01, 0x09, 0x02, 0xe4, 0x92, 0x43, 0x0e, 0x39, 0xe4, 0x94, 0x5b, 0x8e, 0xb9, 0x25, + 0x08, 0x7c, 0xc9, 0x7f, 0x90, 0x20, 0x7f, 0x44, 0x6e, 0x41, 0x4e, 0x39, 0xe6, 0x1a, 0x74, 0x75, + 0xcf, 0x63, 0x97, 0x43, 0x6a, 0x19, 0x18, 0x71, 0x9c, 0xdb, 0xf4, 0xaf, 0xab, 0xaa, 0xbb, 0xab, + 0xab, 0xab, 0xaa, 0xab, 0x07, 0x1a, 0x3d, 0xdb, 0x79, 0x36, 0x1a, 0xae, 0x0f, 0x43, 0x26, 0x18, + 0x59, 0x1c, 0x78, 0xfe, 0xf3, 0x11, 0x57, 0xad, 0x75, 0xd5, 0xb5, 0xf2, 0x5a, 0x9f, 0xb1, 0xbe, + 0x4f, 0x6f, 0x20, 0xd8, 0x1b, 0x1d, 0xde, 0xe0, 0x22, 0x1c, 0x39, 0x42, 0x11, 0x75, 0xfe, 0x9a, + 0x83, 0x5a, 0x37, 0x70, 0xe9, 0x8b, 0x6e, 0x70, 0xc8, 0xc8, 0x25, 0x80, 0x43, 0x8f, 0xfa, 0xae, + 0x15, 0xd8, 0x03, 0xda, 0xce, 0xad, 0xe6, 0xd6, 0x6a, 0x66, 0x0d, 0x91, 0x5d, 0x7b, 0x40, 0x65, + 0xb7, 0x27, 0x69, 0x55, 0x77, 0x5e, 0x75, 0x23, 0x32, 0xd9, 0x2d, 0xc6, 0x43, 0xda, 0x2e, 0xa4, + 0xba, 0x0f, 0xc6, 0x43, 0x4a, 0x36, 0xa1, 0x3c, 0xb4, 0x43, 0x7b, 0xc0, 0xdb, 0xc5, 0xd5, 0xc2, + 0x5a, 0x7d, 0xe3, 0xfa, 0x7a, 0xc6, 0x74, 0xd7, 0xe3, 0xc9, 0xac, 0xef, 0x21, 0xf1, 0x4e, 0x20, + 0xc2, 0xb1, 0xa9, 0x39, 0x57, 0x3e, 0x82, 0x7a, 0x0a, 0x26, 0x06, 0x14, 0x9e, 0xd1, 0xb1, 0x9e, + 0xa8, 0xfc, 0x24, 0x4b, 0x50, 0x7a, 0x6e, 0xfb, 0xa3, 0x68, 0x76, 0xaa, 0x71, 0x2b, 0xff, 0x61, + 0xae, 0xf3, 0xf3, 0x2a, 0x2c, 0x6d, 0x31, 0xdf, 0xa7, 0x8e, 0xf0, 0x58, 0xb0, 0x89, 0xa3, 0xe1, + 0xa2, 0x5b, 0x90, 0xf7, 0x5c, 0x2d, 0x23, 0xef, 0xb9, 0xe4, 0x2e, 0x00, 0x17, 0xb6, 0xa0, 0x96, + 0xc3, 0x5c, 0x25, 0xa7, 0xb5, 0xb1, 0x96, 0x39, 0x57, 0x25, 0xe4, 0xc0, 0xe6, 0xcf, 0xf6, 0x25, + 0xc3, 0x16, 0x73, 0xa9, 0x59, 0xe3, 0xd1, 0x27, 0xe9, 0x40, 0x83, 0x86, 0x21, 0x0b, 0x1f, 0x52, + 0xce, 0xed, 0x7e, 0xa4, 0x91, 0x09, 0x4c, 0xea, 0x8c, 0x0b, 0x3b, 0x14, 0x96, 0xf0, 0x06, 0xb4, + 0x5d, 0x5c, 0xcd, 0xad, 0x15, 0x50, 0x44, 0x28, 0x0e, 0xbc, 0x01, 0x25, 0x17, 0xa0, 0x4a, 0x03, + 0x57, 0x75, 0x96, 0xb0, 0xb3, 0x42, 0x03, 0x17, 0xbb, 0x56, 0xa0, 0x3a, 0x0c, 0x59, 0x3f, 0xa4, + 0x9c, 0xb7, 0xcb, 0xab, 0xb9, 0xb5, 0x92, 0x19, 0xb7, 0xc9, 0xff, 0x40, 0xd3, 0x89, 0x97, 0x6a, + 0x79, 0x6e, 0xbb, 0x82, 0xbc, 0x8d, 0x04, 0xec, 0xba, 0xe4, 0x3c, 0x54, 0xdc, 0x9e, 0xda, 0xca, + 0x2a, 0xce, 0xac, 0xec, 0xf6, 0x70, 0x1f, 0xdf, 0x82, 0xf9, 0x14, 0x37, 0x12, 0xd4, 0x90, 0xa0, + 0x95, 0xc0, 0x48, 0xf8, 0x31, 0x94, 0xb9, 0x73, 0x44, 0x07, 0x76, 0x1b, 0x56, 0x73, 0x6b, 0xf5, + 0x8d, 0x37, 0x33, 0xb5, 0x94, 0x28, 0x7d, 0x1f, 0x89, 0x4d, 0xcd, 0x84, 0x6b, 0x3f, 0xb2, 0x43, + 0x97, 0x5b, 0xc1, 0x68, 0xd0, 0xae, 0xe3, 0x1a, 0x6a, 0x0a, 0xd9, 0x1d, 0x0d, 0x88, 0x09, 0x0b, + 0x0e, 0x0b, 0xb8, 0xc7, 0x05, 0x0d, 0x9c, 0xb1, 0xe5, 0xd3, 0xe7, 0xd4, 0x6f, 0x37, 0x70, 0x3b, + 0x4e, 0x1a, 0x28, 0xa6, 0x7e, 0x20, 0x89, 0x4d, 0xc3, 0x99, 0x42, 0xc8, 0x63, 0x58, 0x18, 0xda, + 0xa1, 0xf0, 0x70, 0x65, 0x8a, 0x8d, 0xb7, 0x9b, 0x68, 0x8e, 0xd9, 0x5b, 0xbc, 0x17, 0x51, 0x27, + 0x06, 0x63, 0x1a, 0xc3, 0x49, 0x90, 0x93, 0x6b, 0x60, 0x28, 0x7a, 0xdc, 0x29, 0x2e, 0xec, 0xc1, + 0xb0, 0xdd, 0x5a, 0xcd, 0xad, 0x15, 0xcd, 0x79, 0x85, 0x1f, 0x44, 0x30, 0x21, 0x50, 0xe4, 0xde, + 0x4b, 0xda, 0x9e, 0xc7, 0x1d, 0xc1, 0x6f, 0x72, 0x11, 0x6a, 0x47, 0x36, 0xb7, 0xf0, 0xa8, 0xb4, + 0x8d, 0xd5, 0xdc, 0x5a, 0xd5, 0xac, 0x1e, 0xd9, 0x1c, 0x8f, 0x02, 0xf9, 0x14, 0xea, 0xea, 0x54, + 0x79, 0xc1, 0x21, 0xe3, 0xed, 0x05, 0x9c, 0xec, 0xeb, 0xa7, 0x9f, 0x1d, 0x53, 0x1d, 0x44, 0xf9, + 0xc9, 0xa5, 0x9a, 0x7d, 0x66, 0xbb, 0x16, 0x1a, 0x66, 0x9b, 0xa8, 0x63, 0x29, 0x11, 0x34, 0x5a, + 0x72, 0x0b, 0x2e, 0xe8, 0xb9, 0x0f, 0x8f, 0xc6, 0xdc, 0x73, 0x6c, 0x3f, 0xb5, 0x88, 0x45, 0x5c, + 0xc4, 0x79, 0x45, 0xb0, 0xa7, 0xfb, 0x93, 0xc5, 0x84, 0xb0, 0xe8, 0x1c, 0xd9, 0x41, 0x40, 0x7d, + 0xcb, 0x39, 0xa2, 0xce, 0xb3, 0x21, 0xf3, 0x02, 0xc1, 0xdb, 0x4b, 0x38, 0xc7, 0xdb, 0xaf, 0xb0, + 0x86, 0x44, 0xa3, 0xeb, 0x5b, 0x4a, 0xc8, 0x56, 0x22, 0x43, 0x1d, 0x7b, 0xe2, 0x1c, 0xeb, 0x58, + 0xd9, 0x81, 0xf3, 0x27, 0x90, 0x9f, 0xc9, 0x1d, 0xfc, 0x2c, 0x0f, 0x8b, 0x19, 0x9b, 0x4b, 0xae, + 0x40, 0x23, 0xb1, 0x10, 0xed, 0x17, 0x0a, 0x66, 0x3d, 0xc6, 0xba, 0x2e, 0x79, 0x13, 0x5a, 0x09, + 0x49, 0xca, 0x15, 0x36, 0x63, 0x14, 0x4f, 0xc7, 0xb1, 0x43, 0x58, 0xc8, 0x38, 0x84, 0x8f, 0x60, + 0x9e, 0xd3, 0xfe, 0x80, 0x06, 0x22, 0x36, 0x47, 0xe5, 0x1d, 0xaf, 0x66, 0x6a, 0x6f, 0x5f, 0xd1, + 0xa6, 0x8c, 0xb1, 0xc5, 0xd3, 0x10, 0x8f, 0xed, 0xab, 0x94, 0xb2, 0xaf, 0x49, 0x0b, 0x28, 0x4f, + 0x59, 0x40, 0xe7, 0xd7, 0x05, 0x58, 0x38, 0x26, 0x18, 0x4f, 0xa7, 0x9e, 0x59, 0xac, 0x86, 0x9a, + 0x46, 0xba, 0xee, 0xf1, 0xd5, 0xe5, 0x33, 0x56, 0x37, 0xad, 0xcc, 0xc2, 0x71, 0x65, 0xbe, 0x0e, + 0xf5, 0x60, 0x34, 0xb0, 0xd8, 0xa1, 0x15, 0xb2, 0x2f, 0x79, 0xe4, 0x01, 0x83, 0xd1, 0xe0, 0xd1, + 0xa1, 0xc9, 0xbe, 0xe4, 0xe4, 0x16, 0x54, 0x7a, 0x5e, 0xe0, 0xb3, 0x3e, 0x6f, 0x97, 0x50, 0x31, + 0xab, 0x99, 0x8a, 0xb9, 0x23, 0x83, 0xd4, 0x26, 0x12, 0x9a, 0x11, 0x03, 0xf9, 0x04, 0xd0, 0x1b, + 0x73, 0xe4, 0x2e, 0xcf, 0xc8, 0x9d, 0xb0, 0x48, 0x7e, 0x97, 0xfa, 0xc2, 0x46, 0xfe, 0xca, 0xac, + 0xfc, 0x31, 0x4b, 0xbc, 0x17, 0xd5, 0xd4, 0x5e, 0x5c, 0x80, 0x6a, 0x3f, 0x64, 0xa3, 0xa1, 0x54, + 0x47, 0x4d, 0x79, 0x74, 0x6c, 0x77, 0x5d, 0xe9, 0xd1, 0x95, 0x3c, 0xea, 0xa2, 0x43, 0xad, 0x9a, + 0x71, 0xbb, 0xf3, 0xfb, 0x02, 0xc0, 0x7f, 0x77, 0xcc, 0x22, 0x50, 0xc4, 0xb3, 0x54, 0xc1, 0x11, + 0xf1, 0x3b, 0xd3, 0xaf, 0x56, 0xb3, 0xfd, 0xea, 0x53, 0x20, 0x29, 0x7b, 0x8c, 0xce, 0x52, 0x0d, + 0x37, 0xed, 0xda, 0xcc, 0x9e, 0xc8, 0x5c, 0x70, 0xa6, 0xd0, 0x64, 0x17, 0x21, 0xb5, 0x8b, 0x6f, + 0x42, 0x4b, 0x89, 0xb4, 0x9e, 0xd3, 0x90, 0x7b, 0x2c, 0xc0, 0xf0, 0x55, 0x33, 0x9b, 0x0a, 0x7d, + 0xa2, 0xc0, 0xce, 0x77, 0xe1, 0x42, 0x32, 0x0a, 0x46, 0xa0, 0xd4, 0x1e, 0x7e, 0x0a, 0x25, 0xe5, + 0xd2, 0x73, 0x67, 0x9d, 0xa4, 0xe2, 0xeb, 0x7c, 0x01, 0xed, 0xd8, 0x83, 0x4d, 0x0b, 0xff, 0x64, + 0x52, 0xf8, 0xec, 0xc1, 0x4d, 0xcb, 0x7e, 0x02, 0xcb, 0xda, 0x25, 0x4c, 0x4b, 0xfe, 0xff, 0x49, + 0xc9, 0xb3, 0xfa, 0x29, 0x2d, 0xf7, 0xa7, 0x05, 0x58, 0xdc, 0x0a, 0xa9, 0x2d, 0xa8, 0xea, 0x33, + 0xe9, 0x0f, 0x47, 0x94, 0x0b, 0xf2, 0x1a, 0xd4, 0x42, 0xf5, 0xd9, 0x8d, 0xec, 0x3a, 0x01, 0xc8, + 0x65, 0xa8, 0x6b, 0x3b, 0x48, 0xb9, 0x5b, 0x50, 0xd0, 0xae, 0x36, 0x94, 0xa9, 0x94, 0x85, 0xb7, + 0x0b, 0xab, 0x85, 0xb5, 0x9a, 0x39, 0x3f, 0x99, 0xb3, 0x70, 0x19, 0x12, 0x6c, 0x3e, 0x0e, 0x1c, + 0x34, 0xdc, 0xaa, 0xa9, 0x1a, 0xe4, 0x63, 0x68, 0xb9, 0x3d, 0x2b, 0xa1, 0xe5, 0x68, 0xba, 0xf5, + 0x8d, 0xe5, 0x75, 0x95, 0x3e, 0xaf, 0x47, 0xe9, 0xf3, 0xfa, 0x13, 0x19, 0x42, 0xcc, 0xa6, 0xdb, + 0x4b, 0xb6, 0x06, 0x85, 0x1e, 0xb2, 0xd0, 0x51, 0xce, 0xb5, 0x6a, 0xaa, 0x86, 0x8c, 0xeb, 0x03, + 0x2a, 0x6c, 0x8b, 0x05, 0xfe, 0x18, 0xed, 0xba, 0x6a, 0x56, 0x25, 0xf0, 0x28, 0xf0, 0xc7, 0xe4, + 0x2a, 0xcc, 0xf7, 0x1d, 0x6b, 0x68, 0x8f, 0x38, 0xb5, 0x68, 0x60, 0xf7, 0x7c, 0xe5, 0x27, 0xaa, + 0x66, 0xb3, 0xef, 0xec, 0x49, 0x74, 0x07, 0x41, 0xb2, 0x06, 0x46, 0x4c, 0xc7, 0xa9, 0xc3, 0x02, + 0x97, 0xa3, 0xe3, 0x28, 0x99, 0x2d, 0x4d, 0xb8, 0xaf, 0xd0, 0x09, 0x4a, 0xdb, 0x75, 0xf1, 0x94, + 0x81, 0x4a, 0xdc, 0x34, 0xe5, 0x6d, 0x85, 0x76, 0x7e, 0x9b, 0x03, 0x92, 0xda, 0x1b, 0xca, 0x87, + 0x2c, 0xe0, 0xf4, 0x15, 0x9b, 0xf0, 0x01, 0x14, 0x53, 0xde, 0xe5, 0x4a, 0xe6, 0xbe, 0x47, 0xa2, + 0xd0, 0xad, 0x20, 0xb9, 0x0c, 0xca, 0x03, 0xde, 0xd7, 0x8e, 0x44, 0x7e, 0x92, 0xf7, 0xa0, 0xe8, + 0xda, 0xc2, 0xc6, 0x0d, 0xa8, 0x6f, 0x5c, 0x3e, 0xc5, 0x4d, 0xe1, 0xec, 0x90, 0xb8, 0xf3, 0xa7, + 0x1c, 0x18, 0x77, 0xa9, 0xf8, 0x5a, 0xad, 0xe6, 0x22, 0xd4, 0x34, 0x81, 0x8e, 0x4d, 0xb5, 0xc8, + 0xe3, 0x6a, 0xee, 0x91, 0xf3, 0x8c, 0x0a, 0xc5, 0x5d, 0xd4, 0xdc, 0x08, 0x21, 0x37, 0x81, 0xe2, + 0xd0, 0x16, 0x47, 0x68, 0x28, 0x35, 0x13, 0xbf, 0xa5, 0x5f, 0xf8, 0xd2, 0x13, 0x47, 0x6c, 0x24, + 0x2c, 0x97, 0x0a, 0xdb, 0xf3, 0xb5, 0x41, 0x34, 0x35, 0xba, 0x8d, 0x60, 0xe7, 0x3b, 0x40, 0x1e, + 0x78, 0x3c, 0x8a, 0xd9, 0xb3, 0xad, 0x26, 0x23, 0x2b, 0xcf, 0x67, 0x65, 0xe5, 0x9d, 0xdf, 0xe5, + 0x60, 0x71, 0x42, 0xfa, 0x37, 0xb5, 0xbb, 0x85, 0xd9, 0x77, 0xf7, 0x00, 0x16, 0xb7, 0xa9, 0x4f, + 0xbf, 0x5e, 0xaf, 0xd0, 0xf9, 0x11, 0x2c, 0x4d, 0x4a, 0xfd, 0xb7, 0x6a, 0xa2, 0xf3, 0x97, 0x12, + 0x2c, 0x99, 0x94, 0x0b, 0x16, 0x7e, 0x63, 0xce, 0xee, 0x6d, 0x48, 0x05, 0x34, 0x8b, 0x8f, 0x0e, + 0x0f, 0xbd, 0x17, 0xda, 0x94, 0x53, 0x32, 0xf6, 0x11, 0x27, 0x6c, 0x22, 0x84, 0x86, 0x54, 0x49, + 0x56, 0x59, 0xd7, 0x67, 0x27, 0xa9, 0xe1, 0xd8, 0xea, 0x52, 0x21, 0xcb, 0x54, 0x22, 0x54, 0x2e, + 0x9f, 0x9a, 0x88, 0xc6, 0x13, 0x57, 0x5c, 0x4e, 0xbb, 0xe2, 0xa9, 0x83, 0x57, 0x39, 0xf1, 0xe0, + 0x55, 0x53, 0x07, 0xef, 0xb8, 0xff, 0xae, 0x9d, 0xc5, 0x7f, 0xaf, 0x40, 0xec, 0x98, 0xa3, 0xd4, + 0x2b, 0x76, 0xd4, 0x1d, 0x68, 0x84, 0x6a, 0x9d, 0x78, 0xbf, 0xc2, 0x48, 0x5f, 0x35, 0x27, 0x30, + 0x49, 0x23, 0xdd, 0xeb, 0x48, 0x30, 0x45, 0xd3, 0x50, 0x34, 0x69, 0x8c, 0xbc, 0x03, 0x8b, 0x6e, + 0xc8, 0x86, 0x3b, 0x2f, 0x3c, 0x2e, 0x92, 0xb1, 0xdb, 0x4d, 0x24, 0xcd, 0xea, 0x22, 0x57, 0xa1, + 0x15, 0xc3, 0x4a, 0x6e, 0x0b, 0x89, 0xa7, 0x50, 0xb2, 0x01, 0x4b, 0xfc, 0x99, 0x37, 0x54, 0x71, + 0x35, 0x25, 0x7a, 0x1e, 0xa9, 0x33, 0xfb, 0x56, 0xb6, 0x61, 0x39, 0x7b, 0xa3, 0xce, 0x74, 0x8b, + 0xfa, 0x2a, 0x1f, 0x9b, 0x78, 0x9c, 0x4c, 0xc8, 0xdc, 0xf2, 0x58, 0x82, 0x7a, 0x2f, 0x23, 0x41, + 0xbd, 0x76, 0x9a, 0x4d, 0xfd, 0x07, 0x66, 0xa8, 0x5d, 0xc0, 0x9b, 0x8b, 0x4e, 0x2e, 0xd1, 0x30, + 0xcf, 0x92, 0x59, 0x81, 0x64, 0x56, 0xed, 0xce, 0x57, 0x65, 0x38, 0xa7, 0x17, 0x9a, 0xec, 0xc2, + 0xb7, 0x5a, 0x71, 0x9f, 0x43, 0x5d, 0x9e, 0xbe, 0x48, 0x39, 0x65, 0x54, 0xce, 0x19, 0x72, 0x5a, + 0x90, 0xdc, 0xaa, 0x4d, 0xde, 0x87, 0x65, 0x61, 0x87, 0x7d, 0x2a, 0xac, 0xe9, 0x88, 0xa7, 0x9c, + 0xc1, 0x92, 0xea, 0xdd, 0x9a, 0xac, 0x46, 0xd9, 0x70, 0x3e, 0xb9, 0x6c, 0xea, 0xd3, 0x69, 0x09, + 0x9b, 0x3f, 0xe3, 0xed, 0xea, 0x29, 0x19, 0x76, 0x96, 0xf9, 0x9a, 0xe7, 0x62, 0x49, 0x29, 0xad, + 0x62, 0x5d, 0x4d, 0x0b, 0x76, 0x2d, 0xbc, 0x13, 0xa8, 0x1b, 0x5c, 0xe4, 0x0b, 0xdc, 0x7d, 0x79, + 0x37, 0xb8, 0x0a, 0xf3, 0x82, 0xc5, 0x13, 0x48, 0x5d, 0x1d, 0x9a, 0x82, 0x69, 0x69, 0x48, 0x97, + 0x36, 0xb5, 0xfa, 0x94, 0xa9, 0xbd, 0x01, 0x2d, 0xad, 0x81, 0xa8, 0x44, 0xd7, 0x50, 0xbb, 0xa5, + 0xd0, 0x6d, 0x55, 0xa8, 0x4b, 0x7b, 0xad, 0xe6, 0x2b, 0xbc, 0x56, 0x6b, 0x06, 0xaf, 0x35, 0x3f, + 0xbb, 0xd7, 0x32, 0xce, 0xe2, 0xb5, 0x16, 0xce, 0xe4, 0xb5, 0xc8, 0xc9, 0x5e, 0xab, 0xf3, 0xab, + 0x02, 0x2c, 0x4c, 0x04, 0x9d, 0x6f, 0xf5, 0x99, 0x71, 0xa1, 0x3d, 0x11, 0x70, 0xd3, 0x26, 0x5b, + 0x3e, 0xa5, 0x46, 0x9e, 0xe9, 0x39, 0xcc, 0xe5, 0x74, 0x80, 0x3d, 0xcd, 0x68, 0x2b, 0xb3, 0x19, + 0x6d, 0xf5, 0x55, 0x46, 0x5b, 0x9b, 0x34, 0xda, 0xce, 0x1f, 0x72, 0xb1, 0x53, 0xfb, 0x46, 0x12, + 0x2e, 0x72, 0x6b, 0xe2, 0x62, 0x71, 0xf5, 0xd5, 0x29, 0x0b, 0xea, 0x4d, 0x65, 0xa0, 0x77, 0x60, + 0xf9, 0x2e, 0x15, 0xd1, 0x52, 0xa5, 0x01, 0xcc, 0x96, 0xad, 0x29, 0xdb, 0xcb, 0x47, 0xb6, 0xd7, + 0xf9, 0x3e, 0xd4, 0x53, 0xd5, 0x20, 0xd2, 0x86, 0x0a, 0xbe, 0x9f, 0x74, 0xb7, 0x75, 0x09, 0x2d, + 0x6a, 0x92, 0x0f, 0x92, 0xc2, 0x56, 0x1e, 0xf7, 0xfa, 0x62, 0x76, 0xaa, 0x3c, 0x59, 0xd3, 0xea, + 0xfc, 0x26, 0x07, 0x65, 0x2d, 0xfb, 0x32, 0xd4, 0x69, 0x20, 0x42, 0x8f, 0xaa, 0x02, 0xba, 0x92, + 0x0f, 0x1a, 0xda, 0x1d, 0x0d, 0xe4, 0x6d, 0x24, 0xae, 0x9b, 0x58, 0x87, 0x21, 0x1b, 0xe0, 0x3c, + 0x8b, 0x66, 0x33, 0x46, 0xef, 0x84, 0x6c, 0x40, 0xae, 0x40, 0x23, 0x21, 0x13, 0x0c, 0x35, 0x5a, + 0x34, 0xeb, 0x31, 0x76, 0xc0, 0xa4, 0x11, 0xfb, 0xac, 0x6f, 0x61, 0xda, 0xa5, 0xd2, 0xc7, 0x8a, + 0xcf, 0xfa, 0x7b, 0x32, 0xf3, 0xd2, 0x5d, 0xa9, 0xa2, 0xa3, 0xec, 0x92, 0xc6, 0xd2, 0xb9, 0x09, + 0x8d, 0xfb, 0x74, 0x8c, 0x09, 0xd7, 0x9e, 0xed, 0x85, 0xb3, 0x66, 0x16, 0x9d, 0x7f, 0xe4, 0x00, + 0x90, 0x0b, 0x35, 0x49, 0x2e, 0x41, 0xad, 0xc7, 0x98, 0x6f, 0xe1, 0xde, 0x4a, 0xe6, 0xea, 0xbd, + 0x39, 0xb3, 0x2a, 0xa1, 0x6d, 0x5b, 0xd8, 0xe4, 0x22, 0x54, 0xbd, 0x40, 0xa8, 0x5e, 0x29, 0xa6, + 0x74, 0x6f, 0xce, 0xac, 0x78, 0x81, 0xc0, 0xce, 0x4b, 0x50, 0xf3, 0x59, 0xd0, 0x57, 0xbd, 0x58, + 0x7e, 0x94, 0xbc, 0x12, 0xc2, 0xee, 0xcb, 0x00, 0x87, 0x3e, 0xb3, 0x35, 0xb7, 0x5c, 0x59, 0xfe, + 0xde, 0x9c, 0x59, 0x43, 0x0c, 0x09, 0xae, 0x40, 0xdd, 0x65, 0xa3, 0x9e, 0x4f, 0x15, 0x85, 0x5c, + 0x60, 0xee, 0xde, 0x9c, 0x09, 0x0a, 0x8c, 0x48, 0xb8, 0x08, 0xbd, 0x68, 0x10, 0x2c, 0xaf, 0x4a, + 0x12, 0x05, 0x46, 0xc3, 0xf4, 0xc6, 0x82, 0x72, 0x45, 0x21, 0xcf, 0x5f, 0x43, 0x0e, 0x83, 0x98, + 0x24, 0xd8, 0x2c, 0x2b, 0xcb, 0xed, 0xfc, 0xad, 0xa8, 0xcd, 0x47, 0x3d, 0x95, 0x9c, 0x62, 0x3e, + 0x51, 0xb9, 0x2c, 0x9f, 0x2a, 0x97, 0xbd, 0x01, 0x2d, 0x8f, 0x5b, 0xc3, 0xd0, 0x1b, 0xd8, 0xe1, + 0xd8, 0x92, 0xaa, 0x2e, 0x28, 0x8f, 0xee, 0xf1, 0x3d, 0x05, 0xde, 0xa7, 0x63, 0xb2, 0x0a, 0x75, + 0x97, 0x72, 0x27, 0xf4, 0x86, 0xe8, 0x6e, 0xd5, 0x76, 0xa6, 0x21, 0x72, 0x0b, 0x6a, 0x72, 0x36, + 0xea, 0x1d, 0xaf, 0x84, 0xa7, 0xf2, 0x52, 0xa6, 0x71, 0xca, 0xb9, 0x1f, 0x8c, 0x87, 0xd4, 0xac, + 0xba, 0xfa, 0x8b, 0x6c, 0x42, 0x5d, 0xb2, 0x59, 0xfa, 0xa9, 0x4f, 0xb9, 0xb1, 0xec, 0x33, 0x9d, + 0xb6, 0x0d, 0x13, 0x24, 0x97, 0x7a, 0xdb, 0x23, 0xdb, 0xd0, 0x50, 0x4f, 0x1e, 0x5a, 0x48, 0x65, + 0x56, 0x21, 0xea, 0xa5, 0x44, 0x4b, 0x59, 0x86, 0xb2, 0x2d, 0xc3, 0xd8, 0xb6, 0xae, 0xab, 0xe8, + 0x16, 0xf9, 0x00, 0x4a, 0xaa, 0x10, 0x5e, 0xc3, 0x95, 0x5d, 0x3e, 0xb9, 0xa2, 0xab, 0xdc, 0x80, + 0xa2, 0x26, 0x9f, 0x41, 0x83, 0xfa, 0x14, 0xeb, 0xe1, 0xa8, 0x17, 0x98, 0x45, 0x2f, 0x75, 0xcd, + 0x82, 0xaa, 0xd9, 0x86, 0xa6, 0x4b, 0x0f, 0xed, 0x91, 0x2f, 0x2c, 0x65, 0xf4, 0xf5, 0x53, 0x0a, + 0x20, 0x89, 0xfd, 0x9b, 0x0d, 0xcd, 0x85, 0x10, 0xbe, 0xb2, 0x72, 0xcb, 0x1d, 0x07, 0xf6, 0xc0, + 0x73, 0xf4, 0x45, 0xa3, 0xe6, 0xf1, 0x6d, 0x05, 0x90, 0x35, 0x30, 0xa4, 0x0d, 0xc4, 0x89, 0x90, + 0xb4, 0x02, 0x95, 0x1b, 0xb4, 0x3c, 0x1e, 0x27, 0x39, 0xf7, 0xe9, 0xb8, 0xf3, 0xe7, 0x1c, 0x18, + 0xd3, 0x6f, 0x73, 0xb1, 0x59, 0xe5, 0x52, 0x66, 0x35, 0x65, 0x30, 0xf9, 0xe3, 0x06, 0x93, 0xa8, + 0xba, 0x30, 0xa1, 0xea, 0x0f, 0xa1, 0x8c, 0xf6, 0x1a, 0x3d, 0x6a, 0x9c, 0x52, 0x3d, 0x8f, 0xde, + 0x06, 0x15, 0x3d, 0x79, 0x07, 0x96, 0x54, 0x51, 0x2c, 0x5a, 0xa9, 0x85, 0x1d, 0x68, 0x8d, 0x55, + 0x93, 0xa8, 0x3e, 0xbd, 0x66, 0xe4, 0xef, 0xb4, 0xa0, 0x81, 0x0f, 0x42, 0xda, 0x6d, 0x77, 0x9e, + 0x42, 0x53, 0xb7, 0x75, 0x10, 0x8a, 0xc2, 0x4c, 0xee, 0x5f, 0x0a, 0x33, 0xf9, 0xe4, 0x5e, 0xff, + 0x93, 0x1c, 0xd4, 0x1f, 0xf2, 0xfe, 0x1e, 0xe3, 0xa8, 0x4b, 0xe9, 0x3f, 0xa3, 0x57, 0xb0, 0x94, + 0xee, 0xea, 0x1a, 0xc3, 0x4c, 0x6d, 0x09, 0x4a, 0x03, 0xde, 0xef, 0x6e, 0xa3, 0x98, 0x86, 0xa9, + 0x1a, 0x98, 0xbf, 0xf1, 0xfe, 0xdd, 0x90, 0x8d, 0x86, 0x51, 0xf9, 0x29, 0x6a, 0xcb, 0xa8, 0x93, + 0xd4, 0xbc, 0x8b, 0xe8, 0x91, 0x13, 0xa0, 0x73, 0x1b, 0xe6, 0xf5, 0x23, 0x58, 0x3c, 0x8b, 0xac, + 0x9d, 0x93, 0xd1, 0x5a, 0xf7, 0xeb, 0x05, 0xc4, 0xed, 0xeb, 0x3f, 0x86, 0x46, 0x7a, 0xb5, 0xa4, + 0x0e, 0x95, 0xfd, 0x91, 0xe3, 0x50, 0xce, 0x8d, 0x39, 0x32, 0x0f, 0xf5, 0x5d, 0x26, 0xac, 0xfd, + 0xd1, 0x70, 0xc8, 0x42, 0x61, 0xe4, 0xc8, 0x02, 0x34, 0x77, 0x99, 0xb5, 0x47, 0xc3, 0x81, 0xc7, + 0xb9, 0xc7, 0x02, 0x23, 0x4f, 0xaa, 0x50, 0xbc, 0x63, 0x7b, 0xbe, 0x51, 0x20, 0x4b, 0x30, 0x8f, + 0x67, 0x8e, 0x0a, 0x1a, 0x5a, 0x3b, 0x32, 0x37, 0x32, 0x7e, 0x51, 0x20, 0x97, 0xa0, 0xad, 0xf7, + 0xc2, 0x7a, 0xd4, 0xfb, 0x01, 0x75, 0x84, 0x25, 0x45, 0xde, 0x61, 0xa3, 0xc0, 0x35, 0x7e, 0x59, + 0xb8, 0xfe, 0x02, 0x16, 0x33, 0xde, 0x22, 0x08, 0x81, 0xd6, 0xe6, 0xed, 0xad, 0xfb, 0x8f, 0xf7, + 0xac, 0xee, 0x6e, 0xf7, 0xa0, 0x7b, 0xfb, 0x81, 0x31, 0x47, 0x96, 0xc0, 0xd0, 0xd8, 0xce, 0xd3, + 0x9d, 0xad, 0xc7, 0x07, 0xdd, 0xdd, 0xbb, 0x46, 0x2e, 0x45, 0xb9, 0xff, 0x78, 0x6b, 0x6b, 0x67, + 0x7f, 0xdf, 0xc8, 0xcb, 0x79, 0x6b, 0xec, 0xce, 0xed, 0xee, 0x03, 0xa3, 0x90, 0x22, 0x3a, 0xe8, + 0x3e, 0xdc, 0x79, 0xf4, 0xf8, 0xc0, 0x28, 0x5e, 0x7f, 0x12, 0xdf, 0x59, 0x27, 0x87, 0xae, 0x43, + 0x25, 0x19, 0xb3, 0x09, 0xb5, 0xf4, 0x60, 0x52, 0x3b, 0xf1, 0x28, 0x72, 0xe5, 0x4a, 0x7c, 0x1d, + 0x2a, 0x89, 0xdc, 0xa7, 0xf2, 0x3c, 0x4d, 0x3d, 0x38, 0x03, 0x94, 0xf7, 0x45, 0xc8, 0x82, 0xbe, + 0x31, 0x87, 0x32, 0xa8, 0xd2, 0x1e, 0x0a, 0xdc, 0x94, 0xaa, 0xa0, 0xae, 0x91, 0x27, 0x2d, 0x80, + 0x9d, 0xe7, 0x34, 0x10, 0x23, 0xdb, 0xf7, 0xc7, 0x46, 0x41, 0xb6, 0xb7, 0x46, 0x5c, 0xb0, 0x81, + 0xf7, 0x92, 0xba, 0x46, 0xf1, 0xfa, 0xdf, 0x73, 0x50, 0x8d, 0x7c, 0x8a, 0x1c, 0x7d, 0x97, 0x05, + 0xd4, 0x98, 0x93, 0x5f, 0x9b, 0x8c, 0xf9, 0x46, 0x4e, 0x7e, 0x75, 0x03, 0xf1, 0xa1, 0x91, 0x27, + 0x35, 0x28, 0x75, 0x03, 0xf1, 0xee, 0x4d, 0xa3, 0xa0, 0x3f, 0xdf, 0xdb, 0x30, 0x8a, 0xfa, 0xf3, + 0xe6, 0xfb, 0x46, 0x49, 0x7e, 0xde, 0x91, 0xe1, 0xcd, 0x00, 0x39, 0xb9, 0x6d, 0x8c, 0x63, 0x46, + 0x5d, 0x4f, 0xd4, 0x0b, 0xfa, 0xc6, 0x92, 0x9c, 0xdb, 0x13, 0x3b, 0xdc, 0x3a, 0xb2, 0x43, 0xe3, + 0x9c, 0xa4, 0xbf, 0x1d, 0x86, 0xf6, 0xd8, 0x58, 0x96, 0xa3, 0x7c, 0xce, 0x59, 0x60, 0x9c, 0x27, + 0x06, 0x34, 0x36, 0xbd, 0xc0, 0x0e, 0xc7, 0x4f, 0xa8, 0x23, 0x58, 0x68, 0xb8, 0x52, 0xf3, 0x28, + 0x56, 0x03, 0x54, 0x5a, 0x0c, 0x02, 0xef, 0xde, 0xd4, 0xd0, 0x21, 0x6e, 0xc6, 0x24, 0xd6, 0x27, + 0xe7, 0x60, 0x61, 0x7f, 0x68, 0x87, 0x9c, 0xa6, 0xb9, 0x8f, 0xae, 0x3f, 0x01, 0x48, 0x5c, 0xb0, + 0x1c, 0x0e, 0x5b, 0xea, 0x3e, 0xe0, 0x1a, 0x73, 0x28, 0x3d, 0x46, 0xe4, 0xac, 0x73, 0x31, 0xb4, + 0x1d, 0xb2, 0xe1, 0x50, 0x42, 0xf9, 0x98, 0x0f, 0x21, 0xea, 0x1a, 0x85, 0x8d, 0x3f, 0x96, 0x60, + 0xf1, 0x21, 0x1e, 0x7c, 0x65, 0x7c, 0xfb, 0x34, 0x7c, 0xee, 0x39, 0x94, 0x38, 0xd0, 0x48, 0x3f, + 0x4b, 0x90, 0xec, 0x6b, 0x7d, 0xc6, 0xcb, 0xc5, 0xca, 0x5b, 0xaf, 0xaa, 0x70, 0xea, 0x43, 0xd6, + 0x99, 0x23, 0xdf, 0x83, 0x5a, 0x5c, 0xc2, 0x26, 0xd9, 0xff, 0x30, 0x4c, 0x97, 0xb8, 0xcf, 0x22, + 0xbe, 0x07, 0xf5, 0x54, 0xdd, 0x97, 0x64, 0x73, 0x1e, 0xaf, 0x3b, 0xaf, 0xac, 0xbd, 0x9a, 0x30, + 0x1e, 0x83, 0x42, 0x23, 0x5d, 0x52, 0x3d, 0x41, 0x4f, 0x19, 0xb5, 0xdc, 0x95, 0x6b, 0x33, 0x50, + 0xc6, 0xc3, 0x1c, 0x41, 0x73, 0x22, 0x51, 0x27, 0xd7, 0x66, 0xae, 0x3f, 0xae, 0x5c, 0x9f, 0x85, + 0x34, 0x1e, 0xa9, 0x0f, 0x90, 0xe4, 0xfd, 0xe4, 0xed, 0x93, 0x36, 0x25, 0xe3, 0x62, 0x70, 0xc6, + 0x81, 0xf6, 0xa0, 0x84, 0xf1, 0x88, 0x64, 0x47, 0x9e, 0x74, 0xec, 0x5a, 0xe9, 0x9c, 0x46, 0x12, + 0x49, 0xdc, 0xfc, 0xe8, 0x8b, 0xff, 0xeb, 0x7b, 0xe2, 0x68, 0xd4, 0x5b, 0x77, 0xd8, 0xe0, 0xc6, + 0x4b, 0xcf, 0xf7, 0xbd, 0x97, 0x82, 0x3a, 0x47, 0x37, 0x14, 0xf3, 0xff, 0x2a, 0xb6, 0x1b, 0x0e, + 0x0b, 0xf5, 0xdf, 0x5f, 0x37, 0x14, 0x32, 0xec, 0xf5, 0xca, 0xd8, 0x7e, 0xef, 0x9f, 0x01, 0x00, + 0x00, 0xff, 0xff, 0xd4, 0x85, 0x93, 0x46, 0x40, 0x26, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/core/storage/minio_chunk_manager.go b/core/storage/minio_chunk_manager.go index cb6f9b5e..42e02fd3 100644 --- a/core/storage/minio_chunk_manager.go +++ b/core/storage/minio_chunk_manager.go @@ -425,7 +425,7 @@ func (mcm *MinioChunkManager) ListWithPrefix(ctx context.Context, bucketName str for object := range objects { if object.Err != nil { - log.Warn("failed to list with prefix", zap.String("prefix", prefix), zap.Error(object.Err)) + log.Warn("failed to list with prefix", zap.String("bucket", bucketName), zap.String("prefix", prefix), zap.Error(object.Err)) return nil, nil, object.Err } objectsKeys = append(objectsKeys, object.Key) @@ -437,7 +437,7 @@ func (mcm *MinioChunkManager) ListWithPrefix(ctx context.Context, bucketName str func (mcm *MinioChunkManager) Copy(ctx context.Context, fromBucketName string, toBucketName string, fromPath string, toPath string) error { objectkeys, _, err := mcm.ListWithPrefix(ctx, fromBucketName, fromPath, true) if err != nil { - log.Warn("listWithPrefix error", zap.String("prefix", fromPath), zap.Error(err)) + log.Warn("listWithPrefix error", zap.String("bucket", fromBucketName), zap.String("prefix", fromPath), zap.Error(err)) return err } for _, objectkey := range objectkeys {