diff --git a/core/backup_context.go b/core/backup_context.go index d382256..7866270 100644 --- a/core/backup_context.go +++ b/core/backup_context.go @@ -26,6 +26,8 @@ const ( COLLECTION_RENAME_SUFFIX = "COLLECTION_RENAME_SUFFIX" RPS = 1000 BackupSegmentGroupMaxSizeInMB = 256 + + GC_Warn_Message = "This warn won't fail the backup process. Pause GC can protect data not to be GCed during backup, it is necessary to backup very large data(cost more than a hour)." ) // makes sure BackupContext implements `Backup` @@ -270,15 +272,15 @@ func (b *BackupContext) GetBackup(ctx context.Context, request *backuppb.GetBack zap.String("backupName", request.GetBackupName()), zap.String("backupId", request.GetBackupId()), zap.String("bucketName", request.GetBucketName()), - zap.String("path", request.GetPath()), - zap.Any("resp", resp)) + zap.String("path", request.GetPath())) } else { log.Info("finish GetBackupRequest", zap.String("requestId", request.GetRequestId()), zap.String("backupName", request.GetBackupName()), zap.String("backupId", request.GetBackupId()), zap.String("bucketName", request.GetBucketName()), - zap.String("path", request.GetPath())) + zap.String("path", request.GetPath()), + zap.Any("resp", resp)) } return resp diff --git a/core/backup_impl_create_backup.go b/core/backup_impl_create_backup.go index 34fb247..244e467 100644 --- a/core/backup_impl_create_backup.go +++ b/core/backup_impl_create_backup.go @@ -395,38 +395,45 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, backupInfo return err } - segmentIDsEntitiesBeforeFlush := lo.Map(segmentEntitiesBeforeFlush, func(segment *entity.Segment, _ int) int64 { return segment.ID }) - segmentIDsEntitiesAfterFlush := lo.Map(segmentEntitiesAfterFlush, func(segment *entity.Segment, _ int) int64 { return segment.ID }) + segmentIDsBeforeFlush := lo.Map(segmentEntitiesBeforeFlush, func(segment *entity.Segment, _ int) int64 { return segment.ID }) + segmentIDsAfterFlush := lo.Map(segmentEntitiesAfterFlush, func(segment *entity.Segment, _ int) int64 { return segment.ID }) + newL0Segments := lo.Filter(segmentEntitiesAfterFlush, func(segment *entity.Segment, _ int) bool { + return !lo.Contains(segmentIDsBeforeFlush, segment.ID) && segment.NumRows == 0 + }) + newL0SegmentsIDs := lo.Map(newL0Segments, func(segment *entity.Segment, _ int) int64 { return segment.ID }) + log.Info("GetPersistentSegmentInfo after flush from milvus", zap.String("databaseName", collectionBackup.GetDbName()), zap.String("collectionName", collectionBackup.GetCollectionName()), - zap.Int("segmentNumBeforeFlush", len(segmentEntitiesBeforeFlush)), - zap.Int("segmentNumAfterFlush", len(segmentEntitiesAfterFlush)), - zap.Int64s("segmentsBeforeFlush", segmentIDsEntitiesBeforeFlush), - zap.Int64s("segmentsAfterFlush", segmentIDsEntitiesAfterFlush)) - segmentDict := utils.ArrayToMap(flushSegmentIDs) + zap.Int64s("segmentIDsBeforeFlush", segmentIDsBeforeFlush), + zap.Int64s("segmentIDsAfterFlush", segmentIDsAfterFlush), + zap.Int64s("newL0SegmentsIDs", newL0SegmentsIDs)) + + segmentEntities := segmentEntitiesBeforeFlush for _, seg := range segmentEntitiesAfterFlush { - sid := seg.ID - if _, ok := segmentDict[sid]; ok { - delete(segmentDict, sid) - unfilledSegments = append(unfilledSegments, seg) - } else { - log.Info("this may be new segments after flush, skip it", zap.Int64("id", sid)) + if !lo.Contains(segmentIDsBeforeFlush, seg.ID) { + segmentEntities = append(segmentEntities, seg) } } - for _, seg := range segmentEntitiesBeforeFlush { - sid := seg.ID - if _, ok := segmentDict[sid]; ok { - delete(segmentDict, sid) - unfilledSegments = append(unfilledSegments, seg) - //} else { - // log.Info("this may be old segments before flush, skip it", zap.Int64("id", sid)) + + // segmentIDs + unfilledSegmentIDs := make([]int64, 0) + // union of (intersection of BeforeFlushIDs and AfterFlushIDs) and flushSegmentIDs + for _, seg := range segmentEntitiesAfterFlush { + if lo.Contains(segmentIDsBeforeFlush, seg.ID) { + unfilledSegmentIDs = append(unfilledSegmentIDs, seg.ID) } } - if len(segmentDict) > 0 { - // very rare situation, segments return in flush doesn't exist in either segmentEntitiesBeforeFlush and segmentEntitiesAfterFlush - errorMsg := "Segment return in Flush not exist in GetPersistentSegmentInfo. segment ids: " + fmt.Sprint(utils.MapKeyArray(segmentDict)) - log.Warn(errorMsg) + for _, segID := range flushSegmentIDs { + if !lo.Contains(unfilledSegmentIDs, segID) { + unfilledSegmentIDs = append(unfilledSegmentIDs, segID) + } + } + unfilledSegmentIDs = append(unfilledSegmentIDs, newL0SegmentsIDs...) + for _, seg := range segmentEntities { + if lo.Contains(unfilledSegmentIDs, seg.ID) { + unfilledSegments = append(unfilledSegments, seg) + } } } else { // Flush @@ -483,29 +490,38 @@ func (b *BackupContext) backupCollectionPrepare(ctx context.Context, backupInfo } b.meta.AddPartition(partitionBackupInfo) } - b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setCollectionLoadState(collectionLoadState)) + + l0segments, exist := partSegInfoMap[-1] + if exist { + b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setL0Segments(l0segments)) + } partitionBackupInfos := b.meta.GetPartitions(collectionBackup.CollectionId) log.Info("finish build partition info", zap.String("collectionName", collectionBackup.GetCollectionName()), - zap.Int("partitionNum", len(partitionBackupInfos))) + zap.Int("partitionNum", len(partitionBackupInfos)), + zap.Int("l0SegmentsNum", len(l0segments))) var collectionBackupSize int64 = 0 for _, part := range partitionBackupInfos { collectionBackupSize += part.GetSize() } - b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setCollectionSize(collectionBackupSize)) + b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setCollectionLoadState(collectionLoadState), setCollectionSize(collectionBackupSize)) return nil } func (b *BackupContext) backupCollectionExecute(ctx context.Context, collectionBackup *backuppb.CollectionBackupInfo) error { log.Info("backupCollectionExecute", zap.Any("collectionMeta", collectionBackup.String())) - + backupInfo := b.meta.GetBackupByCollectionID(collectionBackup.GetCollectionId()) + backupBinlogPath := BackupBinlogDirPath(b.backupRootPath, backupInfo.GetName()) for _, partition := range b.meta.GetPartitions(collectionBackup.CollectionId) { - var segmentBackupInfos []*backuppb.SegmentBackupInfo + segmentBackupInfos := make([]*backuppb.SegmentBackupInfo, 0) var currentSize int64 = 0 var groupID int64 = 1 + // currently not group l0 segments + //var currentL0Size int64 = 0 + //var l0GroupID int64 = 1 segments := b.meta.GetSegments(partition.GetPartitionId()) for _, v := range segments { segment := v @@ -514,20 +530,28 @@ func (b *BackupContext) backupCollectionExecute(ctx context.Context, collectionB log.Error("Fail to fill segment backup info", zap.Error(err)) return err } - if currentSize > BackupSegmentGroupMaxSizeInMB*1024*1024 { // 256MB - groupID++ - currentSize = 0 + if !segment.IsL0 { + if currentSize > BackupSegmentGroupMaxSizeInMB*1024*1024 { // 256MB + groupID++ + currentSize = 0 + } + currentSize = currentSize + segment.GetSize() + b.meta.UpdateSegment(segment.GetPartitionId(), segment.GetSegmentId(), setGroupID(groupID)) + } else { + //if currentSize > BackupSegmentGroupMaxSizeInMB*1024*1024 { // 256MB + // l0GroupID++ + // currentL0Size = 0 + //} + //currentL0Size = currentL0Size + segment.GetSize() + b.meta.UpdateSegment(segment.GetPartitionId(), segment.GetSegmentId(), setGroupID(segment.GetSegmentId())) } - currentSize = currentSize + segment.GetSize() - b.meta.UpdateSegment(segment.GetPartitionId(), segment.GetSegmentId(), setGroupID(groupID)) segmentBackupInfos = append(segmentBackupInfos, segment) } log.Info("Begin copy data", zap.String("dbName", collectionBackup.GetDbName()), zap.String("collectionName", collectionBackup.GetCollectionName()), zap.Int64("collectionID", partition.GetCollectionId()), - zap.Int64("partitionID", partition.GetPartitionId()), - zap.Int("segmentNum", len(segmentBackupInfos))) + zap.Int64("partitionID", partition.GetPartitionId())) sort.SliceStable(segmentBackupInfos, func(i, j int) bool { return segmentBackupInfos[i].Size < segmentBackupInfos[j].Size @@ -536,23 +560,39 @@ func (b *BackupContext) backupCollectionExecute(ctx context.Context, collectionB segmentIDs := lo.Map(segmentBackupInfos, func(segment *backuppb.SegmentBackupInfo, _ int) int64 { return segment.GetSegmentId() }) - err := b.copySegments(ctx, segmentIDs) + err := b.copySegments(ctx, backupBinlogPath, segmentIDs) if err != nil { return err } } - //collectionBackup.EndTime = time.Now().Unix() - b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setCollectionEndTime(time.Now().Unix())) + l0Segments := collectionBackup.GetL0Segments() + segmentBackupInfos := make([]*backuppb.SegmentBackupInfo, 0) + segmentIDs := make([]int64, 0) + for _, v := range l0Segments { + segment := v + err := b.fillSegmentBackupInfo(ctx, segment) + if err != nil { + log.Error("Fail to fill segment backup info", zap.Error(err)) + return err + } + segmentIDs = append(segmentIDs, segment.GetSegmentId()) + segmentBackupInfos = append(segmentBackupInfos, b.meta.GetSegment(segment.GetSegmentId())) + } + err := b.copySegments(ctx, backupBinlogPath, segmentIDs) + if err != nil { + log.Error("Fail to fill segment backup info", zap.Error(err)) + return err + } + b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setL0Segments(segmentBackupInfos)) + b.meta.UpdateCollection(collectionBackup.Id, collectionBackup.CollectionId, setCollectionEndTime(time.Now().Unix())) log.Info("Finish copy data", zap.String("dbName", collectionBackup.GetDbName()), zap.String("collectionName", collectionBackup.GetCollectionName())) return nil } -const GC_Warn_Message = "This warn won't fail the backup process. Pause GC can protect data not to be GCed during backup, it is necessary to backup very large data(cost more than a hour)." - func (b *BackupContext) pauseMilvusGC(ctx context.Context, gcAddress string, pauseSeconds int) { pauseAPI := "/management/datacoord/garbage_collection/pause" params := url.Values{} @@ -613,12 +653,6 @@ func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backup defer b.resumeMilvusGC(ctx, gcAddress) } - // 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 { @@ -628,25 +662,16 @@ func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backup } collectionNames := make([]string, len(toBackupCollections)) for i, coll := range toBackupCollections { - collectionNames[i] = coll.collectionName + collectionNames[i] = coll.db + "." + coll.collectionName } log.Info("collections to backup", zap.Strings("collections", collectionNames)) jobIds := make([]int64, 0) - //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 { - err := b.backupCollectionPrepare(ctx, backupInfo, collectionClone, request.GetForce()) - if err != nil { - return err - } - //toBackupCollectionInfosMutex.Lock() - //defer toBackupCollectionInfosMutex.Unlock() - //toBackupCollectionInfos = append(toBackupCollectionInfos, coll) - return nil + return b.backupCollectionPrepare(ctx, backupInfo, collectionClone, request.GetForce()) }, retry.Sleep(120*time.Second), retry.Attempts(128)) return err } @@ -655,8 +680,6 @@ 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() b.meta.UpdateBackup(backupInfo.Id, setStateCode(backuppb.BackupTaskStateCode_BACKUP_FAIL), setErrorMessage(err.Error())) return err } @@ -679,12 +702,6 @@ func (b *BackupContext) executeCreateBackup(ctx context.Context, request *backup b.meta.UpdateBackup(backupInfo.Id, setStateCode(backuppb.BackupTaskStateCode_BACKUP_FAIL), setErrorMessage(err.Error())) return err } - - //_, 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") } @@ -711,7 +728,7 @@ func (b *BackupContext) writeBackupInfoMeta(ctx context.Context, id string) erro log.Info("Final backupInfo", zap.String("backupInfo", backupInfo.String())) output, _ := serialize(backupInfo) log.Debug("backup meta", zap.String("value", string(output.BackupMetaBytes))) - log.Debug("collection meta", zap.String("value", string(output.CollectionMetaBytes))) + log.Info("collection meta", zap.String("value", string(output.CollectionMetaBytes))) log.Debug("partition meta", zap.String("value", string(output.PartitionMetaBytes))) log.Debug("segment meta", zap.String("value", string(output.SegmentMetaBytes))) @@ -748,12 +765,13 @@ func (b *BackupContext) writeBackupInfoMeta(ctx context.Context, id string) erro return nil } -func (b *BackupContext) copySegments(ctx context.Context, segmentIDs []int64) error { +func (b *BackupContext) copySegments(ctx context.Context, backupBinlogPath string, segmentIDs []int64) error { jobIds := make([]int64, 0) - for _, segmentID := range segmentIDs { - segment := segmentID + for _, v := range segmentIDs { + segmentID := v + segment := b.meta.GetSegment(segmentID) job := func(ctx context.Context) error { - return b.copySegment(ctx, segment) + return b.copySegment(ctx, backupBinlogPath, segment) } jobId := b.getCopyDataWorkerPool().SubmitWithId(job) jobIds = append(jobIds, jobId) @@ -763,29 +781,26 @@ func (b *BackupContext) copySegments(ctx context.Context, segmentIDs []int64) er return err } -func (b *BackupContext) copySegment(ctx context.Context, segmentID int64) error { - segment := b.meta.GetSegment(segmentID) +func (b *BackupContext) copySegment(ctx context.Context, backupBinlogPath string, segment *backuppb.SegmentBackupInfo) error { 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.Info("copy segment") - backupInfo := b.meta.GetBackupBySegmentID(segmentID) - dstPath := BackupBinlogDirPath(b.backupRootPath, backupInfo.GetName()) + log.Info("copy segment", zap.String("backupBinlogPath", backupBinlogPath)) // 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 backupPathFunc := func(binlogPath, rootPath, backupBinlogPath string) string { if rootPath == "" { - return dstPath + SEPERATOR + binlogPath + return backupBinlogPath + SEPERATOR + binlogPath } else { - return strings.Replace(binlogPath, rootPath, dstPath, 1) + return strings.Replace(binlogPath, rootPath, backupBinlogPath, 1) } } // insert log for _, binlogs := range segment.GetBinlogs() { for _, binlog := range binlogs.GetBinlogs() { - targetPath := backupPathFunc(binlog.GetLogPath(), b.milvusRootPath, dstPath) + targetPath := backupPathFunc(binlog.GetLogPath(), b.milvusRootPath, backupBinlogPath) // use segmentID as group id segment.GroupId = segment.SegmentId if segment.GetGroupId() != 0 { @@ -832,7 +847,7 @@ func (b *BackupContext) copySegment(ctx context.Context, segmentID int64) error // delta log for _, binlogs := range segment.GetDeltalogs() { for _, binlog := range binlogs.GetBinlogs() { - targetPath := backupPathFunc(binlog.GetLogPath(), b.milvusRootPath, dstPath) + targetPath := backupPathFunc(binlog.GetLogPath(), b.milvusRootPath, backupBinlogPath) if segment.GetGroupId() != 0 { targetPath = strings.Replace(targetPath, strconv.FormatInt(segment.GetPartitionId(), 10), @@ -890,9 +905,10 @@ func (b *BackupContext) fillSegmentBackupInfo(ctx context.Context, segmentBackup insertPath := fmt.Sprintf("%s%s/%v/%v/%v/", rootPath, "insert_log", segmentBackupInfo.GetCollectionId(), segmentBackupInfo.GetPartitionId(), segmentBackupInfo.GetSegmentId()) log.Debug("insertPath", zap.String("bucket", b.milvusBucketName), zap.String("insertPath", insertPath)) fieldsLogDir, _, err := b.getStorageClient().ListWithPrefix(ctx, b.milvusBucketName, insertPath, false) + // handle segment level + isL0 := false if len(fieldsLogDir) == 0 { - msg := fmt.Sprintf("Get empty input path, but segment should not be empty, %s", insertPath) - return errors.New(msg) + isL0 = true } if err != nil { log.Error("Fail to list segment path", zap.String("insertPath", insertPath), zap.Error(err)) @@ -967,7 +983,8 @@ func (b *BackupContext) fillSegmentBackupInfo(ctx context.Context, segmentBackup //segmentBackupInfo.Binlogs = insertLogs //segmentBackupInfo.Deltalogs = deltaLogs segmentBackupInfo.Size = size - b.meta.UpdateSegment(segmentBackupInfo.GetPartitionId(), segmentBackupInfo.GetSegmentId(), setSegmentBinlogs(insertLogs), setSegmentDeltaBinlogs(deltaLogs), setSegmentSize(size)) + segmentBackupInfo.IsL0 = isL0 + b.meta.UpdateSegment(segmentBackupInfo.GetPartitionId(), segmentBackupInfo.GetSegmentId(), setSegmentBinlogs(insertLogs), setSegmentDeltaBinlogs(deltaLogs), setSegmentSize(size), setSegmentL0(isL0)) 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 1ef83dd..b41e78b 100644 --- a/core/backup_impl_restore_backup.go +++ b/core/backup_impl_restore_backup.go @@ -10,6 +10,7 @@ import ( jsoniter "github.com/json-iterator/go" gomilvus "github.com/milvus-io/milvus-sdk-go/v2/client" "github.com/milvus-io/milvus-sdk-go/v2/entity" + "github.com/samber/lo" "go.uber.org/atomic" "go.uber.org/zap" @@ -381,9 +382,12 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup targetDBName := task.GetTargetDbName() targetCollectionName := task.GetTargetCollectionName() task.StateCode = backuppb.RestoreTaskStateCode_EXECUTING + log := log.With( + zap.String("backup_db_name", task.GetCollBackup().DbName), + zap.String("backup_collection_name", task.GetCollBackup().DbName), + zap.String("target_db_name", targetDBName), + zap.String("target_collection_name", targetCollectionName)) log.Info("start restore", - zap.String("db_name", targetDBName), - zap.String("collection_name", targetCollectionName), zap.String("backupBucketName", backupBucketName), zap.String("backupPath", backupPath)) // create collection @@ -466,13 +470,9 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup return task, err } log.Info("create collection", - zap.String("database", targetDBName), - zap.String("collectionName", targetCollectionName), zap.Bool("hasPartitionKey", hasPartitionKey)) } else { log.Info("skip create collection", - zap.String("database", targetDBName), - zap.String("collectionName", targetCollectionName), zap.Bool("hasPartitionKey", hasPartitionKey)) } @@ -484,7 +484,6 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup strings.HasPrefix(err.Error(), "index doesn't exist") { // todo log.Info("field has no index", - zap.String("collection_name", targetCollectionName), zap.String("field_name", field.Name)) continue } else { @@ -496,13 +495,10 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup err = b.milvusClient.DropIndex(ctx, targetDBName, targetCollectionName, fieldIndex.Name()) if err != nil { log.Warn("Fail to drop index", - zap.String("db", targetDBName), - zap.String("collection", targetCollectionName), zap.Error(err)) return task, err } log.Info("drop index", - zap.String("collection_name", targetCollectionName), zap.String("field_name", field.Name), zap.String("index_name", fieldIndex.Name())) } @@ -563,15 +559,55 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup } }() + // bulk insert + copyAndBulkInsert := func(dbName, collectionName, partitionName string, files []string, isL0 bool) error { + realFiles := make([]string, len(files)) + // if milvus bucket and backup bucket are not the same, should copy the data first + if !isSameBucket { + log.Info("milvus bucket and backup bucket are not the same, copy the data first", zap.Strings("files", files)) + for i, file := range files { + // empty delta file, no need to copy + if file == "" { + realFiles[i] = file + } else { + log.Debug("Copy temporary restore file", zap.String("from", file), zap.String("to", tempDir+file)) + err := retry.Do(ctx, func() error { + return b.getStorageClient().Copy(ctx, backupBucketName, b.milvusBucketName, file, tempDir+file) + }, retry.Sleep(2*time.Second), retry.Attempts(5)) + if err != nil { + log.Error("fail to copy backup date from backup bucket to restore target milvus bucket after retry", zap.Error(err)) + return err + } + realFiles[i] = tempDir + file + } + } + } else { + realFiles = files + } + + err := b.executeBulkInsert(ctx, dbName, collectionName, partitionName, realFiles, int64(task.GetCollBackup().BackupTimestamp), isL0) + if err != nil { + log.Error("fail to bulk insert to partition", + zap.String("partition", partitionName), + zap.Error(err)) + return err + } + return nil + } + jobIds := make([]int64, 0) restoredSize := atomic.Int64{} + + type partitionL0Segment struct { + collectionID int64 + partitionName string + partitionID int64 + segmentID int64 + } + partitionL0Segments := make([]partitionL0Segment, 0) for _, v := range task.GetCollBackup().GetPartitionBackups() { partitionBackup := v - log.Info("start restore partition", - zap.String("backupCollectionName", task.GetCollBackup().GetCollectionName()), - zap.String("targetDBName", targetDBName), - zap.String("targetCollectionName", targetCollectionName), - zap.String("partition", partitionBackup.GetPartitionName())) + log.Info("start restore partition", zap.String("partition", partitionBackup.GetPartitionName())) // pre-check whether partition exist, if not create it exist, err := b.getMilvusClient().HasPartition(ctx, targetDBName, targetCollectionName, partitionBackup.GetPartitionName()) if err != nil { @@ -586,9 +622,7 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup log.Error("fail to create partition", zap.Error(err)) return task, err } - log.Info("create partition", - zap.String("collectionName", targetCollectionName), - zap.String("partitionName", partitionBackup.GetPartitionName())) + log.Info("create partition", zap.String("partitionName", partitionBackup.GetPartitionName())) } type restoreGroup struct { @@ -596,15 +630,20 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup size int64 } restoreFileGroups := make([]restoreGroup, 0) - groupIds := collectGroupIdsFromSegments(partitionBackup.GetSegmentBackups()) + + var l0Segments []*backuppb.SegmentBackupInfo = lo.Filter(partitionBackup.GetSegmentBackups(), func(segment *backuppb.SegmentBackupInfo, _ int) bool { + return segment.IsL0 + }) + notl0Segments := lo.Filter(partitionBackup.GetSegmentBackups(), func(segment *backuppb.SegmentBackupInfo, _ int) bool { + return !segment.IsL0 + }) + groupIds := collectGroupIdsFromSegments(notl0Segments) if len(groupIds) == 1 && groupIds[0] == 0 { // backward compatible old backup without group id files, size, err := b.getBackupPartitionPaths(ctx, backupBucketName, backupPath, partitionBackup) if err != nil { log.Error("fail to get partition backup binlog files", zap.Error(err), - zap.String("backupCollectionName", task.GetCollBackup().GetCollectionName()), - zap.String("targetCollectionName", targetCollectionName), zap.String("partition", partitionBackup.GetPartitionName())) return task, err } @@ -616,8 +655,6 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup if err != nil { log.Error("fail to get partition backup binlog files", zap.Error(err), - zap.String("backupCollectionName", task.GetCollBackup().GetCollectionName()), - zap.String("targetCollectionName", targetCollectionName), zap.String("partition", partitionBackup.GetPartitionName())) return task, err } @@ -625,49 +662,10 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup } } - // bulk insert - copyAndBulkInsert := func(files []string) error { - realFiles := make([]string, len(files)) - // if milvus bucket and backup bucket are not the same, should copy the data first - if !isSameBucket { - log.Info("milvus bucket and backup bucket are not the same, copy the data first", zap.Strings("files", files)) - for i, file := range files { - // empty delta file, no need to copy - if file == "" { - realFiles[i] = file - } else { - log.Debug("Copy temporary restore file", zap.String("from", file), zap.String("to", tempDir+file)) - err := retry.Do(ctx, func() error { - return b.getStorageClient().Copy(ctx, backupBucketName, b.milvusBucketName, file, tempDir+file) - }, retry.Sleep(2*time.Second), retry.Attempts(5)) - if err != nil { - log.Error("fail to copy backup date from backup bucket to restore target milvus bucket after retry", zap.Error(err)) - return err - } - realFiles[i] = tempDir + file - } - } - } else { - realFiles = files - } - - err := b.executeBulkInsert(ctx, targetDBName, targetCollectionName, partitionBackup.GetPartitionName(), realFiles, int64(task.GetCollBackup().BackupTimestamp)) - if err != nil { - log.Error("fail to bulk insert to partition", - zap.String("backupCollectionName", task.GetCollBackup().GetCollectionName()), - zap.String("targetDBName", targetDBName), - zap.String("targetCollectionName", targetCollectionName), - zap.String("partition", partitionBackup.GetPartitionName()), - zap.Error(err)) - return err - } - return nil - } - for _, value := range restoreFileGroups { group := value job := func(ctx context.Context) error { - err := copyAndBulkInsert(group.files) + err := copyAndBulkInsert(targetDBName, targetCollectionName, partitionBackup.GetPartitionName(), group.files, false) if err != nil { return err } else { @@ -680,9 +678,52 @@ func (b *BackupContext) executeRestoreCollectionTask(ctx context.Context, backup jobId := b.getRestoreWorkerPool(parentTaskID).SubmitWithId(job) jobIds = append(jobIds, jobId) } + + if len(l0Segments) > 0 { + for _, segment := range l0Segments { + partitionL0Segments = append(partitionL0Segments, partitionL0Segment{ + collectionID: segment.CollectionId, + partitionName: partitionBackup.GetPartitionName(), + partitionID: segment.GetPartitionId(), + segmentID: segment.GetSegmentId(), + }) + } + } } err := b.getRestoreWorkerPool(parentTaskID).WaitJobs(jobIds) + if err != nil { + return task, err + } + + // restore l0 segments + l0JobIds := make([]int64, 0) + log.Info("start restore l0 segments", zap.Int("global_l0_segment_num", len(task.GetCollBackup().GetL0Segments())), zap.Int("partition_l0_segment_num", len(partitionL0Segments))) + for _, v := range partitionL0Segments { + segmentBackup := v + job := func(ctx context.Context) error { + l0Files := fmt.Sprintf("%s/%s/%s/%d/%d/%d", backupPath, BINGLOG_DIR, DELTA_LOG_DIR, segmentBackup.collectionID, segmentBackup.partitionID, segmentBackup.segmentID) + log.Info("restore l0 segment ", zap.String("files", l0Files)) + return copyAndBulkInsert(targetDBName, targetCollectionName, segmentBackup.partitionName, []string{l0Files}, true) + } + jobId := b.getRestoreWorkerPool(parentTaskID).SubmitWithId(job) + l0JobIds = append(l0JobIds, jobId) + } + + if len(task.GetCollBackup().GetL0Segments()) > 0 { + for _, v := range task.GetCollBackup().GetL0Segments() { + segment := v + job := func(ctx context.Context) error { + l0Files := fmt.Sprintf("%s/%s/%s/%d/%d/%d", backupPath, BINGLOG_DIR, DELTA_LOG_DIR, task.CollBackup.CollectionId, -1, segment.GetSegmentId()) + log.Info("restore l0 segment ", zap.String("files", l0Files)) + return copyAndBulkInsert(targetDBName, targetCollectionName, "", []string{l0Files}, true) + } + jobId := b.getRestoreWorkerPool(parentTaskID).SubmitWithId(job) + l0JobIds = append(l0JobIds, jobId) + } + } + err = b.getRestoreWorkerPool(parentTaskID).WaitJobs(l0JobIds) + return task, err } @@ -698,7 +739,7 @@ func collectGroupIdsFromSegments(segments []*backuppb.SegmentBackupInfo) []int64 return res } -func (b *BackupContext) executeBulkInsert(ctx context.Context, db, coll string, partition string, files []string, endTime int64) error { +func (b *BackupContext) executeBulkInsert(ctx context.Context, db, coll string, partition string, files []string, endTime int64, isL0 bool) error { log.Info("execute bulk insert", zap.String("db", db), zap.String("collection", coll), @@ -707,10 +748,18 @@ func (b *BackupContext) executeBulkInsert(ctx context.Context, db, coll string, zap.Int64("endTime", endTime)) var taskId int64 var err error - if endTime == 0 { - taskId, err = b.getMilvusClient().BulkInsert(ctx, db, coll, partition, files, gomilvus.IsBackup()) + if isL0 { + if endTime == 0 { + taskId, err = b.getMilvusClient().BulkInsert(ctx, db, coll, partition, files, gomilvus.IsL0()) + } else { + taskId, err = b.getMilvusClient().BulkInsert(ctx, db, coll, partition, files, gomilvus.IsL0(), gomilvus.WithEndTs(endTime)) + } } else { - taskId, err = b.getMilvusClient().BulkInsert(ctx, db, coll, partition, files, gomilvus.IsBackup(), gomilvus.WithEndTs(endTime)) + if endTime == 0 { + taskId, err = b.getMilvusClient().BulkInsert(ctx, db, coll, partition, files, gomilvus.IsBackup()) + } else { + taskId, err = b.getMilvusClient().BulkInsert(ctx, db, coll, partition, files, gomilvus.IsBackup(), gomilvus.WithEndTs(endTime)) + } } if err != nil { log.Error("fail to bulk insert", diff --git a/core/backup_meta.go b/core/backup_meta.go index 55e8d00..f501e3d 100644 --- a/core/backup_meta.go +++ b/core/backup_meta.go @@ -76,21 +76,7 @@ func treeToLevel(backup *backuppb.BackupInfo) (LeveledBackupInfo, error) { } collectionBack.Size = collectionSize - cloneCollectionBackup := &backuppb.CollectionBackupInfo{ - CollectionId: collectionBack.GetCollectionId(), - DbName: collectionBack.GetDbName(), - CollectionName: collectionBack.GetCollectionName(), - Schema: collectionBack.GetSchema(), - ShardsNum: collectionBack.GetShardsNum(), - ConsistencyLevel: collectionBack.GetConsistencyLevel(), - BackupTimestamp: collectionBack.GetBackupTimestamp(), - Size: collectionBack.GetSize(), - HasIndex: collectionBack.GetHasIndex(), - IndexInfos: collectionBack.GetIndexInfos(), - LoadState: collectionBack.GetLoadState(), - BackupPhysicalTimestamp: collectionBack.GetBackupPhysicalTimestamp(), - ChannelCheckpoints: collectionBack.GetChannelCheckpoints(), - } + cloneCollectionBackup := proto.Clone(collectionBack).(*backuppb.CollectionBackupInfo) collections = append(collections, cloneCollectionBackup) backupSize = backupSize + collectionSize } diff --git a/core/backup_meta_manager.go b/core/backup_meta_manager.go index b28a7fb..176d9df 100644 --- a/core/backup_meta_manager.go +++ b/core/backup_meta_manager.go @@ -217,6 +217,12 @@ func setCollectionSize(size int64) CollectionOpt { } } +func setL0Segments(segments []*backuppb.SegmentBackupInfo) CollectionOpt { + return func(collection *backuppb.CollectionBackupInfo) { + collection.L0Segments = segments + } +} + func incCollectionSize(size int64) CollectionOpt { return func(collection *backuppb.CollectionBackupInfo) { collection.Size = collection.Size + size @@ -361,6 +367,12 @@ func setSegmentSize(size int64) SegmentOpt { } } +func setSegmentL0(isL0 bool) SegmentOpt { + return func(segment *backuppb.SegmentBackupInfo) { + segment.IsL0 = isL0 + } +} + func setGroupID(groupID int64) SegmentOpt { return func(segment *backuppb.SegmentBackupInfo) { segment.GroupId = groupID @@ -426,6 +438,16 @@ func (meta *MetaManager) GetBackupBySegmentID(segmentID int64) *backuppb.BackupI return meta.backups[backupID] } +func (meta *MetaManager) GetBackupByCollectionID(collectionID int64) *backuppb.BackupInfo { + meta.mu.Lock() + defer meta.mu.Unlock() + 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() diff --git a/core/milvus_sdk_wrapper.go b/core/milvus_sdk_wrapper.go index 3c9ca65..b7df559 100644 --- a/core/milvus_sdk_wrapper.go +++ b/core/milvus_sdk_wrapper.go @@ -2,12 +2,14 @@ package core import ( "context" + "sync" + "time" + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" gomilvus "github.com/milvus-io/milvus-sdk-go/v2/client" "github.com/milvus-io/milvus-sdk-go/v2/entity" + "github.com/zilliztech/milvus-backup/internal/util/retry" - "sync" - "time" ) // MilvusClient wrap db into milvus API to make it thread safe diff --git a/core/paramtable/base_table.go b/core/paramtable/base_table.go index b73e298..0720b73 100644 --- a/core/paramtable/base_table.go +++ b/core/paramtable/base_table.go @@ -28,12 +28,8 @@ import ( memkv "github.com/zilliztech/milvus-backup/internal/kv/mem" "github.com/zilliztech/milvus-backup/internal/log" "github.com/zilliztech/milvus-backup/internal/util/logutil" - "github.com/zilliztech/milvus-backup/internal/util/typeutil" ) -// UniqueID is type alias of typeutil.UniqueID -type UniqueID = typeutil.UniqueID - const ( DefaultBackupYaml = "backup.yaml" diff --git a/core/proto/backup.proto b/core/proto/backup.proto index 8c1b5e3..d4685fa 100644 --- a/core/proto/backup.proto +++ b/core/proto/backup.proto @@ -64,6 +64,7 @@ message CollectionBackupInfo { // physical unix time of backup uint64 backup_physical_timestamp = 19; map channel_checkpoints = 20; + repeated SegmentBackupInfo l0_segments = 21; } message PartitionBackupInfo { @@ -93,6 +94,7 @@ message SegmentBackupInfo { // and will bulkinsert in one call during restore int64 group_id = 9; bool backuped = 10; + bool is_l0 = 11; } /** diff --git a/core/proto/backuppb/backup.pb.go b/core/proto/backuppb/backup.pb.go index ba0923d..ea4af99 100644 --- a/core/proto/backuppb/backup.pb.go +++ b/core/proto/backuppb/backup.pb.go @@ -353,11 +353,12 @@ type CollectionBackupInfo struct { IndexInfos []*IndexInfo `protobuf:"bytes,17,rep,name=index_infos,json=indexInfos,proto3" json:"index_infos,omitempty"` LoadState string `protobuf:"bytes,18,opt,name=load_state,json=loadState,proto3" json:"load_state,omitempty"` // physical unix time of backup - BackupPhysicalTimestamp uint64 `protobuf:"varint,19,opt,name=backup_physical_timestamp,json=backupPhysicalTimestamp,proto3" json:"backup_physical_timestamp,omitempty"` - ChannelCheckpoints map[string]string `protobuf:"bytes,20,rep,name=channel_checkpoints,json=channelCheckpoints,proto3" json:"channel_checkpoints,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + BackupPhysicalTimestamp uint64 `protobuf:"varint,19,opt,name=backup_physical_timestamp,json=backupPhysicalTimestamp,proto3" json:"backup_physical_timestamp,omitempty"` + ChannelCheckpoints map[string]string `protobuf:"bytes,20,rep,name=channel_checkpoints,json=channelCheckpoints,proto3" json:"channel_checkpoints,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + L0Segments []*SegmentBackupInfo `protobuf:"bytes,21,rep,name=l0_segments,json=l0Segments,proto3" json:"l0_segments,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *CollectionBackupInfo) Reset() { *m = CollectionBackupInfo{} } @@ -525,6 +526,13 @@ func (m *CollectionBackupInfo) GetChannelCheckpoints() map[string]string { return nil } +func (m *CollectionBackupInfo) GetL0Segments() []*SegmentBackupInfo { + if m != nil { + return m.L0Segments + } + return nil +} + type PartitionBackupInfo struct { PartitionId int64 `protobuf:"varint,1,opt,name=partition_id,json=partitionId,proto3" json:"partition_id,omitempty"` PartitionName string `protobuf:"bytes,2,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` @@ -621,6 +629,7 @@ type SegmentBackupInfo struct { // 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"` + IsL0 bool `protobuf:"varint,11,opt,name=is_l0,json=isL0,proto3" json:"is_l0,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -721,6 +730,13 @@ func (m *SegmentBackupInfo) GetBackuped() bool { return false } +func (m *SegmentBackupInfo) GetIsL0() bool { + if m != nil { + return m.IsL0 + } + return false +} + // * // root of backup type BackupInfo struct { @@ -2899,195 +2915,197 @@ func init() { func init() { proto.RegisterFile("backup.proto", fileDescriptor_65240d19de191688) } var fileDescriptor_65240d19de191688 = []byte{ - // 3008 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, 0xe4, 0x92, 0x43, 0x0e, 0x39, 0xe4, 0x94, 0x5b, 0x8e, 0xb9, 0x25, - 0x08, 0x7c, 0xc9, 0x7f, 0x90, 0x20, 0xff, 0x47, 0x90, 0x53, 0x72, 0xcb, 0x35, 0xe8, 0xea, 0x9e, - 0xc7, 0x2e, 0x87, 0xd4, 0x32, 0x30, 0xe2, 0x38, 0xb7, 0xe9, 0x5f, 0x57, 0x55, 0x77, 0x57, 0x57, - 0x57, 0x55, 0x57, 0x0f, 0x34, 0x7a, 0xb6, 0xf3, 0x6c, 0x34, 0x5c, 0x1f, 0x86, 0x4c, 0x30, 0xb2, - 0x38, 0xf0, 0xfc, 0xe7, 0x23, 0xae, 0x5a, 0xeb, 0xaa, 0x6b, 0xe5, 0xb5, 0x3e, 0x63, 0x7d, 0x9f, - 0xde, 0x40, 0xb0, 0x37, 0x3a, 0xbc, 0xc1, 0x45, 0x38, 0x72, 0x84, 0x22, 0xea, 0xfc, 0x25, 0x07, - 0xb5, 0x6e, 0xe0, 0xd2, 0x17, 0xdd, 0xe0, 0x90, 0x91, 0x4b, 0x00, 0x87, 0x1e, 0xf5, 0x5d, 0x2b, - 0xb0, 0x07, 0xb4, 0x9d, 0x5b, 0xcd, 0xad, 0xd5, 0xcc, 0x1a, 0x22, 0xbb, 0xf6, 0x80, 0xca, 0x6e, - 0x4f, 0xd2, 0xaa, 0xee, 0xbc, 0xea, 0x46, 0x64, 0xb2, 0x5b, 0x8c, 0x87, 0xb4, 0x5d, 0x48, 0x75, - 0x1f, 0x8c, 0x87, 0x94, 0x6c, 0x42, 0x79, 0x68, 0x87, 0xf6, 0x80, 0xb7, 0x8b, 0xab, 0x85, 0xb5, - 0xfa, 0xc6, 0xf5, 0xf5, 0x8c, 0xe9, 0xae, 0xc7, 0x93, 0x59, 0xdf, 0x43, 0xe2, 0x9d, 0x40, 0x84, - 0x63, 0x53, 0x73, 0xae, 0x7c, 0x04, 0xf5, 0x14, 0x4c, 0x0c, 0x28, 0x3c, 0xa3, 0x63, 0x3d, 0x51, - 0xf9, 0x49, 0x96, 0xa0, 0xf4, 0xdc, 0xf6, 0x47, 0xd1, 0xec, 0x54, 0xe3, 0x56, 0xfe, 0xc3, 0x5c, - 0xe7, 0xe7, 0x55, 0x58, 0xda, 0x62, 0xbe, 0x4f, 0x1d, 0xe1, 0xb1, 0x60, 0x13, 0x47, 0xc3, 0x45, - 0xb7, 0x20, 0xef, 0xb9, 0x5a, 0x46, 0xde, 0x73, 0xc9, 0x5d, 0x00, 0x2e, 0x6c, 0x41, 0x2d, 0x87, - 0xb9, 0x4a, 0x4e, 0x6b, 0x63, 0x2d, 0x73, 0xae, 0x4a, 0xc8, 0x81, 0xcd, 0x9f, 0xed, 0x4b, 0x86, - 0x2d, 0xe6, 0x52, 0xb3, 0xc6, 0xa3, 0x4f, 0xd2, 0x81, 0x06, 0x0d, 0x43, 0x16, 0x3e, 0xa4, 0x9c, - 0xdb, 0xfd, 0x48, 0x23, 0x13, 0x98, 0xd4, 0x19, 0x17, 0x76, 0x28, 0x2c, 0xe1, 0x0d, 0x68, 0xbb, - 0xb8, 0x9a, 0x5b, 0x2b, 0xa0, 0x88, 0x50, 0x1c, 0x78, 0x03, 0x4a, 0x2e, 0x40, 0x95, 0x06, 0xae, - 0xea, 0x2c, 0x61, 0x67, 0x85, 0x06, 0x2e, 0x76, 0xad, 0x40, 0x75, 0x18, 0xb2, 0x7e, 0x48, 0x39, - 0x6f, 0x97, 0x57, 0x73, 0x6b, 0x25, 0x33, 0x6e, 0x93, 0xff, 0x81, 0xa6, 0x13, 0x2f, 0xd5, 0xf2, - 0xdc, 0x76, 0x05, 0x79, 0x1b, 0x09, 0xd8, 0x75, 0xc9, 0x79, 0xa8, 0xb8, 0x3d, 0xb5, 0x95, 0x55, - 0x9c, 0x59, 0xd9, 0xed, 0xe1, 0x3e, 0xbe, 0x05, 0xf3, 0x29, 0x6e, 0x24, 0xa8, 0x21, 0x41, 0x2b, - 0x81, 0x91, 0xf0, 0x63, 0x28, 0x73, 0xe7, 0x88, 0x0e, 0xec, 0x36, 0xac, 0xe6, 0xd6, 0xea, 0x1b, - 0x6f, 0x66, 0x6a, 0x29, 0x51, 0xfa, 0x3e, 0x12, 0x9b, 0x9a, 0x09, 0xd7, 0x7e, 0x64, 0x87, 0x2e, - 0xb7, 0x82, 0xd1, 0xa0, 0x5d, 0xc7, 0x35, 0xd4, 0x14, 0xb2, 0x3b, 0x1a, 0x10, 0x13, 0x16, 0x1c, - 0x16, 0x70, 0x8f, 0x0b, 0x1a, 0x38, 0x63, 0xcb, 0xa7, 0xcf, 0xa9, 0xdf, 0x6e, 0xe0, 0x76, 0x9c, - 0x34, 0x50, 0x4c, 0xfd, 0x40, 0x12, 0x9b, 0x86, 0x33, 0x85, 0x90, 0xc7, 0xb0, 0x30, 0xb4, 0x43, - 0xe1, 0xe1, 0xca, 0x14, 0x1b, 0x6f, 0x37, 0xd1, 0x1c, 0xb3, 0xb7, 0x78, 0x2f, 0xa2, 0x4e, 0x0c, - 0xc6, 0x34, 0x86, 0x93, 0x20, 0x27, 0xd7, 0xc0, 0x50, 0xf4, 0xb8, 0x53, 0x5c, 0xd8, 0x83, 0x61, - 0xbb, 0xb5, 0x9a, 0x5b, 0x2b, 0x9a, 0xf3, 0x0a, 0x3f, 0x88, 0x60, 0x42, 0xa0, 0xc8, 0xbd, 0x97, - 0xb4, 0x3d, 0x8f, 0x3b, 0x82, 0xdf, 0xe4, 0x22, 0xd4, 0x8e, 0x6c, 0x6e, 0xe1, 0x51, 0x69, 0x1b, - 0xab, 0xb9, 0xb5, 0xaa, 0x59, 0x3d, 0xb2, 0x39, 0x1e, 0x05, 0xf2, 0x29, 0xd4, 0xd5, 0xa9, 0xf2, - 0x82, 0x43, 0xc6, 0xdb, 0x0b, 0x38, 0xd9, 0xd7, 0x4f, 0x3f, 0x3b, 0xa6, 0x3a, 0x88, 0xf2, 0x93, - 0x4b, 0x35, 0xfb, 0xcc, 0x76, 0x2d, 0x34, 0xcc, 0x36, 0x51, 0xc7, 0x52, 0x22, 0x68, 0xb4, 0xe4, - 0x16, 0x5c, 0xd0, 0x73, 0x1f, 0x1e, 0x8d, 0xb9, 0xe7, 0xd8, 0x7e, 0x6a, 0x11, 0x8b, 0xb8, 0x88, - 0xf3, 0x8a, 0x60, 0x4f, 0xf7, 0x27, 0x8b, 0x09, 0x61, 0xd1, 0x39, 0xb2, 0x83, 0x80, 0xfa, 0x96, - 0x73, 0x44, 0x9d, 0x67, 0x43, 0xe6, 0x05, 0x82, 0xb7, 0x97, 0x70, 0x8e, 0xb7, 0x5f, 0x61, 0x0d, - 0x89, 0x46, 0xd7, 0xb7, 0x94, 0x90, 0xad, 0x44, 0x86, 0x3a, 0xf6, 0xc4, 0x39, 0xd6, 0xb1, 0xb2, - 0x03, 0xe7, 0x4f, 0x20, 0x3f, 0x93, 0x3b, 0xf8, 0x59, 0x1e, 0x16, 0x33, 0x36, 0x97, 0x5c, 0x81, - 0x46, 0x62, 0x21, 0xda, 0x2f, 0x14, 0xcc, 0x7a, 0x8c, 0x75, 0x5d, 0xf2, 0x26, 0xb4, 0x12, 0x92, - 0x94, 0x2b, 0x6c, 0xc6, 0x28, 0x9e, 0x8e, 0x63, 0x87, 0xb0, 0x90, 0x71, 0x08, 0x1f, 0xc1, 0x3c, - 0xa7, 0xfd, 0x01, 0x0d, 0x44, 0x6c, 0x8e, 0xca, 0x3b, 0x5e, 0xcd, 0xd4, 0xde, 0xbe, 0xa2, 0x4d, - 0x19, 0x63, 0x8b, 0xa7, 0x21, 0x1e, 0xdb, 0x57, 0x29, 0x65, 0x5f, 0x93, 0x16, 0x50, 0x9e, 0xb2, - 0x80, 0xce, 0xaf, 0x0b, 0xb0, 0x70, 0x4c, 0x30, 0x9e, 0x4e, 0x3d, 0xb3, 0x58, 0x0d, 0x35, 0x8d, - 0x74, 0xdd, 0xe3, 0xab, 0xcb, 0x67, 0xac, 0x6e, 0x5a, 0x99, 0x85, 0xe3, 0xca, 0x7c, 0x1d, 0xea, - 0xc1, 0x68, 0x60, 0xb1, 0x43, 0x2b, 0x64, 0x5f, 0xf2, 0xc8, 0x03, 0x06, 0xa3, 0xc1, 0xa3, 0x43, - 0x93, 0x7d, 0xc9, 0xc9, 0x2d, 0xa8, 0xf4, 0xbc, 0xc0, 0x67, 0x7d, 0xde, 0x2e, 0xa1, 0x62, 0x56, - 0x33, 0x15, 0x73, 0x47, 0x06, 0xa9, 0x4d, 0x24, 0x34, 0x23, 0x06, 0xf2, 0x09, 0xa0, 0x37, 0xe6, - 0xc8, 0x5d, 0x9e, 0x91, 0x3b, 0x61, 0x91, 0xfc, 0x2e, 0xf5, 0x85, 0x8d, 0xfc, 0x95, 0x59, 0xf9, - 0x63, 0x96, 0x78, 0x2f, 0xaa, 0xa9, 0xbd, 0xb8, 0x00, 0xd5, 0x7e, 0xc8, 0x46, 0x43, 0xa9, 0x8e, - 0x9a, 0xf2, 0xe8, 0xd8, 0xee, 0xba, 0xd2, 0xa3, 0x2b, 0x79, 0xd4, 0x45, 0x87, 0x5a, 0x35, 0xe3, - 0x76, 0xe7, 0xf7, 0x05, 0x80, 0xff, 0xee, 0x98, 0x45, 0xa0, 0x88, 0x67, 0xa9, 0x82, 0x23, 0xe2, - 0x77, 0xa6, 0x5f, 0xad, 0x66, 0xfb, 0xd5, 0xa7, 0x40, 0x52, 0xf6, 0x18, 0x9d, 0xa5, 0x1a, 0x6e, - 0xda, 0xb5, 0x99, 0x3d, 0x91, 0xb9, 0xe0, 0x4c, 0xa1, 0xc9, 0x2e, 0x42, 0x6a, 0x17, 0xdf, 0x84, - 0x96, 0x12, 0x69, 0x3d, 0xa7, 0x21, 0xf7, 0x58, 0x80, 0xe1, 0xab, 0x66, 0x36, 0x15, 0xfa, 0x44, - 0x81, 0x9d, 0xef, 0xc2, 0x85, 0x64, 0x14, 0x8c, 0x40, 0xa9, 0x3d, 0xfc, 0x14, 0x4a, 0xca, 0xa5, - 0xe7, 0xce, 0x3a, 0x49, 0xc5, 0xd7, 0xf9, 0x02, 0xda, 0xb1, 0x07, 0x9b, 0x16, 0xfe, 0xc9, 0xa4, - 0xf0, 0xd9, 0x83, 0x9b, 0x96, 0xfd, 0x04, 0x96, 0xb5, 0x4b, 0x98, 0x96, 0xfc, 0xff, 0x93, 0x92, - 0x67, 0xf5, 0x53, 0x5a, 0xee, 0x4f, 0x0b, 0xb0, 0xb8, 0x15, 0x52, 0x5b, 0x50, 0xd5, 0x67, 0xd2, - 0x1f, 0x8e, 0x28, 0x17, 0xe4, 0x35, 0xa8, 0x85, 0xea, 0xb3, 0x1b, 0xd9, 0x75, 0x02, 0x90, 0xcb, - 0x50, 0xd7, 0x76, 0x90, 0x72, 0xb7, 0xa0, 0xa0, 0x5d, 0x6d, 0x28, 0x53, 0x29, 0x0b, 0x6f, 0x17, - 0x56, 0x0b, 0x6b, 0x35, 0x73, 0x7e, 0x32, 0x67, 0xe1, 0x32, 0x24, 0xd8, 0x7c, 0x1c, 0x38, 0x68, - 0xb8, 0x55, 0x53, 0x35, 0xc8, 0xc7, 0xd0, 0x72, 0x7b, 0x56, 0x42, 0xcb, 0xd1, 0x74, 0xeb, 0x1b, - 0xcb, 0xeb, 0x2a, 0x7d, 0x5e, 0x8f, 0xd2, 0xe7, 0xf5, 0x27, 0x32, 0x84, 0x98, 0x4d, 0xb7, 0x97, - 0x6c, 0x0d, 0x0a, 0x3d, 0x64, 0xa1, 0xa3, 0x9c, 0x6b, 0xd5, 0x54, 0x0d, 0x19, 0xd7, 0x07, 0x54, - 0xd8, 0x16, 0x0b, 0xfc, 0x31, 0xda, 0x75, 0xd5, 0xac, 0x4a, 0xe0, 0x51, 0xe0, 0x8f, 0xc9, 0x55, - 0x98, 0xef, 0x3b, 0xd6, 0xd0, 0x1e, 0x71, 0x6a, 0xd1, 0xc0, 0xee, 0xf9, 0xca, 0x4f, 0x54, 0xcd, - 0x66, 0xdf, 0xd9, 0x93, 0xe8, 0x0e, 0x82, 0x64, 0x0d, 0x8c, 0x98, 0x8e, 0x53, 0x87, 0x05, 0x2e, - 0x47, 0xc7, 0x51, 0x32, 0x5b, 0x9a, 0x70, 0x5f, 0xa1, 0x13, 0x94, 0xb6, 0xeb, 0xe2, 0x29, 0x03, - 0x95, 0xb8, 0x69, 0xca, 0xdb, 0x0a, 0xed, 0xfc, 0x36, 0x07, 0x24, 0xb5, 0x37, 0x94, 0x0f, 0x59, - 0xc0, 0xe9, 0x2b, 0x36, 0xe1, 0x03, 0x28, 0xa6, 0xbc, 0xcb, 0x95, 0xcc, 0x7d, 0x8f, 0x44, 0xa1, - 0x5b, 0x41, 0x72, 0x19, 0x94, 0x07, 0xbc, 0xaf, 0x1d, 0x89, 0xfc, 0x24, 0xef, 0x41, 0xd1, 0xb5, - 0x85, 0x8d, 0x1b, 0x50, 0xdf, 0xb8, 0x7c, 0x8a, 0x9b, 0xc2, 0xd9, 0x21, 0x71, 0xe7, 0x4f, 0x39, - 0x30, 0xee, 0x52, 0xf1, 0xb5, 0x5a, 0xcd, 0x45, 0xa8, 0x69, 0x02, 0x1d, 0x9b, 0x6a, 0x91, 0xc7, - 0xd5, 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, 0xb3, 0x67, 0x5b, 0x4d, 0x46, 0x56, 0x9e, 0xcf, 0xca, 0xca, 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, 0xef, 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, - 0xb2, 0xae, 0xcf, 0x4e, 0x52, 0xc3, 0xb1, 0xd5, 0xa5, 0x42, 0x96, 0xa9, 0x44, 0xa8, 0x5c, 0x3e, - 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, 0x47, 0xa9, 0x57, - 0xec, 0xa8, 0x3b, 0xd0, 0x08, 0xd5, 0x3a, 0xf1, 0x7e, 0x85, 0x91, 0xbe, 0x6a, 0x4e, 0x60, 0x92, - 0x46, 0xba, 0xd7, 0x91, 0x60, 0x8a, 0xa6, 0xa1, 0x68, 0xd2, 0x18, 0x79, 0x07, 0x16, 0xdd, 0x90, - 0x0d, 0x77, 0x5e, 0x78, 0x5c, 0x24, 0x63, 0xb7, 0x9b, 0x48, 0x9a, 0xd5, 0x45, 0xae, 0x42, 0x2b, - 0x86, 0x95, 0xdc, 0x16, 0x12, 0x4f, 0xa1, 0x64, 0x03, 0x96, 0xf8, 0x33, 0x6f, 0xa8, 0xe2, 0x6a, - 0x4a, 0xf4, 0x3c, 0x52, 0x67, 0xf6, 0xe9, 0x0c, 0xd2, 0x88, 0x32, 0xc8, 0x95, 0x6d, 0x58, 0xce, - 0xde, 0xb8, 0x33, 0xdd, 0xaa, 0xbe, 0xca, 0xc7, 0x26, 0x1f, 0x27, 0x17, 0x32, 0xd7, 0x3c, 0x96, - 0xb0, 0xde, 0xcb, 0x48, 0x58, 0xaf, 0x9d, 0x66, 0x63, 0xff, 0x81, 0x19, 0x6b, 0x17, 0xf0, 0x26, - 0xa3, 0x93, 0x4d, 0x34, 0xd4, 0xb3, 0x64, 0x5a, 0x20, 0x99, 0x55, 0xbb, 0xf3, 0x55, 0x19, 0xce, - 0xe9, 0x85, 0x26, 0xbb, 0xf0, 0xad, 0x56, 0xdc, 0xe7, 0x50, 0x97, 0xa7, 0x31, 0x52, 0x4e, 0x19, - 0x95, 0x73, 0x86, 0x1c, 0x17, 0x24, 0xb7, 0x6a, 0x93, 0xf7, 0x61, 0x59, 0xd8, 0x61, 0x9f, 0x0a, - 0x6b, 0x3a, 0x02, 0x2a, 0xe7, 0xb0, 0xa4, 0x7a, 0xb7, 0x26, 0xab, 0x53, 0x36, 0x9c, 0x4f, 0x2e, - 0x9f, 0xfa, 0xb4, 0x5a, 0xc2, 0xe6, 0xcf, 0x78, 0xbb, 0x7a, 0x4a, 0xc6, 0x9d, 0x65, 0xbe, 0xe6, - 0xb9, 0x58, 0x52, 0x4a, 0xab, 0x58, 0x67, 0xd3, 0x82, 0x5d, 0x0b, 0xef, 0x08, 0xea, 0x46, 0x17, - 0xf9, 0x06, 0x77, 0x5f, 0xde, 0x15, 0xae, 0xc2, 0xbc, 0x60, 0xf1, 0x04, 0x52, 0x57, 0x89, 0xa6, - 0x60, 0x5a, 0x1a, 0xd2, 0xa5, 0x4d, 0xad, 0x3e, 0x65, 0x6a, 0x6f, 0x40, 0x4b, 0x6b, 0x20, 0x2a, - 0xd9, 0x35, 0xd4, 0x6e, 0x29, 0x74, 0x5b, 0x15, 0xee, 0xd2, 0x5e, 0xac, 0xf9, 0x0a, 0x2f, 0xd6, - 0x9a, 0xc1, 0x8b, 0xcd, 0xcf, 0xee, 0xc5, 0x8c, 0xb3, 0x78, 0xb1, 0x85, 0x33, 0x79, 0x31, 0x72, - 0xb2, 0x17, 0xeb, 0xfc, 0xaa, 0x00, 0x0b, 0x13, 0x41, 0xe8, 0x5b, 0x7d, 0x66, 0x5c, 0x68, 0x4f, - 0x04, 0xe0, 0xb4, 0xc9, 0x96, 0x4f, 0xa9, 0x99, 0x67, 0x7a, 0x0e, 0x73, 0x39, 0x1d, 0x70, 0x4f, - 0x33, 0xda, 0xca, 0x6c, 0x46, 0x5b, 0x7d, 0x95, 0xd1, 0xd6, 0x26, 0x8d, 0xb6, 0xf3, 0x87, 0x5c, - 0xec, 0xd4, 0xbe, 0x91, 0x04, 0x8c, 0xdc, 0x9a, 0xb8, 0x68, 0x5c, 0x7d, 0x75, 0x0a, 0x83, 0x7a, - 0x53, 0x19, 0xe9, 0x1d, 0x58, 0xbe, 0x4b, 0x45, 0xb4, 0x54, 0x69, 0x00, 0xb3, 0x65, 0x6f, 0xca, - 0xf6, 0xf2, 0x91, 0xed, 0x75, 0xbe, 0x0f, 0xf5, 0x54, 0x75, 0x88, 0xb4, 0xa1, 0x82, 0xef, 0x29, - 0xdd, 0x6d, 0x5d, 0x52, 0x8b, 0x9a, 0xe4, 0x83, 0xa4, 0xd0, 0x95, 0xc7, 0xbd, 0xbe, 0x98, 0x9d, - 0x3a, 0x4f, 0xd6, 0xb8, 0x3a, 0xbf, 0xc9, 0x41, 0x59, 0xcb, 0xbe, 0x0c, 0x75, 0x1a, 0x88, 0xd0, - 0xa3, 0xaa, 0xa0, 0xae, 0xe4, 0x83, 0x86, 0x76, 0x47, 0x03, 0x79, 0x3b, 0x89, 0xeb, 0x28, 0xd6, - 0x61, 0xc8, 0x06, 0x38, 0xcf, 0xa2, 0xd9, 0x8c, 0xd1, 0x3b, 0x21, 0x1b, 0x90, 0x2b, 0xd0, 0x48, - 0xc8, 0x04, 0x43, 0x8d, 0x16, 0xcd, 0x7a, 0x8c, 0x1d, 0x30, 0x69, 0xc4, 0x3e, 0xeb, 0x5b, 0x98, - 0x86, 0xa9, 0x74, 0xb2, 0xe2, 0xb3, 0xfe, 0x9e, 0xcc, 0xc4, 0x74, 0x57, 0xaa, 0x08, 0x29, 0xbb, - 0xa4, 0xb1, 0x74, 0x6e, 0x42, 0xe3, 0x3e, 0x1d, 0x63, 0x02, 0xb6, 0x67, 0x7b, 0xe1, 0xac, 0x99, - 0x45, 0xe7, 0x1f, 0x39, 0x00, 0xe4, 0x42, 0x4d, 0x92, 0x4b, 0x50, 0xeb, 0x31, 0xe6, 0x5b, 0xb8, - 0xb7, 0x92, 0xb9, 0x7a, 0x6f, 0xce, 0xac, 0x4a, 0x68, 0xdb, 0x16, 0x36, 0xb9, 0x08, 0x55, 0x2f, - 0x10, 0xaa, 0x57, 0x8a, 0x29, 0xdd, 0x9b, 0x33, 0x2b, 0x5e, 0x20, 0xb0, 0xf3, 0x12, 0xd4, 0x7c, - 0x16, 0xf4, 0x55, 0x2f, 0x96, 0x23, 0x25, 0xaf, 0x84, 0xb0, 0xfb, 0x32, 0xc0, 0xa1, 0xcf, 0x6c, - 0xcd, 0x2d, 0x57, 0x96, 0xbf, 0x37, 0x67, 0xd6, 0x10, 0x43, 0x82, 0x2b, 0x50, 0x77, 0xd9, 0xa8, - 0xe7, 0x53, 0x45, 0x21, 0x17, 0x98, 0xbb, 0x37, 0x67, 0x82, 0x02, 0x23, 0x12, 0x2e, 0x42, 0x2f, - 0x1a, 0x04, 0xcb, 0xad, 0x92, 0x44, 0x81, 0xd1, 0x30, 0xbd, 0xb1, 0xa0, 0x5c, 0x51, 0xc8, 0xf3, - 0xd7, 0x90, 0xc3, 0x20, 0x26, 0x09, 0x36, 0xcb, 0xca, 0x72, 0x3b, 0x7f, 0x2d, 0x6a, 0xf3, 0x51, - 0x4f, 0x27, 0xa7, 0x98, 0x4f, 0x54, 0x3e, 0xcb, 0xa7, 0xca, 0x67, 0x6f, 0x40, 0xcb, 0xe3, 0xd6, - 0x30, 0xf4, 0x06, 0x76, 0x38, 0xb6, 0xa4, 0xaa, 0x0b, 0xca, 0xa3, 0x7b, 0x7c, 0x4f, 0x81, 0xf7, - 0xe9, 0x98, 0xac, 0x42, 0xdd, 0xa5, 0xdc, 0x09, 0xbd, 0x21, 0xba, 0x5b, 0xb5, 0x9d, 0x69, 0x88, - 0xdc, 0x82, 0x9a, 0x9c, 0x8d, 0x7a, 0xd7, 0x2b, 0xe1, 0xa9, 0xbc, 0x94, 0x69, 0x9c, 0x72, 0xee, - 0x07, 0xe3, 0x21, 0x35, 0xab, 0xae, 0xfe, 0x22, 0x9b, 0x50, 0x97, 0x6c, 0x96, 0x7e, 0xfa, 0x53, - 0x6e, 0x2c, 0xfb, 0x4c, 0xa7, 0x6d, 0xc3, 0x04, 0xc9, 0xa5, 0xde, 0xfa, 0xc8, 0x36, 0x34, 0xd4, - 0x13, 0x88, 0x16, 0x52, 0x99, 0x55, 0x88, 0x7a, 0x39, 0xd1, 0x52, 0x96, 0xa1, 0x6c, 0xcb, 0x30, - 0xb6, 0xad, 0xeb, 0x2c, 0xba, 0x45, 0x3e, 0x80, 0x92, 0x2a, 0x8c, 0xd7, 0x70, 0x65, 0x97, 0x4f, - 0xae, 0xf0, 0x2a, 0x37, 0xa0, 0xa8, 0xc9, 0x67, 0xd0, 0xa0, 0x3e, 0xc5, 0xfa, 0x38, 0xea, 0x05, - 0x66, 0xd1, 0x4b, 0x5d, 0xb3, 0xa0, 0x6a, 0xb6, 0xa1, 0xe9, 0xd2, 0x43, 0x7b, 0xe4, 0x0b, 0x4b, - 0x19, 0x7d, 0xfd, 0x94, 0x82, 0x48, 0x62, 0xff, 0x66, 0x43, 0x73, 0x21, 0x84, 0xaf, 0xae, 0xdc, - 0x72, 0xc7, 0x81, 0x3d, 0xf0, 0x1c, 0x7d, 0xf1, 0xa8, 0x79, 0x7c, 0x5b, 0x01, 0x64, 0x0d, 0x0c, - 0x69, 0x03, 0x71, 0x22, 0x24, 0xad, 0x40, 0xe5, 0x06, 0x2d, 0x8f, 0xc7, 0x49, 0xce, 0x7d, 0x3a, - 0xee, 0xfc, 0x39, 0x07, 0xc6, 0xf4, 0x5b, 0x5d, 0x6c, 0x56, 0xb9, 0x94, 0x59, 0x4d, 0x19, 0x4c, - 0xfe, 0xb8, 0xc1, 0x24, 0xaa, 0x2e, 0x4c, 0xa8, 0xfa, 0x43, 0x28, 0xa3, 0xbd, 0x46, 0x8f, 0x1c, - 0xa7, 0x54, 0xd3, 0xa3, 0xb7, 0x42, 0x45, 0x4f, 0xde, 0x81, 0x25, 0x55, 0x24, 0x8b, 0x56, 0x6a, - 0x61, 0x07, 0x5a, 0x63, 0xd5, 0x24, 0xaa, 0x4f, 0xaf, 0x19, 0xf9, 0x3b, 0x2d, 0x68, 0xe0, 0x03, - 0x91, 0x76, 0xdb, 0x9d, 0xa7, 0xd0, 0xd4, 0x6d, 0x1d, 0x84, 0xa2, 0x30, 0x93, 0xfb, 0x97, 0xc2, - 0x4c, 0x3e, 0xb9, 0xe7, 0xff, 0x24, 0x07, 0xf5, 0x87, 0xbc, 0xbf, 0xc7, 0x38, 0xea, 0x52, 0xfa, - 0xcf, 0xe8, 0x55, 0x2c, 0xa5, 0xbb, 0xba, 0xc6, 0x30, 0x53, 0x5b, 0x82, 0xd2, 0x80, 0xf7, 0xbb, - 0xdb, 0x28, 0xa6, 0x61, 0xaa, 0x06, 0xe6, 0x6f, 0xbc, 0x7f, 0x37, 0x64, 0xa3, 0x61, 0x54, 0x8e, - 0x8a, 0xda, 0x32, 0xea, 0x24, 0x35, 0xf0, 0x22, 0x7a, 0xe4, 0x04, 0xe8, 0xdc, 0x86, 0x79, 0xfd, - 0x28, 0x16, 0xcf, 0x22, 0x6b, 0xe7, 0x64, 0xb4, 0xd6, 0xfd, 0x7a, 0x01, 0x71, 0xfb, 0xfa, 0x8f, - 0xa1, 0x91, 0x5e, 0x2d, 0xa9, 0x43, 0x65, 0x7f, 0xe4, 0x38, 0x94, 0x73, 0x63, 0x8e, 0xcc, 0x43, - 0x7d, 0x97, 0x09, 0x6b, 0x7f, 0x34, 0x1c, 0xb2, 0x50, 0x18, 0x39, 0xb2, 0x00, 0xcd, 0x5d, 0x66, - 0xed, 0xd1, 0x70, 0xe0, 0x71, 0xee, 0xb1, 0xc0, 0xc8, 0x93, 0x2a, 0x14, 0xef, 0xd8, 0x9e, 0x6f, - 0x14, 0xc8, 0x12, 0xcc, 0xe3, 0x99, 0xa3, 0x82, 0x86, 0xd6, 0x8e, 0xcc, 0x8d, 0x8c, 0x5f, 0x14, - 0xc8, 0x25, 0x68, 0xeb, 0xbd, 0xb0, 0x1e, 0xf5, 0x7e, 0x40, 0x1d, 0x61, 0x49, 0x91, 0x77, 0xd8, - 0x28, 0x70, 0x8d, 0x5f, 0x16, 0xae, 0xbf, 0x80, 0xc5, 0x8c, 0xb7, 0x09, 0x42, 0xa0, 0xb5, 0x79, - 0x7b, 0xeb, 0xfe, 0xe3, 0x3d, 0xab, 0xbb, 0xdb, 0x3d, 0xe8, 0xde, 0x7e, 0x60, 0xcc, 0x91, 0x25, - 0x30, 0x34, 0xb6, 0xf3, 0x74, 0x67, 0xeb, 0xf1, 0x41, 0x77, 0xf7, 0xae, 0x91, 0x4b, 0x51, 0xee, - 0x3f, 0xde, 0xda, 0xda, 0xd9, 0xdf, 0x37, 0xf2, 0x72, 0xde, 0x1a, 0xbb, 0x73, 0xbb, 0xfb, 0xc0, - 0x28, 0xa4, 0x88, 0x0e, 0xba, 0x0f, 0x77, 0x1e, 0x3d, 0x3e, 0x30, 0x8a, 0xd7, 0x9f, 0xc4, 0x77, - 0xd6, 0xc9, 0xa1, 0xeb, 0x50, 0x49, 0xc6, 0x6c, 0x42, 0x2d, 0x3d, 0x98, 0xd4, 0x4e, 0x3c, 0x8a, - 0x5c, 0xb9, 0x12, 0x5f, 0x87, 0x4a, 0x22, 0xf7, 0xa9, 0x3c, 0x4f, 0x53, 0x0f, 0xd0, 0x00, 0xe5, - 0x7d, 0x11, 0xb2, 0xa0, 0x6f, 0xcc, 0xa1, 0x0c, 0xaa, 0xb4, 0x87, 0x02, 0x37, 0xa5, 0x2a, 0xa8, - 0x6b, 0xe4, 0x49, 0x0b, 0x60, 0xe7, 0x39, 0x0d, 0xc4, 0xc8, 0xf6, 0xfd, 0xb1, 0x51, 0x90, 0xed, - 0xad, 0x11, 0x17, 0x6c, 0xe0, 0xbd, 0xa4, 0xae, 0x51, 0xbc, 0xfe, 0xb7, 0x1c, 0x54, 0x23, 0x9f, - 0x22, 0x47, 0xdf, 0x65, 0x01, 0x35, 0xe6, 0xe4, 0xd7, 0x26, 0x63, 0xbe, 0x91, 0x93, 0x5f, 0xdd, - 0x40, 0x7c, 0x68, 0xe4, 0x49, 0x0d, 0x4a, 0xdd, 0x40, 0xbc, 0x7b, 0xd3, 0x28, 0xe8, 0xcf, 0xf7, - 0x36, 0x8c, 0xa2, 0xfe, 0xbc, 0xf9, 0xbe, 0x51, 0x92, 0x9f, 0x77, 0x64, 0x78, 0x33, 0x40, 0x4e, - 0x6e, 0x1b, 0xe3, 0x98, 0x51, 0xd7, 0x13, 0xf5, 0x82, 0xbe, 0xb1, 0x24, 0xe7, 0xf6, 0xc4, 0x0e, - 0xb7, 0x8e, 0xec, 0xd0, 0x38, 0x27, 0xe9, 0x6f, 0x87, 0xa1, 0x3d, 0x36, 0x96, 0xe5, 0x28, 0x9f, - 0x73, 0x16, 0x18, 0xe7, 0x89, 0x01, 0x8d, 0x4d, 0x2f, 0xb0, 0xc3, 0xf1, 0x13, 0xea, 0x08, 0x16, - 0x1a, 0xae, 0xd4, 0x3c, 0x8a, 0xd5, 0x00, 0x95, 0x16, 0x83, 0xc0, 0xbb, 0x37, 0x35, 0x74, 0x88, - 0x9b, 0x31, 0x89, 0xf5, 0xc9, 0x39, 0x58, 0xd8, 0x1f, 0xda, 0x21, 0xa7, 0x69, 0xee, 0xa3, 0xeb, - 0x4f, 0x00, 0x12, 0x17, 0x2c, 0x87, 0xc3, 0x96, 0xba, 0x0f, 0xb8, 0xc6, 0x1c, 0x4a, 0x8f, 0x11, - 0x39, 0xeb, 0x5c, 0x0c, 0x6d, 0x87, 0x6c, 0x38, 0x94, 0x50, 0x3e, 0xe6, 0x43, 0x88, 0xba, 0x46, - 0x61, 0xe3, 0x8f, 0x25, 0x58, 0x7c, 0x88, 0x07, 0x5f, 0x19, 0xdf, 0x3e, 0x0d, 0x9f, 0x7b, 0x0e, - 0x25, 0x0e, 0x34, 0xd2, 0xcf, 0x14, 0x24, 0xfb, 0x5a, 0x9f, 0xf1, 0x92, 0xb1, 0xf2, 0xd6, 0xab, - 0x2a, 0x9e, 0xfa, 0x90, 0x75, 0xe6, 0xc8, 0xf7, 0xa0, 0x16, 0x97, 0xb4, 0x49, 0xf6, 0x3f, 0x0d, - 0xd3, 0x25, 0xef, 0xb3, 0x88, 0xef, 0x41, 0x3d, 0x55, 0x07, 0x26, 0xd9, 0x9c, 0xc7, 0xeb, 0xd0, - 0x2b, 0x6b, 0xaf, 0x26, 0x8c, 0xc7, 0xa0, 0xd0, 0x48, 0x97, 0x58, 0x4f, 0xd0, 0x53, 0x46, 0x6d, - 0x77, 0xe5, 0xda, 0x0c, 0x94, 0xf1, 0x30, 0x47, 0xd0, 0x9c, 0x48, 0xd4, 0xc9, 0xb5, 0x99, 0xeb, - 0x91, 0x2b, 0xd7, 0x67, 0x21, 0x8d, 0x47, 0xea, 0x03, 0x24, 0x79, 0x3f, 0x79, 0xfb, 0xa4, 0x4d, - 0xc9, 0xb8, 0x18, 0x9c, 0x71, 0xa0, 0x3d, 0x28, 0x61, 0x3c, 0x22, 0xd9, 0x91, 0x27, 0x1d, 0xbb, - 0x56, 0x3a, 0xa7, 0x91, 0x44, 0x12, 0x37, 0x3f, 0xfa, 0xe2, 0xff, 0xfa, 0x9e, 0x38, 0x1a, 0xf5, - 0xd6, 0x1d, 0x36, 0xb8, 0xf1, 0xd2, 0xf3, 0x7d, 0xef, 0xa5, 0xa0, 0xce, 0xd1, 0x0d, 0xc5, 0xfc, - 0xbf, 0x8a, 0xed, 0x86, 0xc3, 0x42, 0xfd, 0x37, 0xd8, 0x0d, 0x85, 0x0c, 0x7b, 0xbd, 0x32, 0xb6, - 0xdf, 0xfb, 0x67, 0x00, 0x00, 0x00, 0xff, 0xff, 0x05, 0x08, 0xff, 0x95, 0x50, 0x26, 0x00, 0x00, + // 3037 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x5a, 0x4b, 0x73, 0x1b, 0xc7, + 0x11, 0x26, 0xde, 0x40, 0x2f, 0x00, 0x2e, 0x87, 0x14, 0x05, 0x51, 0x96, 0x45, 0x21, 0xb6, 0x4c, + 0xc9, 0x15, 0x4a, 0xa6, 0x6d, 0xc5, 0x56, 0xc5, 0x0f, 0xf1, 0x21, 0x09, 0x96, 0x44, 0xb1, 0x96, + 0x94, 0x4a, 0xe5, 0x3c, 0xb6, 0x16, 0xbb, 0x43, 0x60, 0xa3, 0xc5, 0x0e, 0xb2, 0xb3, 0x90, 0x05, + 0x55, 0x25, 0x95, 0xaa, 0x5c, 0x72, 0xcc, 0x21, 0xa7, 0xfc, 0x83, 0xdc, 0x92, 0x4a, 0x39, 0x87, + 0xfc, 0x83, 0xa4, 0x72, 0xcd, 0x6f, 0x48, 0xe5, 0x94, 0xdc, 0x72, 0x4d, 0x4d, 0xcf, 0xec, 0x03, + 0xe0, 0x92, 0x04, 0x53, 0xae, 0x38, 0xce, 0x6d, 0xe7, 0x9b, 0xee, 0x9e, 0x99, 0x9e, 0x9e, 0xee, + 0x9e, 0x9e, 0x85, 0x7a, 0xd7, 0xb2, 0x9f, 0x8f, 0x86, 0xeb, 0xc3, 0x80, 0x85, 0x8c, 0x2c, 0x0e, + 0x5c, 0xef, 0xc5, 0x88, 0xcb, 0xd6, 0xba, 0xec, 0x5a, 0x79, 0xad, 0xc7, 0x58, 0xcf, 0xa3, 0x37, + 0x10, 0xec, 0x8e, 0x0e, 0x6f, 0xf0, 0x30, 0x18, 0xd9, 0xa1, 0x24, 0x6a, 0xff, 0x2d, 0x07, 0xb5, + 0x8e, 0xef, 0xd0, 0x97, 0x1d, 0xff, 0x90, 0x91, 0x4b, 0x00, 0x87, 0x2e, 0xf5, 0x1c, 0xd3, 0xb7, + 0x06, 0xb4, 0x95, 0x5b, 0xcd, 0xad, 0xd5, 0x8c, 0x1a, 0x22, 0xbb, 0xd6, 0x80, 0x8a, 0x6e, 0x57, + 0xd0, 0xca, 0xee, 0xbc, 0xec, 0x46, 0x64, 0xb2, 0x3b, 0x1c, 0x0f, 0x69, 0xab, 0x90, 0xea, 0x3e, + 0x18, 0x0f, 0x29, 0xd9, 0x84, 0xf2, 0xd0, 0x0a, 0xac, 0x01, 0x6f, 0x15, 0x57, 0x0b, 0x6b, 0xda, + 0xc6, 0xf5, 0xf5, 0x8c, 0xe9, 0xae, 0xc7, 0x93, 0x59, 0xdf, 0x43, 0xe2, 0x1d, 0x3f, 0x0c, 0xc6, + 0x86, 0xe2, 0x5c, 0xf9, 0x10, 0xb4, 0x14, 0x4c, 0x74, 0x28, 0x3c, 0xa7, 0x63, 0x35, 0x51, 0xf1, + 0x49, 0x96, 0xa0, 0xf4, 0xc2, 0xf2, 0x46, 0xd1, 0xec, 0x64, 0xe3, 0x76, 0xfe, 0x83, 0x5c, 0xfb, + 0xaf, 0x55, 0x58, 0xda, 0x62, 0x9e, 0x47, 0xed, 0xd0, 0x65, 0xfe, 0x26, 0x8e, 0x86, 0x8b, 0x6e, + 0x42, 0xde, 0x75, 0x94, 0x8c, 0xbc, 0xeb, 0x90, 0x7b, 0x00, 0x3c, 0xb4, 0x42, 0x6a, 0xda, 0xcc, + 0x91, 0x72, 0x9a, 0x1b, 0x6b, 0x99, 0x73, 0x95, 0x42, 0x0e, 0x2c, 0xfe, 0x7c, 0x5f, 0x30, 0x6c, + 0x31, 0x87, 0x1a, 0x35, 0x1e, 0x7d, 0x92, 0x36, 0xd4, 0x69, 0x10, 0xb0, 0xe0, 0x11, 0xe5, 0xdc, + 0xea, 0x45, 0x1a, 0x99, 0xc0, 0x84, 0xce, 0x78, 0x68, 0x05, 0xa1, 0x19, 0xba, 0x03, 0xda, 0x2a, + 0xae, 0xe6, 0xd6, 0x0a, 0x28, 0x22, 0x08, 0x0f, 0xdc, 0x01, 0x25, 0x17, 0xa0, 0x4a, 0x7d, 0x47, + 0x76, 0x96, 0xb0, 0xb3, 0x42, 0x7d, 0x07, 0xbb, 0x56, 0xa0, 0x3a, 0x0c, 0x58, 0x2f, 0xa0, 0x9c, + 0xb7, 0xca, 0xab, 0xb9, 0xb5, 0x92, 0x11, 0xb7, 0xc9, 0xb7, 0xa0, 0x61, 0xc7, 0x4b, 0x35, 0x5d, + 0xa7, 0x55, 0x41, 0xde, 0x7a, 0x02, 0x76, 0x1c, 0x72, 0x1e, 0x2a, 0x4e, 0x57, 0x6e, 0x65, 0x15, + 0x67, 0x56, 0x76, 0xba, 0xb8, 0x8f, 0x6f, 0xc1, 0x7c, 0x8a, 0x1b, 0x09, 0x6a, 0x48, 0xd0, 0x4c, + 0x60, 0x24, 0xfc, 0x08, 0xca, 0xdc, 0xee, 0xd3, 0x81, 0xd5, 0x82, 0xd5, 0xdc, 0x9a, 0xb6, 0xf1, + 0x66, 0xa6, 0x96, 0x12, 0xa5, 0xef, 0x23, 0xb1, 0xa1, 0x98, 0x70, 0xed, 0x7d, 0x2b, 0x70, 0xb8, + 0xe9, 0x8f, 0x06, 0x2d, 0x0d, 0xd7, 0x50, 0x93, 0xc8, 0xee, 0x68, 0x40, 0x0c, 0x58, 0xb0, 0x99, + 0xcf, 0x5d, 0x1e, 0x52, 0xdf, 0x1e, 0x9b, 0x1e, 0x7d, 0x41, 0xbd, 0x56, 0x1d, 0xb7, 0xe3, 0xb8, + 0x81, 0x62, 0xea, 0x87, 0x82, 0xd8, 0xd0, 0xed, 0x29, 0x84, 0x3c, 0x81, 0x85, 0xa1, 0x15, 0x84, + 0x2e, 0xae, 0x4c, 0xb2, 0xf1, 0x56, 0x03, 0xcd, 0x31, 0x7b, 0x8b, 0xf7, 0x22, 0xea, 0xc4, 0x60, + 0x0c, 0x7d, 0x38, 0x09, 0x72, 0x72, 0x0d, 0x74, 0x49, 0x8f, 0x3b, 0xc5, 0x43, 0x6b, 0x30, 0x6c, + 0x35, 0x57, 0x73, 0x6b, 0x45, 0x63, 0x5e, 0xe2, 0x07, 0x11, 0x4c, 0x08, 0x14, 0xb9, 0xfb, 0x8a, + 0xb6, 0xe6, 0x71, 0x47, 0xf0, 0x9b, 0x5c, 0x84, 0x5a, 0xdf, 0xe2, 0x26, 0x1e, 0x95, 0x96, 0xbe, + 0x9a, 0x5b, 0xab, 0x1a, 0xd5, 0xbe, 0xc5, 0xf1, 0x28, 0x90, 0x4f, 0x40, 0x93, 0xa7, 0xca, 0xf5, + 0x0f, 0x19, 0x6f, 0x2d, 0xe0, 0x64, 0x5f, 0x3f, 0xf9, 0xec, 0x18, 0xf2, 0x20, 0x8a, 0x4f, 0x2e, + 0xd4, 0xec, 0x31, 0xcb, 0x31, 0xd1, 0x30, 0x5b, 0x44, 0x1e, 0x4b, 0x81, 0xa0, 0xd1, 0x92, 0xdb, + 0x70, 0x41, 0xcd, 0x7d, 0xd8, 0x1f, 0x73, 0xd7, 0xb6, 0xbc, 0xd4, 0x22, 0x16, 0x71, 0x11, 0xe7, + 0x25, 0xc1, 0x9e, 0xea, 0x4f, 0x16, 0x13, 0xc0, 0xa2, 0xdd, 0xb7, 0x7c, 0x9f, 0x7a, 0xa6, 0xdd, + 0xa7, 0xf6, 0xf3, 0x21, 0x73, 0xfd, 0x90, 0xb7, 0x96, 0x70, 0x8e, 0x77, 0x4e, 0xb1, 0x86, 0x44, + 0xa3, 0xeb, 0x5b, 0x52, 0xc8, 0x56, 0x22, 0x43, 0x1e, 0x7b, 0x62, 0x1f, 0xe9, 0x20, 0xf7, 0x40, + 0xf3, 0x6e, 0x9a, 0x9c, 0xf6, 0x06, 0x54, 0x8c, 0x75, 0x0e, 0xc7, 0xba, 0x9a, 0x39, 0xd6, 0xbe, + 0x24, 0x4a, 0x6d, 0x1d, 0x78, 0x37, 0x15, 0xc8, 0x57, 0x76, 0xe0, 0xfc, 0x31, 0xe3, 0x9e, 0xc9, + 0xaf, 0xfc, 0x22, 0x0f, 0x8b, 0x19, 0x56, 0x42, 0xae, 0x40, 0x3d, 0x31, 0x35, 0xe5, 0x60, 0x0a, + 0x86, 0x16, 0x63, 0x1d, 0x87, 0xbc, 0x09, 0xcd, 0x84, 0x24, 0xe5, 0x53, 0x1b, 0x31, 0x8a, 0xc7, + 0xec, 0xc8, 0x69, 0x2e, 0x64, 0x9c, 0xe6, 0xc7, 0x30, 0xaf, 0x74, 0x12, 0xdb, 0x75, 0xf1, 0x4c, + 0xaa, 0x69, 0xf2, 0x34, 0xc4, 0x63, 0x43, 0x2d, 0xa5, 0x0c, 0x75, 0xd2, 0x94, 0xca, 0x53, 0xa6, + 0xd4, 0xfe, 0x43, 0x01, 0x16, 0x8e, 0x08, 0xc6, 0x63, 0xae, 0x66, 0x16, 0xab, 0xa1, 0xa6, 0x90, + 0x8e, 0x73, 0x74, 0x75, 0xf9, 0x8c, 0xd5, 0x4d, 0x2b, 0xb3, 0x70, 0x54, 0x99, 0xaf, 0x83, 0xe6, + 0x8f, 0x06, 0x26, 0x3b, 0x34, 0x03, 0xf6, 0x05, 0x8f, 0x5c, 0xa9, 0x3f, 0x1a, 0x3c, 0x3e, 0x34, + 0xd8, 0x17, 0x9c, 0xdc, 0x86, 0x4a, 0xd7, 0xf5, 0x3d, 0xd6, 0xe3, 0xad, 0x12, 0x2a, 0x66, 0x35, + 0x53, 0x31, 0x77, 0x45, 0xb4, 0xdb, 0x44, 0x42, 0x23, 0x62, 0x20, 0x1f, 0x03, 0xba, 0x75, 0x8e, + 0xdc, 0xe5, 0x19, 0xb9, 0x13, 0x16, 0xc1, 0xef, 0x50, 0x2f, 0xb4, 0x90, 0xbf, 0x32, 0x2b, 0x7f, + 0xcc, 0x12, 0xef, 0x45, 0x35, 0xb5, 0x17, 0x17, 0xa0, 0xda, 0x0b, 0xd8, 0x68, 0x28, 0xd4, 0x51, + 0x93, 0xa1, 0x01, 0xdb, 0x1d, 0x47, 0x84, 0x06, 0x29, 0x8f, 0x3a, 0xe8, 0x99, 0xab, 0x46, 0xdc, + 0x26, 0x8b, 0x50, 0x72, 0xb9, 0xe9, 0xdd, 0x44, 0x7f, 0x5b, 0x35, 0x8a, 0x2e, 0x7f, 0x78, 0xb3, + 0xfd, 0xfb, 0x02, 0xc0, 0xff, 0x77, 0x44, 0x24, 0x50, 0xc4, 0x03, 0x56, 0xc1, 0x11, 0xf1, 0x3b, + 0xd3, 0x6b, 0x57, 0xb3, 0xbd, 0xf6, 0x33, 0x20, 0x29, 0x23, 0x8d, 0x0e, 0x58, 0x0d, 0x77, 0xf2, + 0xda, 0xcc, 0x7e, 0xce, 0x58, 0xb0, 0xa7, 0xd0, 0x64, 0x6b, 0x21, 0xb5, 0xb5, 0x6f, 0x42, 0x53, + 0x8a, 0x34, 0x5f, 0xd0, 0x80, 0xbb, 0xcc, 0xc7, 0xcd, 0xaa, 0x19, 0x0d, 0x89, 0x3e, 0x95, 0x60, + 0xfb, 0xfb, 0x70, 0x21, 0x19, 0x05, 0xe3, 0x5b, 0x6a, 0x0f, 0x3f, 0x81, 0x92, 0x0c, 0x18, 0xb9, + 0xb3, 0x4e, 0x52, 0xf2, 0xb5, 0x3f, 0x87, 0x56, 0xec, 0xd6, 0xa6, 0x85, 0x7f, 0x3c, 0x29, 0x7c, + 0xf6, 0xd0, 0xa9, 0x64, 0x3f, 0x85, 0x65, 0xe5, 0x27, 0xa6, 0x25, 0x7f, 0x77, 0x52, 0xf2, 0xac, + 0xce, 0x4b, 0xc9, 0xfd, 0x79, 0x01, 0x16, 0xb7, 0x02, 0x6a, 0x85, 0x54, 0xf6, 0x19, 0xf4, 0xc7, + 0x23, 0xca, 0x43, 0xf2, 0x1a, 0xd4, 0x02, 0xf9, 0xd9, 0x89, 0xec, 0x3a, 0x01, 0xc8, 0x65, 0xd0, + 0x94, 0x1d, 0xa4, 0x7c, 0x30, 0x48, 0x68, 0x57, 0x19, 0xca, 0x54, 0x42, 0xc4, 0x5b, 0x85, 0xd5, + 0xc2, 0x5a, 0xcd, 0x98, 0x9f, 0xcc, 0x88, 0xb8, 0x88, 0x13, 0x16, 0x1f, 0xfb, 0x36, 0x1a, 0x6e, + 0xd5, 0x90, 0x0d, 0xf2, 0x11, 0x34, 0x9d, 0xae, 0x99, 0xd0, 0x72, 0x34, 0x5d, 0x6d, 0x63, 0x79, + 0x5d, 0x26, 0xe7, 0xeb, 0x51, 0x72, 0xbe, 0xfe, 0x54, 0xc4, 0x15, 0xa3, 0xe1, 0x74, 0x93, 0xad, + 0x41, 0xa1, 0x87, 0x2c, 0xb0, 0xa5, 0xc7, 0xad, 0x1a, 0xb2, 0x21, 0xb2, 0x86, 0x01, 0x0d, 0x2d, + 0x93, 0xf9, 0xde, 0x18, 0xed, 0xba, 0x6a, 0x54, 0x05, 0xf0, 0xd8, 0xf7, 0xc6, 0xe4, 0x2a, 0xcc, + 0xf7, 0x6c, 0x73, 0x68, 0x8d, 0x38, 0x35, 0xa9, 0x6f, 0x75, 0x3d, 0xe9, 0x3c, 0xaa, 0x46, 0xa3, + 0x67, 0xef, 0x09, 0x74, 0x07, 0x41, 0xb2, 0x06, 0x7a, 0x4c, 0xc7, 0xa9, 0xcd, 0x7c, 0x87, 0xa3, + 0x37, 0x29, 0x19, 0x4d, 0x45, 0xb8, 0x2f, 0xd1, 0x09, 0x4a, 0xcb, 0x71, 0xf0, 0x94, 0x81, 0x4c, + 0x0b, 0x15, 0xe5, 0x1d, 0x89, 0xb6, 0x7f, 0x9b, 0x03, 0x92, 0xda, 0x1b, 0xca, 0x87, 0xcc, 0xe7, + 0xf4, 0x94, 0x4d, 0x78, 0x1f, 0x8a, 0x29, 0xef, 0x72, 0x25, 0x73, 0xdf, 0x23, 0x51, 0xe8, 0x56, + 0x90, 0x5c, 0x44, 0xea, 0x01, 0xef, 0x29, 0x47, 0x22, 0x3e, 0xc9, 0xbb, 0x50, 0x74, 0xac, 0xd0, + 0xc2, 0x0d, 0xd0, 0x36, 0x2e, 0x9f, 0xe0, 0xa6, 0x70, 0x76, 0x48, 0xdc, 0xfe, 0x73, 0x0e, 0xf4, + 0x7b, 0x34, 0xfc, 0x4a, 0xad, 0xe6, 0x22, 0xd4, 0x14, 0x81, 0x0a, 0x58, 0xb5, 0xc8, 0x0d, 0x2b, + 0xee, 0x91, 0xfd, 0x9c, 0x86, 0x92, 0xbb, 0xa8, 0xb8, 0x11, 0x42, 0x6e, 0x02, 0xc5, 0xa1, 0x15, + 0xf6, 0xd1, 0x50, 0x6a, 0x06, 0x7e, 0x0b, 0xbf, 0xf0, 0x85, 0x1b, 0xf6, 0xd9, 0x28, 0x34, 0x1d, + 0x1a, 0x5a, 0xae, 0xa7, 0x0c, 0xa2, 0xa1, 0xd0, 0x6d, 0x04, 0xdb, 0xdf, 0x03, 0xf2, 0xd0, 0xe5, + 0x51, 0x20, 0x9f, 0x6d, 0x35, 0x19, 0x39, 0x7f, 0x3e, 0x2b, 0xe7, 0x6f, 0xff, 0x2e, 0x07, 0x8b, + 0x13, 0xd2, 0xbf, 0xae, 0xdd, 0x2d, 0xcc, 0xbe, 0xbb, 0x07, 0xb0, 0xb8, 0x4d, 0x3d, 0xfa, 0xd5, + 0x7a, 0x85, 0xf6, 0x4f, 0x60, 0x69, 0x52, 0xea, 0x7f, 0x55, 0x13, 0xed, 0x7f, 0x96, 0x60, 0xc9, + 0xa0, 0x3c, 0x64, 0xc1, 0xd7, 0xe6, 0xec, 0xde, 0x86, 0x54, 0x40, 0x33, 0xf9, 0xe8, 0xf0, 0xd0, + 0x7d, 0xa9, 0x4c, 0x39, 0x25, 0x63, 0x1f, 0x71, 0xc2, 0x26, 0x42, 0x68, 0x40, 0xa5, 0x64, 0x99, + 0x8a, 0x7d, 0x7a, 0x9c, 0x1a, 0x8e, 0xac, 0x2e, 0x15, 0xb2, 0x0c, 0x29, 0x42, 0xde, 0x14, 0x52, + 0x13, 0x51, 0x78, 0xe2, 0x8a, 0xcb, 0x69, 0x57, 0x3c, 0x75, 0xf0, 0x2a, 0xc7, 0x1e, 0xbc, 0x6a, + 0xea, 0xe0, 0x1d, 0xf5, 0xdf, 0xb5, 0xb3, 0xf8, 0xef, 0x15, 0x88, 0x1d, 0x73, 0x94, 0x8f, 0xc5, + 0x8e, 0xba, 0x0d, 0xf5, 0x40, 0xae, 0x13, 0x6f, 0x6f, 0x2a, 0x2d, 0x9b, 0xc0, 0x04, 0x8d, 0x70, + 0xaf, 0xa3, 0x90, 0x49, 0x9a, 0xba, 0xa4, 0x49, 0x63, 0xe4, 0x26, 0x2c, 0x3a, 0x01, 0x1b, 0xee, + 0xbc, 0x74, 0x79, 0x98, 0x8c, 0xdd, 0x6a, 0x20, 0x69, 0x56, 0x17, 0xb9, 0x0a, 0xcd, 0x18, 0x96, + 0x72, 0x9b, 0x48, 0x3c, 0x85, 0x92, 0x0d, 0x58, 0xe2, 0xcf, 0xdd, 0xa1, 0x8c, 0xab, 0x29, 0xd1, + 0xf3, 0x48, 0x9d, 0xd9, 0xa7, 0x32, 0x48, 0x3d, 0xca, 0x20, 0x57, 0xb6, 0x61, 0x39, 0x7b, 0xe3, + 0xce, 0x74, 0xd5, 0xfa, 0x32, 0x1f, 0x9b, 0x7c, 0x9c, 0x5c, 0x88, 0x5c, 0xf3, 0x48, 0xc2, 0x7a, + 0x3f, 0x23, 0x61, 0xbd, 0x76, 0x92, 0x8d, 0xfd, 0x0f, 0x66, 0xac, 0x1d, 0xc0, 0xeb, 0x8d, 0x4a, + 0x36, 0xd1, 0x50, 0xcf, 0x92, 0x69, 0x81, 0x60, 0x96, 0xed, 0xf6, 0x97, 0x65, 0x38, 0xa7, 0x16, + 0x9a, 0xec, 0xc2, 0x37, 0x5a, 0x71, 0x9f, 0x81, 0x26, 0x4e, 0x63, 0xa4, 0x9c, 0x32, 0x2a, 0xe7, + 0x0c, 0x39, 0x2e, 0x08, 0x6e, 0xd9, 0x26, 0xef, 0xc1, 0x72, 0x68, 0x05, 0x3d, 0x1a, 0x9a, 0xd3, + 0x11, 0x50, 0x3a, 0x87, 0x25, 0xd9, 0xbb, 0x35, 0x59, 0xfb, 0xb2, 0xe0, 0x7c, 0x72, 0x23, 0x55, + 0xa7, 0xd5, 0x0c, 0x2d, 0xfe, 0x9c, 0xb7, 0xaa, 0x27, 0x64, 0xdc, 0x59, 0xe6, 0x6b, 0x9c, 0x8b, + 0x25, 0xa5, 0xb4, 0x8a, 0x55, 0x3c, 0x25, 0xd8, 0x31, 0xf1, 0x8e, 0x20, 0xaf, 0x79, 0x91, 0x6f, + 0x70, 0xf6, 0xc5, 0x5d, 0xe1, 0x2a, 0xcc, 0x87, 0x2c, 0x9e, 0x40, 0xea, 0x2a, 0xd1, 0x08, 0x99, + 0x92, 0x86, 0x74, 0x69, 0x53, 0xd3, 0xa6, 0x4c, 0xed, 0x0d, 0x68, 0x2a, 0x0d, 0x44, 0x05, 0xc1, + 0xba, 0xdc, 0x2d, 0x89, 0x6e, 0xcb, 0xb2, 0x60, 0xda, 0x8b, 0x35, 0x4e, 0xf1, 0x62, 0xcd, 0x19, + 0xbc, 0xd8, 0xfc, 0xec, 0x5e, 0x4c, 0x3f, 0x8b, 0x17, 0x5b, 0x38, 0x93, 0x17, 0x23, 0xc7, 0x7b, + 0xb1, 0xf6, 0xaf, 0x0b, 0xb0, 0x30, 0x11, 0x84, 0xbe, 0xd1, 0x67, 0xc6, 0x81, 0xd6, 0x44, 0x00, + 0x4e, 0x9b, 0x6c, 0xf9, 0x84, 0x8a, 0x7c, 0xa6, 0xe7, 0x30, 0x96, 0xd3, 0x01, 0xf7, 0x24, 0xa3, + 0xad, 0xcc, 0x66, 0xb4, 0xd5, 0xd3, 0x8c, 0xb6, 0x36, 0x69, 0xb4, 0xed, 0x3f, 0xe6, 0x62, 0xa7, + 0xf6, 0xb5, 0x24, 0x60, 0xe4, 0xf6, 0xc4, 0x45, 0xe3, 0xea, 0xe9, 0x29, 0x0c, 0xea, 0x4d, 0x66, + 0xa4, 0x77, 0x61, 0xf9, 0x1e, 0x0d, 0xa3, 0xa5, 0x0a, 0x03, 0x98, 0x2d, 0x7b, 0x93, 0xb6, 0x97, + 0x8f, 0x6c, 0xaf, 0xfd, 0x43, 0xd0, 0x52, 0x25, 0x23, 0xd2, 0x82, 0x0a, 0xbe, 0xd6, 0x74, 0xb6, + 0x55, 0x9d, 0x2d, 0x6a, 0x92, 0xf7, 0x93, 0xea, 0x57, 0x1e, 0xf7, 0xfa, 0x62, 0x76, 0xea, 0x3c, + 0x59, 0xf8, 0x6a, 0xff, 0x26, 0x07, 0x65, 0x25, 0xfb, 0x32, 0x68, 0xd4, 0x0f, 0x03, 0x97, 0xca, + 0x72, 0xbd, 0x94, 0x0f, 0x0a, 0xda, 0x1d, 0x0d, 0xc4, 0xed, 0x24, 0xae, 0xa3, 0x98, 0x87, 0x01, + 0x1b, 0xe0, 0x3c, 0x8b, 0x46, 0x23, 0x46, 0xef, 0x06, 0x6c, 0x40, 0xae, 0x40, 0x3d, 0x21, 0x0b, + 0x19, 0x6a, 0xb4, 0x68, 0x68, 0x31, 0x76, 0xc0, 0x84, 0x11, 0x7b, 0xac, 0x67, 0x62, 0x1a, 0x26, + 0xd3, 0xc9, 0x8a, 0xc7, 0x7a, 0x7b, 0x22, 0x13, 0x53, 0x5d, 0xa9, 0xca, 0xa4, 0xe8, 0x12, 0xc6, + 0xd2, 0xbe, 0x05, 0xf5, 0x07, 0x74, 0x8c, 0x09, 0xd8, 0x9e, 0xe5, 0x06, 0xb3, 0x66, 0x16, 0xed, + 0x7f, 0xe5, 0x00, 0x90, 0x0b, 0x35, 0x49, 0x2e, 0x41, 0xad, 0xcb, 0x98, 0x67, 0xe2, 0xde, 0x0a, + 0xe6, 0xea, 0xfd, 0x39, 0xa3, 0x2a, 0xa0, 0x6d, 0x2b, 0xb4, 0xc8, 0x45, 0xa8, 0xba, 0x7e, 0x28, + 0x7b, 0x85, 0x98, 0xd2, 0xfd, 0x39, 0xa3, 0xe2, 0xfa, 0x21, 0x76, 0x5e, 0x82, 0x9a, 0xc7, 0xfc, + 0x9e, 0xec, 0xc5, 0x1a, 0xa5, 0xe0, 0x15, 0x10, 0x76, 0x5f, 0x06, 0x38, 0xf4, 0x98, 0xa5, 0xb8, + 0xc5, 0xca, 0xf2, 0xf7, 0xe7, 0x8c, 0x1a, 0x62, 0x48, 0x70, 0x05, 0x34, 0x87, 0x8d, 0xba, 0x1e, + 0x95, 0x14, 0x62, 0x81, 0xb9, 0xfb, 0x73, 0x06, 0x48, 0x30, 0x22, 0xe1, 0x61, 0xe0, 0x46, 0x83, + 0x60, 0x0d, 0x56, 0x90, 0x48, 0x30, 0x1a, 0xa6, 0x3b, 0x0e, 0x29, 0x97, 0x14, 0xe2, 0xfc, 0xd5, + 0xc5, 0x30, 0x88, 0x09, 0x82, 0xcd, 0xb2, 0xb4, 0xdc, 0xf6, 0xdf, 0x8b, 0xca, 0x7c, 0xe4, 0xc3, + 0xcc, 0x09, 0xe6, 0x13, 0x95, 0xcf, 0xf2, 0xa9, 0xf2, 0xd9, 0x1b, 0xd0, 0x74, 0xb9, 0x39, 0x0c, + 0xdc, 0x81, 0x15, 0x8c, 0x4d, 0xa1, 0xea, 0x82, 0xf4, 0xe8, 0x2e, 0xdf, 0x93, 0xe0, 0x03, 0x3a, + 0x26, 0xab, 0xa0, 0x39, 0x94, 0xdb, 0x81, 0x3b, 0x44, 0x77, 0x2b, 0xb7, 0x33, 0x0d, 0x91, 0xdb, + 0x50, 0x13, 0xb3, 0x91, 0xaf, 0x86, 0x25, 0x3c, 0x95, 0x97, 0x32, 0x8d, 0x53, 0xcc, 0xfd, 0x60, + 0x3c, 0xa4, 0x46, 0xd5, 0x51, 0x5f, 0x64, 0x13, 0x34, 0xc1, 0x66, 0xaa, 0x87, 0x45, 0xe9, 0xc6, + 0xb2, 0xcf, 0x74, 0xda, 0x36, 0x0c, 0x10, 0x5c, 0xf2, 0x25, 0x91, 0x6c, 0x43, 0x5d, 0x3e, 0xb0, + 0x28, 0x21, 0x95, 0x59, 0x85, 0xc8, 0x77, 0x19, 0x25, 0x65, 0x19, 0xca, 0x96, 0x08, 0x63, 0xdb, + 0xaa, 0xce, 0xa2, 0x5a, 0xe4, 0x7d, 0x28, 0xc9, 0x6a, 0x79, 0x0d, 0x57, 0x76, 0xf9, 0xf8, 0xb2, + 0xaf, 0x74, 0x03, 0x92, 0x9a, 0x7c, 0x0a, 0x75, 0xea, 0x51, 0x2c, 0x9a, 0xa3, 0x5e, 0x60, 0x16, + 0xbd, 0x68, 0x8a, 0x05, 0x55, 0xb3, 0x0d, 0x0d, 0x87, 0x1e, 0x5a, 0x23, 0x2f, 0x34, 0xa5, 0xd1, + 0x6b, 0x27, 0x14, 0x44, 0x12, 0xfb, 0x37, 0xea, 0x8a, 0x0b, 0x21, 0x7c, 0xd3, 0xe5, 0xa6, 0x33, + 0xf6, 0xad, 0x81, 0x6b, 0xab, 0x8b, 0x47, 0xcd, 0xe5, 0xdb, 0x12, 0x20, 0x6b, 0xa0, 0x0b, 0x1b, + 0x88, 0x13, 0x21, 0x61, 0x05, 0x32, 0x37, 0x68, 0xba, 0x3c, 0x4e, 0x72, 0x1e, 0xd0, 0x71, 0xfb, + 0x2f, 0x39, 0xd0, 0xa7, 0x5f, 0x02, 0x63, 0xb3, 0xca, 0xa5, 0xcc, 0x6a, 0xca, 0x60, 0xf2, 0x47, + 0x0d, 0x26, 0x51, 0x75, 0x61, 0x42, 0xd5, 0x1f, 0x40, 0x19, 0xed, 0x35, 0x7a, 0xf9, 0x38, 0xa1, + 0xc4, 0x1e, 0xbd, 0x44, 0x4a, 0x7a, 0x72, 0x13, 0x96, 0x64, 0x91, 0x2c, 0x5a, 0xa9, 0x89, 0x1d, + 0x68, 0x8d, 0x55, 0x83, 0xc8, 0x3e, 0xb5, 0x66, 0xe4, 0x6f, 0x37, 0xa1, 0x8e, 0xaf, 0x46, 0xca, + 0x6d, 0xb7, 0x9f, 0x41, 0x43, 0xb5, 0x55, 0x10, 0x8a, 0xc2, 0x4c, 0xee, 0x3f, 0x0a, 0x33, 0xf9, + 0xe4, 0x9e, 0xff, 0xb3, 0x1c, 0x68, 0x8f, 0x78, 0x6f, 0x8f, 0x71, 0xd4, 0xa5, 0xf0, 0x9f, 0xd1, + 0x9b, 0x5b, 0x4a, 0x77, 0x9a, 0xc2, 0x30, 0x53, 0x5b, 0x82, 0xd2, 0x80, 0xf7, 0x3a, 0xdb, 0x28, + 0xa6, 0x6e, 0xc8, 0x06, 0xe6, 0x6f, 0xbc, 0x77, 0x2f, 0x60, 0xa3, 0x61, 0x54, 0x8e, 0x8a, 0xda, + 0x22, 0xea, 0x24, 0x35, 0xf0, 0x22, 0x7a, 0xe4, 0x04, 0x68, 0xdf, 0x81, 0x79, 0xf5, 0x52, 0x16, + 0xcf, 0x22, 0x6b, 0xe7, 0x44, 0xb4, 0x56, 0xfd, 0x6a, 0x01, 0x71, 0xfb, 0xfa, 0x4f, 0xa1, 0x9e, + 0x5e, 0x2d, 0xd1, 0xa0, 0xb2, 0x3f, 0xb2, 0x6d, 0xca, 0xb9, 0x3e, 0x47, 0xe6, 0x41, 0xdb, 0x65, + 0xa1, 0xb9, 0x3f, 0x1a, 0x0e, 0x59, 0x10, 0xea, 0x39, 0xb2, 0x00, 0x8d, 0x5d, 0x66, 0xee, 0xd1, + 0x60, 0xe0, 0x72, 0xee, 0x32, 0x5f, 0xcf, 0x93, 0x2a, 0x14, 0xef, 0x5a, 0xae, 0xa7, 0x17, 0xc8, + 0x12, 0xcc, 0xe3, 0x99, 0xa3, 0x21, 0x0d, 0xcc, 0x1d, 0x91, 0x1b, 0xe9, 0xbf, 0x2c, 0x90, 0x4b, + 0xd0, 0x52, 0x7b, 0x61, 0x3e, 0xee, 0xfe, 0x88, 0xda, 0xa1, 0x29, 0x44, 0xde, 0x65, 0x23, 0xdf, + 0xd1, 0x7f, 0x55, 0xb8, 0xfe, 0x12, 0x16, 0x33, 0xde, 0x26, 0x08, 0x81, 0xe6, 0xe6, 0x9d, 0xad, + 0x07, 0x4f, 0xf6, 0xcc, 0xce, 0x6e, 0xe7, 0xa0, 0x73, 0xe7, 0xa1, 0x3e, 0x47, 0x96, 0x40, 0x57, + 0xd8, 0xce, 0xb3, 0x9d, 0xad, 0x27, 0x07, 0x9d, 0xdd, 0x7b, 0x7a, 0x2e, 0x45, 0xb9, 0xff, 0x64, + 0x6b, 0x6b, 0x67, 0x7f, 0x5f, 0xcf, 0x8b, 0x79, 0x2b, 0xec, 0xee, 0x9d, 0xce, 0x43, 0xbd, 0x90, + 0x22, 0x3a, 0xe8, 0x3c, 0xda, 0x79, 0xfc, 0xe4, 0x40, 0x2f, 0x5e, 0x7f, 0x1a, 0xdf, 0x59, 0x27, + 0x87, 0xd6, 0xa0, 0x92, 0x8c, 0xd9, 0x80, 0x5a, 0x7a, 0x30, 0xa1, 0x9d, 0x78, 0x14, 0xb1, 0x72, + 0x29, 0x5e, 0x83, 0x4a, 0x22, 0xf7, 0x99, 0x38, 0x4f, 0x53, 0xcf, 0xdb, 0x00, 0xe5, 0xfd, 0x30, + 0x60, 0x7e, 0x4f, 0x9f, 0x43, 0x19, 0x54, 0x6a, 0x0f, 0x05, 0x6e, 0x0a, 0x55, 0x50, 0x47, 0xcf, + 0x93, 0x26, 0xc0, 0xce, 0x0b, 0xea, 0x87, 0x23, 0xcb, 0xf3, 0xc6, 0x7a, 0x41, 0xb4, 0xb7, 0x46, + 0x3c, 0x64, 0x03, 0xf7, 0x15, 0x75, 0xf4, 0xe2, 0xf5, 0x7f, 0xe4, 0xa0, 0x1a, 0xf9, 0x14, 0x31, + 0xfa, 0x2e, 0xf3, 0xa9, 0x3e, 0x27, 0xbe, 0x36, 0x19, 0xf3, 0xf4, 0x9c, 0xf8, 0xea, 0xf8, 0xe1, + 0x07, 0x7a, 0x9e, 0xd4, 0xa0, 0xd4, 0xf1, 0xc3, 0x77, 0x6e, 0xe9, 0x05, 0xf5, 0xf9, 0xee, 0x86, + 0x5e, 0x54, 0x9f, 0xb7, 0xde, 0xd3, 0x4b, 0xe2, 0xf3, 0xae, 0x08, 0x6f, 0x3a, 0x88, 0xc9, 0x6d, + 0x63, 0x1c, 0xd3, 0x35, 0x35, 0x51, 0xd7, 0xef, 0xe9, 0x4b, 0x62, 0x6e, 0x4f, 0xad, 0x60, 0xab, + 0x6f, 0x05, 0xfa, 0x39, 0x41, 0x7f, 0x27, 0x08, 0xac, 0xb1, 0xbe, 0x2c, 0x46, 0xf9, 0x8c, 0x33, + 0x5f, 0x3f, 0x4f, 0x74, 0xa8, 0x6f, 0xba, 0xbe, 0x15, 0x8c, 0x9f, 0x52, 0x3b, 0x64, 0x81, 0xee, + 0x08, 0xcd, 0xa3, 0x58, 0x05, 0x50, 0x61, 0x31, 0x08, 0xbc, 0x73, 0x4b, 0x41, 0x87, 0xb8, 0x19, + 0x93, 0x58, 0x8f, 0x9c, 0x83, 0x85, 0xfd, 0xa1, 0x15, 0x70, 0x9a, 0xe6, 0xee, 0x5f, 0x7f, 0x0a, + 0x90, 0xb8, 0x60, 0x31, 0x1c, 0xb6, 0xe4, 0x7d, 0xc0, 0xd1, 0xe7, 0x50, 0x7a, 0x8c, 0x88, 0x59, + 0xe7, 0x62, 0x68, 0x3b, 0x60, 0xc3, 0xa1, 0x80, 0xf2, 0x31, 0x1f, 0x42, 0xd4, 0xd1, 0x0b, 0x1b, + 0x7f, 0x2a, 0xc1, 0xe2, 0x23, 0x3c, 0xf8, 0xd2, 0xf8, 0xf6, 0x69, 0xf0, 0xc2, 0xb5, 0x29, 0xb1, + 0xa1, 0x9e, 0x7e, 0xa6, 0x20, 0xd9, 0xd7, 0xfa, 0x8c, 0x97, 0x8c, 0x95, 0xb7, 0x4e, 0xab, 0x78, + 0xaa, 0x43, 0xd6, 0x9e, 0x23, 0x3f, 0x80, 0x5a, 0x5c, 0xd2, 0x26, 0xd9, 0x7f, 0x4c, 0x4c, 0x97, + 0xbc, 0xcf, 0x22, 0xbe, 0x0b, 0x5a, 0xaa, 0x0e, 0x4c, 0xb2, 0x39, 0x8f, 0xd6, 0xa1, 0x57, 0xd6, + 0x4e, 0x27, 0x8c, 0xc7, 0xa0, 0x50, 0x4f, 0x97, 0x58, 0x8f, 0xd1, 0x53, 0x46, 0x6d, 0x77, 0xe5, + 0xda, 0x0c, 0x94, 0xf1, 0x30, 0x7d, 0x68, 0x4c, 0x24, 0xea, 0xe4, 0xda, 0xcc, 0xf5, 0xc8, 0x95, + 0xeb, 0xb3, 0x90, 0xc6, 0x23, 0xf5, 0x00, 0x92, 0xbc, 0x9f, 0xbc, 0x7d, 0xdc, 0xa6, 0x64, 0x5c, + 0x0c, 0xce, 0x38, 0xd0, 0x1e, 0x94, 0x30, 0x1e, 0x91, 0xec, 0xc8, 0x93, 0x8e, 0x5d, 0x2b, 0xed, + 0x93, 0x48, 0x22, 0x89, 0x9b, 0x1f, 0x7e, 0xfe, 0x9d, 0x9e, 0x1b, 0xf6, 0x47, 0xdd, 0x75, 0x9b, + 0x0d, 0x6e, 0xbc, 0x72, 0x3d, 0xcf, 0x7d, 0x15, 0x52, 0xbb, 0x7f, 0x43, 0x32, 0x7f, 0x5b, 0xb2, + 0xdd, 0xb0, 0x59, 0xa0, 0xfe, 0x35, 0xbb, 0x21, 0x91, 0x61, 0xb7, 0x5b, 0xc6, 0xf6, 0xbb, 0xff, + 0x0e, 0x00, 0x00, 0xff, 0xff, 0x33, 0x1d, 0x41, 0x89, 0xae, 0x26, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/example/delete_support/prepare_data.py b/example/delete_support/prepare_data.py index cb1e842..6a20685 100644 --- a/example/delete_support/prepare_data.py +++ b/example/delete_support/prepare_data.py @@ -6,7 +6,6 @@ # 5. search, query, and hybrid search on entities # 6. delete entities by PK # 7. drop collection -import time import os import numpy as np from pymilvus import ( @@ -64,15 +63,6 @@ print(fmt.format("Create collection `hello_milvus`")) hello_milvus = Collection("hello_milvus", schema, consistency_level="Strong") -################################################################################ -# 3. insert data -# We are going to insert 3000 rows of data into `hello_milvus` -# Data to be inserted must be organized in fields. -# -# The insert() method returns: -# - either automatically generated primary keys by Milvus if auto_id=True in the schema; -# - or the existing primary key field from the entities if auto_id=False in the schema. - print(fmt.format("Start inserting entities")) rng = np.random.default_rng(seed=19530) entities = [ @@ -84,54 +74,29 @@ ] insert_result = hello_milvus.insert(entities) -hello_milvus.flush() +# hello_milvus.flush() hello_milvus.delete("pk in [0,1,2,3,4]") -hello_milvus.flush() +# both flush and not flush should be tested +# hello_milvus.flush() print(f"Number of entities in hello_milvus: {hello_milvus.num_entities}") # check the num_entites -# create another collection -fields2 = [ - FieldSchema(name="pk", dtype=DataType.INT64, is_primary=True, auto_id=False), - FieldSchema(name="random", dtype=DataType.DOUBLE), - FieldSchema(name="var", dtype=DataType.VARCHAR, max_length=65535), - FieldSchema(name="embeddings", dtype=DataType.FLOAT_VECTOR, dim=dim) -] - -schema2 = CollectionSchema(fields2, "hello_milvus2") - -print(fmt.format("Create collection `hello_milvus2`")) -hello_milvus2 = Collection("hello_milvus2", schema2, consistency_level="Strong") +hello_milvus2 = Collection("hello_milvus2", schema, consistency_level="Strong") +hello_milvus2.create_partition("p1") +hello_milvus2.create_partition("p2") entities2 = [ - [i for i in range(num_entities)], - rng.random(num_entities).tolist(), # field random, only supports list - [str(i) for i in range(num_entities)], - rng.random((num_entities, dim)), # field embeddings, supports numpy.ndarray and list -] - -insert_result2 = hello_milvus2.insert(entities2) -hello_milvus.delete("pk in [0,1,2,3,4]") -hello_milvus2.flush() - -entities3 = [ + # provide the pk field because `auto_id` is set to False [i + num_entities for i in range(num_entities)], rng.random(num_entities).tolist(), # field random, only supports list [str(i) for i in range(num_entities)], rng.random((num_entities, dim)), # field embeddings, supports numpy.ndarray and list ] -insert_result2 = hello_milvus2.insert(entities3) -hello_milvus.delete("pk in [5,6,7,8,9]") -hello_milvus.delete("pk in [3000,3001,3002,3003,3004]") -hello_milvus2.flush() - -index_params = {"index_type": "IVF_FLAT", "params": {"nlist": 128}, "metric_type": "L2"} -hello_milvus.create_index("embeddings", index_params) - -index_params2 = {"index_type": "Trie"} -hello_milvus2.create_index("var", index_params2) - -print(f"Number of entities in hello_milvus2: {hello_milvus2.num_entities}") # check the num_entites +insert_result2 = hello_milvus2.insert(partition_name="p1", data=entities) +insert_result3 = hello_milvus2.insert(partition_name="p2", data=entities2) +hello_milvus2.delete(expr="pk in [0,1,2,3,4]", partition_name="p1") +hello_milvus2.delete(expr="pk in [3001,3002,3003,3004]", partition_name="p2") +hello_milvus2.flush() \ No newline at end of file diff --git a/example/delete_support/verify_data.py b/example/delete_support/verify_data.py new file mode 100644 index 0000000..5d61560 --- /dev/null +++ b/example/delete_support/verify_data.py @@ -0,0 +1,80 @@ +import os +import time +import numpy as np +from pymilvus import ( + connections, + Collection, +) + +fmt = "\n=== {:30} ===\n" +dim = 8 + +print(fmt.format("start connecting to Milvus")) +host = os.environ.get('MILVUS_HOST') +if host == None: + host = "localhost" +print(fmt.format(f"Milvus host: {host}")) +connections.connect("default", host=host, port="19530") + +hello_milvus = Collection("hello_milvus_recover") + +print(fmt.format("Start Creating index for recovered collection")) +index = { + "index_type": "IVF_FLAT", + "metric_type": "L2", + "params": {"nlist": 128}, +} +hello_milvus.create_index("embeddings", index) + +print(fmt.format("Start loading")) +hello_milvus.load() +# ----------------------------------------------------------------------------- +# search based on vector similarity +print(fmt.format("Start searching based on vector similarity")) + +rng = np.random.default_rng(seed=19530) +vectors_to_search = rng.random((1, dim)) +search_params = { + "metric_type": "L2", + "params": {"nprobe": 10}, +} + +start_time = time.time() + +expr = f'pk in [1, 2]' + +print(fmt.format(f"Start query with expr `{expr}`")) +result = hello_milvus.query(expr=expr, output_fields=["pk", "random", "var"]) +print(len(result)) + + +hello_milvus2 = Collection("hello_milvus2_recover") + +print(fmt.format("Start Creating index for recovered collection")) +index = { + "index_type": "IVF_FLAT", + "metric_type": "L2", + "params": {"nlist": 128}, +} +hello_milvus2.create_index("embeddings", index) + +print(fmt.format("Start loading")) +hello_milvus2.load() +# ----------------------------------------------------------------------------- +# search based on vector similarity +print(fmt.format("Start searching based on vector similarity")) + +rng = np.random.default_rng(seed=19530) +vectors_to_search = rng.random((1, dim)) +search_params = { + "metric_type": "L2", + "params": {"nprobe": 10}, +} + +start_time = time.time() + +expr = f'pk in [1, 2,3,4,3001,3002,3003,3004]' + +print(fmt.format(f"Start query with expr `{expr}`")) +result = hello_milvus.query(expr=expr, output_fields=["pk", "random", "var"]) +print(len(result)) \ No newline at end of file diff --git a/go.mod b/go.mod index 1658cf7..be128d9 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,7 @@ require ( github.com/google/uuid v1.3.0 github.com/json-iterator/go v1.1.12 github.com/lingdor/stackerror v0.0.0-20191119040541-976d8885ed76 - github.com/milvus-io/milvus-proto/go-api/v2 v2.4.2 + //github.com/milvus-io/milvus-proto/go-api/v2 v2.4.3 github.com/milvus-io/milvus-sdk-go/v2 v2.4.0 github.com/minio/minio-go/v7 v7.0.61 github.com/pkg/errors v0.9.1 @@ -41,6 +41,8 @@ require ( gopkg.in/yaml.v3 v3.0.1 ) +require github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240430025921-135167be0694 + require ( cloud.google.com/go v0.81.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.3.0 // indirect @@ -116,4 +118,4 @@ require ( gopkg.in/yaml.v2 v2.4.0 // indirect ) -replace github.com/milvus-io/milvus-sdk-go/v2 => github.com/wayblink/milvus-sdk-go/v2 v2.4.2-cdc-rc1 +replace github.com/milvus-io/milvus-sdk-go/v2 => github.com/wayblink/milvus-sdk-go/v2 v2.3.0-beta4.0.20240607085051-1ac5aaf24c6d diff --git a/go.sum b/go.sum index f0cda4b..4554050 100644 --- a/go.sum +++ b/go.sum @@ -400,8 +400,8 @@ github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5 github.com/mediocregopher/radix/v3 v3.4.2/go.mod h1:8FL3F6UQRXHXIBSPUs5h0RybMF8i4n7wVopoX3x7Bv8= github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/milvus-io/milvus-proto/go-api/v2 v2.4.2 h1:jgXBS8x8DTriF2pEI0RH/A+eJ8NI1f51iJcdiYEZOBg= -github.com/milvus-io/milvus-proto/go-api/v2 v2.4.2/go.mod h1:1OIl0v5PQeNxIJhCvY+K55CBUOYDZevw9g9380u1Wek= +github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240430025921-135167be0694 h1:iub0yx8peGNtnb9n11iuWNmhIhIXw3xfZooIDcrfeU8= +github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240430025921-135167be0694/go.mod h1:1OIl0v5PQeNxIJhCvY+K55CBUOYDZevw9g9380u1Wek= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= github.com/minio/minio-go/v7 v7.0.61 h1:87c+x8J3jxQ5VUGimV9oHdpjsAvy3fhneEBKuoKEVUI= @@ -574,8 +574,8 @@ github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBn github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= github.com/valyala/fasttemplate v1.2.1/go.mod h1:KHLXt3tVN2HBp8eijSv/kGJopbvo7S+qRAEEKiv+SiQ= github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= -github.com/wayblink/milvus-sdk-go/v2 v2.4.2-cdc-rc1 h1:VbJAwVv5WsnFu9kDM10pm7iegtL5Q1P+Bj6JMRSToTc= -github.com/wayblink/milvus-sdk-go/v2 v2.4.2-cdc-rc1/go.mod h1:FGj9dlQajwW3N5BDdC12mSRPCN9/6Fi9Ah0Rom0x/7Y= +github.com/wayblink/milvus-sdk-go/v2 v2.3.0-beta4.0.20240607085051-1ac5aaf24c6d h1:mitaOAMaGXTXnYs8+/lWTpmZ73xPhM7qM/OgGaa9qYc= +github.com/wayblink/milvus-sdk-go/v2 v2.3.0-beta4.0.20240607085051-1ac5aaf24c6d/go.mod h1:6ckCQ8h5iFncZcaIpGE8T8sqliwaw2Eu9UzHaCURMA4= github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= diff --git a/internal/common/common.go b/internal/common/common.go deleted file mode 100644 index 828ccb7..0000000 --- a/internal/common/common.go +++ /dev/null @@ -1,59 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package common - -import "encoding/binary" - -// system filed id: -// 0: unique row id -// 1: timestamp -// 100: first user field id -// 101: second user field id -// 102: ... - -const ( - // StartOfUserFieldID represents the starting ID of the user-defined field - StartOfUserFieldID = 100 - - // RowIDField is the ID of the RowID field reserved by the system - RowIDField = 0 - - // TimeStampField is the ID of the Timestamp field reserved by the system - TimeStampField = 1 - - // RowIDFieldName defines the name of the RowID field - RowIDFieldName = "RowID" - - // TimeStampFieldName defines the name of the Timestamp field - TimeStampFieldName = "Timestamp" - - // DefaultShardsNum defines the default number of shards when creating a collection - DefaultShardsNum = int32(2) - - // InvalidPartitionID indicates that the partition is not specified. It will be set when the partitionName is empty - InvalidPartitionID = int64(-1) - - // InvalidFieldID indicates that the field does not exist . It will be set when the field is not found. - InvalidFieldID = int64(-1) - - // NotRegisteredID means node is not registered into etcd. - NotRegisteredID = int64(-1) -) - -// Endian is type alias of binary.LittleEndian. -// Milvus uses little endian by default. -var Endian = binary.LittleEndian diff --git a/internal/kv/kv.go b/internal/kv/kv.go index 2e9c39c..aa209a3 100644 --- a/internal/kv/kv.go +++ b/internal/kv/kv.go @@ -16,11 +16,6 @@ package kv -import ( - "github.com/zilliztech/milvus-backup/internal/util/typeutil" - clientv3 "go.etcd.io/etcd/client/v3" -) - // CompareFailedError is a helper type for checking MetaKv CompareAndSwap series func error type type CompareFailedError struct { internalError error @@ -49,39 +44,3 @@ type BaseKV interface { Close() } - -// TxnKV contains extra txn operations of kv. The extra operations is transactional. -type TxnKV interface { - BaseKV - MultiSaveAndRemove(saves map[string]string, removals []string) error - MultiRemoveWithPrefix(keys []string) error - MultiSaveAndRemoveWithPrefix(saves map[string]string, removals []string) error -} - -// MetaKv is TxnKV for metadata. It should save data with lease. -type MetaKv interface { - TxnKV - GetPath(key string) string - LoadWithPrefix(key string) ([]string, []string, error) - LoadWithPrefix2(key string) ([]string, []string, []int64, error) - LoadWithRevisionAndVersions(key string) ([]string, []string, []int64, int64, error) - LoadWithRevision(key string) ([]string, []string, int64, error) - Watch(key string) clientv3.WatchChan - WatchWithPrefix(key string) clientv3.WatchChan - WatchWithRevision(key string, revision int64) clientv3.WatchChan - SaveWithLease(key, value string, id clientv3.LeaseID) error - SaveWithIgnoreLease(key, value string) error - Grant(ttl int64) (id clientv3.LeaseID, err error) - KeepAlive(id clientv3.LeaseID) (<-chan *clientv3.LeaseKeepAliveResponse, error) - CompareValueAndSwap(key, value, target string, opts ...clientv3.OpOption) (bool, error) - CompareVersionAndSwap(key string, version int64, target string, opts ...clientv3.OpOption) (bool, error) -} - -// SnapShotKV is TxnKV for snapshot data. It must save timestamp. -type SnapShotKV interface { - Save(key string, value string, ts typeutil.Timestamp) error - Load(key string, ts typeutil.Timestamp) (string, error) - MultiSave(kvs map[string]string, ts typeutil.Timestamp) error - LoadWithPrefix(key string, ts typeutil.Timestamp) ([]string, []string, error) - MultiSaveAndRemoveWithPrefix(saves map[string]string, removals []string, ts typeutil.Timestamp) error -} diff --git a/internal/util/typeutil/convension.go b/internal/util/typeutil/convension.go deleted file mode 100644 index 3db8ebd..0000000 --- a/internal/util/typeutil/convension.go +++ /dev/null @@ -1,108 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package typeutil - -import ( - "encoding/binary" - "fmt" - "math" - "reflect" - - "github.com/zilliztech/milvus-backup/internal/common" -) - -// Float32ToBytes converts a float to byte slice. -func Float32ToBytes(float float32) []byte { - bits := math.Float32bits(float) - bytes := make([]byte, 4) - common.Endian.PutUint32(bytes, bits) - - return bytes -} - -// BytesToFloat32 converts a byte slice to float32. -func BytesToFloat32(bytes []byte) float32 { - bits := common.Endian.Uint32(bytes) - - return math.Float32frombits(bits) -} - -// BytesToInt64 converts a byte slice to uint64. -func BytesToInt64(b []byte) (int64, error) { - if len(b) != 8 { - return 0, fmt.Errorf("failed to convert []byte to int64: invalid data, must 8 bytes, but %d", len(b)) - } - - return int64(common.Endian.Uint64(b)), nil -} - -// Int64ToBytes converts uint64 to a byte slice. -func Int64ToBytes(v int64) []byte { - b := make([]byte, 8) - common.Endian.PutUint64(b, uint64(v)) - return b -} - -// BigEndianBytesToUint64 converts a byte slice (big endian) to uint64. -func BigEndianBytesToUint64(b []byte) (uint64, error) { - if len(b) != 8 { - return 0, fmt.Errorf("failed to convert []byte to uint64: invalid data, must 8 bytes, but %d", len(b)) - } - - // do not use little or common endian for compatibility issues(the msgid used in rocksmq is using this) - return binary.BigEndian.Uint64(b), nil -} - -// Uint64ToBytesBigEndian converts uint64 to a byte slice(big endian). -func Uint64ToBytesBigEndian(v uint64) []byte { - b := make([]byte, 8) - binary.BigEndian.PutUint64(b, v) - return b -} - -// BytesToUint64 converts a byte slice to uint64. -func BytesToUint64(b []byte) (uint64, error) { - if len(b) != 8 { - return 0, fmt.Errorf("Failed to convert []byte to uint64: invalid data, must 8 bytes, but %d", len(b)) - } - return common.Endian.Uint64(b), nil -} - -// Uint64ToBytes converts uint64 to a byte slice. -func Uint64ToBytes(v uint64) []byte { - b := make([]byte, 8) - common.Endian.PutUint64(b, v) - return b -} - -// SliceRemoveDuplicate is used to dedup a Slice -func SliceRemoveDuplicate(a interface{}) (ret []interface{}) { - if reflect.TypeOf(a).Kind() != reflect.Slice { - fmt.Printf("input is not slice but %T\n", a) - return ret - } - - va := reflect.ValueOf(a) - for i := 0; i < va.Len(); i++ { - if i > 0 && reflect.DeepEqual(va.Index(i-1).Interface(), va.Index(i).Interface()) { - continue - } - ret = append(ret, va.Index(i).Interface()) - } - - return ret -} diff --git a/internal/util/typeutil/set.go b/internal/util/typeutil/set.go deleted file mode 100644 index 18974dc..0000000 --- a/internal/util/typeutil/set.go +++ /dev/null @@ -1,61 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package typeutil - -// UniqueSet is set type, which contains only UniqueIDs, -// the underlying type is map[UniqueID]struct{}. -// Create a UniqueSet instance with make(UniqueSet) like creating a map instance. -type UniqueSet map[UniqueID]struct{} - -// Insert elements into the set, -// do nothing if the id existed -func (set UniqueSet) Insert(ids ...UniqueID) { - for i := range ids { - set[ids[i]] = struct{}{} - } -} - -// Check whether the elements exist -func (set UniqueSet) Contain(ids ...UniqueID) bool { - for i := range ids { - _, ok := set[ids[i]] - if !ok { - return false - } - } - - return true -} - -// Remove elements from the set, -// do nothing if set is nil or id not exists -func (set UniqueSet) Remove(ids ...UniqueID) { - for i := range ids { - delete(set, ids[i]) - } -} - -// Get all elements in the set -func (set UniqueSet) Collect() []UniqueID { - ids := make([]UniqueID, 0, len(set)) - - for id := range set { - ids = append(ids, id) - } - - return ids -} diff --git a/internal/util/typeutil/string_util.go b/internal/util/typeutil/string_util.go deleted file mode 100644 index 41a120f..0000000 --- a/internal/util/typeutil/string_util.go +++ /dev/null @@ -1,47 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package typeutil - -import "strings" - -// AddOne add one to last byte in string, on empty string return empty -// it helps with key iteration upper bound -func AddOne(data string) string { - if len(data) == 0 { - return data - } - var datab = []byte(data) - if datab[len(datab)-1] != 255 { - datab[len(datab)-1]++ - } else { - datab = append(datab, byte(0)) - } - return string(datab) -} - -// After get substring after sub string. -func After(str string, sub string) string { - pos := strings.LastIndex(str, sub) - if pos == -1 { - return "" - } - adjustedPos := pos + len(sub) - if adjustedPos >= len(str) { - return "" - } - return str[adjustedPos:] -} diff --git a/internal/util/typeutil/time.go b/internal/util/typeutil/time.go deleted file mode 100644 index bad3b97..0000000 --- a/internal/util/typeutil/time.go +++ /dev/null @@ -1,47 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package typeutil - -import ( - "math" - "time" -) - -// MaxTimestamp is the max timestamp. -const MaxTimestamp = Timestamp(math.MaxUint64) - -// ZeroTime is a zero time. -var ZeroTime = time.Time{} - -// ZeroTimestamp is a zero timestamp -var ZeroTimestamp = Timestamp(0) - -// ParseTimestamp returns a timestamp for a given byte slice. -func ParseTimestamp(data []byte) (time.Time, error) { - //we use big endian here for compatibility issues - nano, err := BigEndianBytesToUint64(data) - if err != nil { - return ZeroTime, err - } - - return time.Unix(0, int64(nano)), nil -} - -// SubTimeByWallClock returns the duration between two different timestamps. -func SubTimeByWallClock(after, before time.Time) time.Duration { - return time.Duration(after.UnixNano() - before.UnixNano()) -} diff --git a/internal/util/typeutil/type.go b/internal/util/typeutil/type.go deleted file mode 100644 index 2b6d9a5..0000000 --- a/internal/util/typeutil/type.go +++ /dev/null @@ -1,73 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package typeutil - -// Timestamp is an alias of uint64 -type Timestamp = uint64 - -// IntPrimaryKey is an alias of int64 -type IntPrimaryKey = int64 - -// UniqueID is an alias of int64 -type UniqueID = int64 - -const ( - // EmbeddedRole is for embedded Milvus. - EmbeddedRole = "embedded" - // StandaloneRole is a constant represent Standalone - StandaloneRole = "standalone" - // RootCoordRole is a constant represent RootCoord - RootCoordRole = "rootcoord" - // ProxyRole is a constant represent Proxy - ProxyRole = "proxy" - // QueryCoordRole is a constant represent QueryCoord - QueryCoordRole = "querycoord" - // QueryNodeRole is a constant represent QueryNode - QueryNodeRole = "querynode" - // IndexCoordRole is a constant represent IndexCoord - IndexCoordRole = "indexcoord" - // IndexNodeRole is a constant represent IndexNode - IndexNodeRole = "indexnode" - // DataCoordRole is a constant represent DataCoord - DataCoordRole = "datacoord" - // DataNodeRole is a constant represent DataNode - DataNodeRole = "datanode" -) - -func ServerTypeMap() map[string]interface{} { - return map[string]interface{}{ - EmbeddedRole: nil, - StandaloneRole: nil, - RootCoordRole: nil, - ProxyRole: nil, - QueryCoordRole: nil, - QueryNodeRole: nil, - IndexCoordRole: nil, - IndexNodeRole: nil, - DataCoordRole: nil, - DataNodeRole: nil, - } -} - -func ServerTypeList() []string { - serverTypeMap := ServerTypeMap() - types := make([]string, 0, len(serverTypeMap)) - for key := range serverTypeMap { - types = append(types, key) - } - return types -}