From fc3309b8b86eadae7b616b0c78f0461b269ac6c3 Mon Sep 17 00:00:00 2001 From: "cai.zhang" Date: Wed, 21 Sep 2022 16:34:51 +0800 Subject: [PATCH] All indexes segments both write handoff (#19230) Signed-off-by: cai.zhang Signed-off-by: cai.zhang --- internal/indexcoord/flush_segment_watcher.go | 158 +----- .../flushed_segment_watcher_test.go | 198 +++++-- internal/indexcoord/handoff.go | 282 ++++++++++ internal/indexcoord/handoff_test.go | 429 +++++++++++++++ internal/indexcoord/index_coord.go | 32 +- internal/indexcoord/index_coord_test.go | 80 +++ internal/indexcoord/meta_table.go | 15 + internal/indexcoord/task.go | 5 + internal/indexcoord/util.go | 6 + internal/metastore/model/segment_index.go | 4 + internal/proto/data_coord.proto | 4 + internal/proto/datapb/data_coord.pb.go | 512 +++++++++--------- internal/proto/index_coord.proto | 1 + internal/proto/indexpb/index_coord.pb.go | 263 ++++----- .../observers/handoff_observer.go | 1 + 15 files changed, 1434 insertions(+), 556 deletions(-) create mode 100644 internal/indexcoord/handoff.go create mode 100644 internal/indexcoord/handoff_test.go diff --git a/internal/indexcoord/flush_segment_watcher.go b/internal/indexcoord/flush_segment_watcher.go index bdf0036eb3..c98dd2dfa9 100644 --- a/internal/indexcoord/flush_segment_watcher.go +++ b/internal/indexcoord/flush_segment_watcher.go @@ -25,16 +25,13 @@ import ( "sync" "time" - "github.com/golang/protobuf/proto" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/zap" - "github.com/milvus-io/milvus/api/commonpb" "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/util" "github.com/milvus-io/milvus/internal/util/logutil" ) @@ -56,6 +53,7 @@ type flushedSegmentWatcher struct { meta *metaTable builder *indexBuilder ic *IndexCoord + handoff *handoff internalTasks map[UniqueID]*internalTask } @@ -65,7 +63,8 @@ type internalTask struct { segmentInfo *datapb.SegmentInfo } -func newFlushSegmentWatcher(ctx context.Context, kv kv.MetaKv, meta *metaTable, builder *indexBuilder, ic *IndexCoord) (*flushedSegmentWatcher, error) { +func newFlushSegmentWatcher(ctx context.Context, kv kv.MetaKv, meta *metaTable, builder *indexBuilder, + handoff *handoff, ic *IndexCoord) (*flushedSegmentWatcher, error) { ctx, cancel := context.WithCancel(ctx) fsw := &flushedSegmentWatcher{ ctx: ctx, @@ -77,6 +76,7 @@ func newFlushSegmentWatcher(ctx context.Context, kv kv.MetaKv, meta *metaTable, internalNotify: make(chan struct{}, 1), meta: meta, builder: builder, + handoff: handoff, ic: ic, } err := fsw.reloadFromKV() @@ -128,9 +128,8 @@ func (fsw *flushedSegmentWatcher) enqueueInternalTask(segmentID UniqueID) { state: indexTaskPrepare, segmentInfo: nil, } - return } - logutil.Logger(fsw.ctx).Info("flushedSegmentWatcher already have the task", zap.Int64("segmentID", segmentID)) + logutil.Logger(fsw.ctx).Info("flushedSegmentWatcher enqueueInternalTask success", zap.Int64("segmentID", segmentID)) } func (fsw *flushedSegmentWatcher) internalScheduler() { @@ -179,6 +178,13 @@ func (fsw *flushedSegmentWatcher) internalNotifyFunc() { } } +func (fsw *flushedSegmentWatcher) Len() int { + fsw.internalTaskMutex.RLock() + defer fsw.internalTaskMutex.RUnlock() + + return len(fsw.internalTasks) +} + func (fsw *flushedSegmentWatcher) updateInternalTaskState(segID UniqueID, state indexTaskState) { fsw.internalTaskMutex.Lock() defer fsw.internalTaskMutex.Unlock() @@ -216,19 +222,6 @@ func (fsw *flushedSegmentWatcher) setInternalTaskSegmentInfo(segID UniqueID, seg log.Debug("flushedSegmentWatcher set internal task segment info success", zap.Int64("segID", segID)) } -func (fsw *flushedSegmentWatcher) allParentsDone(segIDs []UniqueID) bool { - fsw.internalTaskMutex.RLock() - defer fsw.internalTaskMutex.RUnlock() - done := true - for _, segID := range segIDs { - if _, ok := fsw.internalTasks[segID]; ok { - done = false - break - } - } - return done -} - func (fsw *flushedSegmentWatcher) internalProcess(segID UniqueID) { t := fsw.getInternalTask(segID) log.Debug("IndexCoord flushedSegmentWatcher process internal task", zap.Int64("segID", segID), @@ -249,52 +242,11 @@ func (fsw *flushedSegmentWatcher) internalProcess(segID UniqueID) { fsw.updateInternalTaskState(segID, indexTaskInProgress) fsw.internalNotifyFunc() case indexTaskInProgress: - state := fsw.meta.GetSegmentIndexState(segID) - if state.state == commonpb.IndexState_Finished || state.state == commonpb.IndexState_Failed || state.state == commonpb.IndexState_IndexStateNone { - log.Debug("all tasks are finished", zap.Int64("segID", segID), zap.String("state", state.state.String())) + if fsw.handoff.taskDone(segID) { fsw.updateInternalTaskState(segID, indexTaskDone) fsw.internalNotifyFunc() } case indexTaskDone: - if !fsw.allParentsDone(t.segmentInfo.CompactionFrom) { - log.Debug("flushed segment create index done, but there are still parent task that haven't written handoff event", - zap.Int64("segID", segID), zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom)) - return - } - indexInfos := fsw.meta.GetSegmentIndexes(segID) - enableIndex := len(indexInfos) > 0 - handoffTask := &querypb.SegmentInfo{ - SegmentID: segID, - CollectionID: t.segmentInfo.CollectionID, - PartitionID: t.segmentInfo.PartitionID, - NumRows: t.segmentInfo.NumOfRows, - DmChannel: t.segmentInfo.GetInsertChannel(), - IndexName: "", - IndexID: 0, - CompactionFrom: t.segmentInfo.CompactionFrom, - CreatedByCompaction: t.segmentInfo.CreatedByCompaction, - SegmentState: t.segmentInfo.State, - IndexInfos: make([]*querypb.FieldIndexInfo, 0), - EnableIndex: enableIndex, - } - for _, indexInfo := range indexInfos { - handoffTask.IndexInfos = append(handoffTask.IndexInfos, &querypb.FieldIndexInfo{ - FieldID: fsw.meta.GetFieldIDByIndexID(t.segmentInfo.CollectionID, indexInfo.IndexID), - EnableIndex: true, - IndexName: fsw.meta.GetIndexNameByID(t.segmentInfo.CollectionID, indexInfo.IndexID), - IndexID: indexInfo.IndexID, - BuildID: indexInfo.BuildID, - IndexParams: fsw.meta.GetIndexParams(t.segmentInfo.CollectionID, indexInfo.IndexID), - //IndexFilePaths: nil, - //IndexSize: 0, - }) - } - - if err := fsw.writeHandoffSegment(handoffTask); err != nil { - log.Error("IndexCoord flushSegmentWatcher writeHandoffSegment with no index info fail", - zap.Int64("segID", segID), zap.Error(err)) - return - } if err := fsw.removeFlushedSegment(t); err != nil { log.Error("IndexCoord flushSegmentWatcher removeFlushedSegment fail", zap.Int64("segID", segID), zap.Error(err)) @@ -341,31 +293,12 @@ func (fsw *flushedSegmentWatcher) constructTask(t *internalTask) error { fsw.builder.enqueue(buildID) } } + fsw.handoff.enqueue(t.segmentInfo.ID) log.Debug("flushedSegmentWatcher construct children task success", zap.Int64("segID", t.segmentInfo.ID), zap.Int("tasks num", len(fieldIndexes))) return nil } -func (fsw *flushedSegmentWatcher) writeHandoffSegment(t *querypb.SegmentInfo) error { - key := fmt.Sprintf("%s/%d/%d/%d", util.HandoffSegmentPrefix, t.CollectionID, t.PartitionID, t.SegmentID) - value, err := proto.Marshal(t) - if err != nil { - log.Error("IndexCoord marshal handoff task fail", zap.Int64("collID", t.CollectionID), - zap.Int64("partID", t.PartitionID), zap.Int64("segID", t.SegmentID), zap.Error(err)) - return err - } - err = fsw.kvClient.Save(key, string(value)) - if err != nil { - log.Error("IndexCoord save handoff task fail", zap.Int64("collID", t.CollectionID), - zap.Int64("partID", t.PartitionID), zap.Int64("segID", t.SegmentID), zap.Error(err)) - return err - } - - log.Info("IndexCoord write handoff task success", zap.Int64("collID", t.CollectionID), - zap.Int64("partID", t.PartitionID), zap.Int64("segID", t.SegmentID)) - return nil -} - func (fsw *flushedSegmentWatcher) removeFlushedSegment(t *internalTask) error { deletedKeys := fmt.Sprintf("%s/%d/%d/%d", util.FlushedSegmentPrefix, t.segmentInfo.CollectionID, t.segmentInfo.PartitionID, t.segmentInfo.ID) err := fsw.kvClient.RemoveWithPrefix(deletedKeys) @@ -379,42 +312,15 @@ func (fsw *flushedSegmentWatcher) removeFlushedSegment(t *internalTask) error { return nil } -func (fsw *flushedSegmentWatcher) pullSegmentInfo(segmentID UniqueID) error { - t := fsw.getInternalTask(segmentID) - if t.segmentInfo != nil { - return nil - } - resp, err := fsw.ic.dataCoordClient.GetSegmentInfo(fsw.ctx, &datapb.GetSegmentInfoRequest{ - SegmentIDs: []int64{segmentID}, - IncludeUnHealthy: true, - }) - if err != nil { - log.Error("flushedSegmentWatcher get segment info fail", zap.Int64("segID", segmentID), zap.Error(err)) - return err - } - if resp.Status.GetErrorCode() != commonpb.ErrorCode_Success { - log.Error("flushedSegmentWatcher get segment info fail", zap.Int64("segID", segmentID), - zap.String("fail reason", resp.Status.GetReason())) - if resp.Status.GetReason() == msgSegmentNotFound(segmentID) { - return errSegmentNotFound(segmentID) - } - return errors.New(resp.Status.GetReason()) - } - for _, info := range resp.Infos { - if info.ID == segmentID { - fsw.setInternalTaskSegmentInfo(segmentID, info) - return nil - } - } - errMsg := fmt.Sprintf("flushedSegmentWatcher get segment info fail, the segment is not include in the response with ID: %d", segmentID) - log.Error(errMsg) - return errors.New(errMsg) -} - func (fsw *flushedSegmentWatcher) prepare(segID UniqueID) error { defer fsw.internalNotifyFunc() log.Debug("prepare flushed segment task", zap.Int64("segID", segID)) - if err := fsw.pullSegmentInfo(segID); err != nil { + t := fsw.getInternalTask(segID) + if t.segmentInfo != nil { + return nil + } + info, err := fsw.ic.pullSegmentInfo(fsw.ctx, segID) + if err != nil { log.Error("flushedSegmentWatcher get segment info fail", zap.Int64("segID", segID), zap.Error(err)) if errors.Is(err, ErrSegmentNotFound) { @@ -423,28 +329,6 @@ func (fsw *flushedSegmentWatcher) prepare(segID UniqueID) error { } return err } - //t := fsw.getInternalTask(segID) - //if t.segmentInfo.CreatedByCompaction { - // if err := fsw.removeCompactedTasks(t); err != nil { - // return err - // } - //} + fsw.setInternalTaskSegmentInfo(segID, info) return nil } - -//func (fsw *flushedSegmentWatcher) removeCompactedTasks(t *internalTask) error { -// log.Debug("IndexCoord flushedSegmentWatcher mark task as deleted which is compacted", zap.Int64("segID", t.segmentInfo.ID), -// zap.Int64s("compactionFrom", t.segmentInfo.CompactionFrom)) -// if err := fsw.builder.markTasksAsDeleted(fsw.meta.GetBuildIDsFromSegIDs(t.segmentInfo.CompactionFrom)); err != nil { -// log.Error("mark index meta fail, try again", zap.Int64s("compacted segIDs", t.segmentInfo.CompactionFrom), -// zap.Error(err)) -// return err -// } -// for _, segID := range t.segmentInfo.CompactionFrom { -// fsw.deleteChildrenTask(segID) -// if _, ok := fsw.internalTasks[segID]; ok { -// fsw.updateInternalTaskState(segID, indexTaskDeleted) -// } -// } -// return nil -//} diff --git a/internal/indexcoord/flushed_segment_watcher_test.go b/internal/indexcoord/flushed_segment_watcher_test.go index 2a019c0abd..70e17dca7f 100644 --- a/internal/indexcoord/flushed_segment_watcher_test.go +++ b/internal/indexcoord/flushed_segment_watcher_test.go @@ -21,8 +21,7 @@ import ( "errors" "sync" "testing" - - "github.com/milvus-io/milvus/internal/proto/indexpb" + "time" "github.com/stretchr/testify/assert" @@ -36,22 +35,43 @@ import ( func Test_flushSegmentWatcher(t *testing.T) { ctx := context.Background() - watcher, err := newFlushSegmentWatcher(ctx, + fsw, err := newFlushSegmentWatcher(ctx, &mockETCDKV{ loadWithRevision: func(key string) ([]string, []string, int64, error) { - return []string{"seg1"}, []string{"12345"}, 1, nil + return []string{"1", "2", "3"}, []string{"1", "2", "3"}, 1, nil + }, + removeWithPrefix: func(key string) error { + return nil }, }, &metaTable{ catalog: &indexcoord.Catalog{ Txn: NewMockEtcdKV(), }, + indexLock: sync.RWMutex{}, + segmentIndexLock: sync.RWMutex{}, + collectionIndexes: map[UniqueID]map[UniqueID]*model.Index{}, + segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{}, + buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{}, }, - &indexBuilder{}, &IndexCoord{ + &indexBuilder{}, &handoff{}, &IndexCoord{ dataCoordClient: NewDataCoordMock(), }) assert.NoError(t, err) - assert.NotNil(t, watcher) + assert.NotNil(t, fsw) + + fsw.enqueueInternalTask(1) + + fsw.Start() + + // hold ticker.C + time.Sleep(time.Second * 2) + + for fsw.Len() != 0 { + time.Sleep(time.Second) + } + + fsw.Stop() } func Test_flushSegmentWatcher_newFlushSegmentWatcher(t *testing.T) { @@ -61,7 +81,7 @@ func Test_flushSegmentWatcher_newFlushSegmentWatcher(t *testing.T) { loadWithRevision: func(key string) ([]string, []string, int64, error) { return []string{"segID1"}, []string{"12345"}, 1, nil }, - }, &metaTable{}, &indexBuilder{}, &IndexCoord{ + }, &metaTable{}, &indexBuilder{}, &handoff{}, &IndexCoord{ dataCoordClient: NewDataCoordMock(), }) assert.NoError(t, err) @@ -74,7 +94,7 @@ func Test_flushSegmentWatcher_newFlushSegmentWatcher(t *testing.T) { loadWithRevision: func(key string) ([]string, []string, int64, error) { return []string{"segID1"}, []string{"12345"}, 1, errors.New("error") }, - }, &metaTable{}, &indexBuilder{}, &IndexCoord{ + }, &metaTable{}, &indexBuilder{}, &handoff{}, &IndexCoord{ dataCoordClient: NewDataCoordMock(), }) assert.Error(t, err) @@ -87,7 +107,7 @@ func Test_flushSegmentWatcher_newFlushSegmentWatcher(t *testing.T) { loadWithRevision: func(key string) ([]string, []string, int64, error) { return []string{"segID1"}, []string{"segID"}, 1, nil }, - }, &metaTable{}, &indexBuilder{}, &IndexCoord{ + }, &metaTable{}, &indexBuilder{}, &handoff{}, &IndexCoord{ dataCoordClient: NewDataCoordMock(), }) assert.Error(t, err) @@ -95,6 +115,48 @@ func Test_flushSegmentWatcher_newFlushSegmentWatcher(t *testing.T) { }) } +func Test_flushedSegmentWatcher_internalRun(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + fsw := &flushedSegmentWatcher{ + ctx: ctx, + cancel: cancel, + kvClient: NewMockEtcdKV(), + wg: sync.WaitGroup{}, + scheduleDuration: time.Second, + internalTaskMutex: sync.RWMutex{}, + internalNotify: make(chan struct{}, 1), + etcdRevision: 0, + watchChan: nil, + meta: nil, + builder: nil, + ic: &IndexCoord{ + dataCoordClient: NewDataCoordMock(), + }, + handoff: nil, + internalTasks: map[UniqueID]*internalTask{ + segID: { + state: indexTaskPrepare, + segmentInfo: &datapb.SegmentInfo{ + CollectionID: collID, + PartitionID: partID, + ID: segID, + }, + }, + segID + 1: { + state: indexTaskPrepare, + segmentInfo: nil, + }, + segID - 1: { + state: indexTaskPrepare, + segmentInfo: nil, + }, + }, + } + + fsw.internalRun() + assert.Equal(t, 3, fsw.Len()) +} + func Test_flushSegmentWatcher_internalProcess_success(t *testing.T) { meta := &metaTable{ segmentIndexLock: sync.RWMutex{}, @@ -150,6 +212,15 @@ func Test_flushSegmentWatcher_internalProcess_success(t *testing.T) { } fsw := &flushedSegmentWatcher{ + handoff: &handoff{ + tasks: map[UniqueID]struct{}{}, + taskMutex: sync.RWMutex{}, + wg: sync.WaitGroup{}, + meta: meta, + notifyChan: make(chan struct{}, 1), + scheduleDuration: time.Second, + kvClient: nil, + }, ic: &IndexCoord{ dataCoordClient: &DataCoordMock{ CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { @@ -217,14 +288,7 @@ func Test_flushSegmentWatcher_internalProcess_success(t *testing.T) { fsw.internalTaskMutex.RUnlock() }) - err := fsw.meta.FinishTask(&indexpb.IndexTaskInfo{ - BuildID: buildID, - State: commonpb.IndexState_Finished, - IndexFiles: []string{"file1", "file2"}, - SerializedSize: 100, - FailReason: "", - }) - assert.NoError(t, err) + fsw.handoff.deleteTask(segID) t.Run("inProgress", func(t *testing.T) { fsw.internalProcess(segID) @@ -304,9 +368,9 @@ func Test_flushSegmentWatcher_internalProcess_error(t *testing.T) { fsw.internalTaskMutex.RUnlock() }) - t.Run("write handoff event fail", func(t *testing.T) { + t.Run("remove flushed segment fail", func(t *testing.T) { fsw.kvClient = &mockETCDKV{ - save: func(s string, s2 string) error { + removeWithPrefix: func(key string) error { return errors.New("error") }, } @@ -339,28 +403,6 @@ func Test_flushSegmentWatcher_internalProcess_error(t *testing.T) { fsw.internalTaskMutex.RUnlock() }) - t.Run("remove flushed segment fail", func(t *testing.T) { - fsw.kvClient = &mockETCDKV{ - save: func(s string, s2 string) error { - return nil - }, - removeWithPrefix: func(key string) error { - return errors.New("error") - }, - } - fsw.internalProcess(segID) - fsw.internalTaskMutex.RLock() - assert.Equal(t, indexTaskDone, fsw.internalTasks[segID].state) - fsw.internalTaskMutex.RUnlock() - }) - - t.Run("index is not zero", func(t *testing.T) { - fsw.internalProcess(segID) - fsw.internalTaskMutex.RLock() - assert.Equal(t, indexTaskDone, fsw.internalTasks[segID].state) - fsw.internalTaskMutex.RUnlock() - }) - t.Run("invalid state", func(t *testing.T) { fsw.internalTasks = map[UniqueID]*internalTask{ segID: { @@ -371,3 +413,77 @@ func Test_flushSegmentWatcher_internalProcess_error(t *testing.T) { fsw.internalProcess(segID) }) } + +func Test_flushSegmentWatcher_prepare_error(t *testing.T) { + t.Run("segmentInfo already exist", func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + fsw := &flushedSegmentWatcher{ + ctx: ctx, + cancel: cancel, + kvClient: NewMockEtcdKV(), + wg: sync.WaitGroup{}, + scheduleDuration: time.Second, + internalTaskMutex: sync.RWMutex{}, + internalNotify: make(chan struct{}, 1), + etcdRevision: 0, + watchChan: nil, + meta: nil, + builder: nil, + ic: &IndexCoord{ + dataCoordClient: NewDataCoordMock(), + }, + handoff: nil, + internalTasks: map[UniqueID]*internalTask{ + segID: { + state: indexTaskPrepare, + segmentInfo: &datapb.SegmentInfo{ + CollectionID: collID, + PartitionID: partID, + ID: segID, + }, + }, + }, + } + err := fsw.prepare(segID) + assert.NoError(t, err) + }) + + t.Run("segment is not exist", func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + fsw := &flushedSegmentWatcher{ + ctx: ctx, + cancel: cancel, + kvClient: NewMockEtcdKV(), + wg: sync.WaitGroup{}, + scheduleDuration: time.Second, + internalTaskMutex: sync.RWMutex{}, + internalNotify: make(chan struct{}, 1), + etcdRevision: 0, + watchChan: nil, + meta: nil, + builder: nil, + ic: &IndexCoord{ + dataCoordClient: &DataCoordMock{ + CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + return &datapb.GetSegmentInfoResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Infos: nil, + }, nil + }, + }, + }, + handoff: nil, + internalTasks: map[UniqueID]*internalTask{ + segID: { + state: indexTaskPrepare, + segmentInfo: nil, + }, + }, + } + + err := fsw.prepare(segID) + assert.ErrorIs(t, err, ErrSegmentNotFound) + }) +} diff --git a/internal/indexcoord/handoff.go b/internal/indexcoord/handoff.go new file mode 100644 index 0000000000..723b3e0a99 --- /dev/null +++ b/internal/indexcoord/handoff.go @@ -0,0 +1,282 @@ +// 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 indexcoord + +import ( + "context" + "errors" + "sort" + "sync" + "time" + + "github.com/golang/protobuf/proto" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/api/commonpb" + "github.com/milvus-io/milvus/internal/kv" + "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/proto/querypb" +) + +type handoff struct { + ctx context.Context + cancel context.CancelFunc + + tasks map[UniqueID]struct{} + taskMutex sync.RWMutex + wg sync.WaitGroup + + meta *metaTable + + notifyChan chan struct{} + scheduleDuration time.Duration + + kvClient kv.MetaKv + ic *IndexCoord +} + +func newHandoff(ctx context.Context, metaTable *metaTable, kvClient kv.MetaKv, ic *IndexCoord) *handoff { + ctx, cancel := context.WithCancel(ctx) + hd := &handoff{ + ctx: ctx, + cancel: cancel, + tasks: make(map[UniqueID]struct{}), + taskMutex: sync.RWMutex{}, + wg: sync.WaitGroup{}, + meta: metaTable, + notifyChan: make(chan struct{}, 1), + scheduleDuration: time.Second, + kvClient: kvClient, + ic: ic, + } + hd.recoveryFromMeta() + log.Ctx(ctx).Info("new handoff success") + return hd +} + +func (hd *handoff) recoveryFromMeta() { + allSegIndexes := hd.meta.GetAllSegIndexes() + hd.taskMutex.Lock() + defer hd.taskMutex.Unlock() + + hd.tasks = make(map[UniqueID]struct{}, 0) + for segID, segIdx := range allSegIndexes { + if segIdx.IsDeleted { + continue + } + if segIdx.WriteHandoff { + continue + } + hd.tasks[segID] = struct{}{} + } + log.Ctx(hd.ctx).Info("recovery from meta success", zap.Int("task num", len(hd.tasks))) +} + +func (hd *handoff) enqueue(segID UniqueID) { + defer hd.Notify() + hd.taskMutex.Lock() + defer hd.taskMutex.Unlock() + + // note: don't reset state if the task contains state + hd.tasks[segID] = struct{}{} + log.Ctx(hd.ctx).Info("segment need to write handoff", zap.Int64("segID", segID)) +} + +func (hd *handoff) Start() { + hd.wg.Add(1) + go hd.scheduler() +} + +func (hd *handoff) Stop() { + hd.cancel() + hd.wg.Wait() +} + +func (hd *handoff) Notify() { + select { + case hd.notifyChan <- struct{}{}: + default: + } +} + +func (hd *handoff) scheduler() { + log.Ctx(hd.ctx).Info("IndexCoord handoff start...") + defer hd.wg.Done() + + ticker := time.NewTicker(hd.scheduleDuration) + defer ticker.Stop() + for { + select { + case <-hd.ctx.Done(): + log.Info("IndexCoord handoff context done, exit...") + return + case <-ticker.C: + hd.run() + case <-hd.notifyChan: + hd.run() + } + } +} + +func (hd *handoff) run() { + hd.taskMutex.RLock() + segIDs := make([]UniqueID, 0, len(hd.tasks)) + for segID := range hd.tasks { + segIDs = append(segIDs, segID) + } + hd.taskMutex.RUnlock() + + sort.Slice(segIDs, func(i, j int) bool { + return segIDs[i] < segIDs[j] + }) + if len(segIDs) > 0 { + log.Ctx(hd.ctx).Debug("handoff process...", zap.Int("task num", len(segIDs))) + } + for i, segID := range segIDs { + hd.process(segID, i == 0) + } +} + +func (hd *handoff) process(segID UniqueID, front bool) { + state := hd.meta.GetSegmentIndexState(segID) + log.Ctx(hd.ctx).RatedDebug(30, "handoff task is process", zap.Int64("segID", segID), + zap.Bool("front", front), zap.String("state", state.state.String())) + if state.state == commonpb.IndexState_Failed { + log.Ctx(hd.ctx).Error("build index failed, may be need manual intervention", zap.Int64("segID", segID), + zap.String("fail reason", state.failReason)) + hd.deleteTask(segID) + // TODO @xiaocai2333: need write handoff event? + return + } + if state.state == commonpb.IndexState_Finished { + log.Ctx(hd.ctx).Debug("build index for segment success, write handoff event...", zap.Int64("segID", segID)) + info, err := hd.ic.pullSegmentInfo(hd.ctx, segID) + if err != nil { + if errors.Is(err, ErrSegmentNotFound) { + log.Ctx(hd.ctx).Error("handoff get segment fail", zap.Error(err)) + hd.deleteTask(segID) + return + } + log.Ctx(hd.ctx).Warn("handoff get segment fail, need to retry", zap.Error(err)) + return + } + if info.IsImporting { + log.Debug("segment is importing, can't write handoff event", zap.Int64("segID", segID)) + return + } + if front || hd.allParentsDone(info.CompactionFrom) { + log.Ctx(hd.ctx).Debug("segment can write handoff event", zap.Int64("segID", segID), zap.Bool("front", front), + zap.Int64s("compactionFrom", info.CompactionFrom)) + indexInfos := hd.meta.GetSegmentIndexes(segID) + if len(indexInfos) == 0 { + log.Ctx(hd.ctx).Warn("ready to write handoff, but there is no index, may be dropped", zap.Int64("segID", segID)) + hd.deleteTask(segID) + return + } + handoffTask := &querypb.SegmentInfo{ + SegmentID: segID, + CollectionID: info.CollectionID, + PartitionID: info.PartitionID, + NumRows: info.NumOfRows, + DmChannel: info.GetInsertChannel(), + CompactionFrom: info.CompactionFrom, + CreatedByCompaction: info.CreatedByCompaction, + SegmentState: info.State, + IndexInfos: make([]*querypb.FieldIndexInfo, 0), + EnableIndex: true, + } + for _, indexInfo := range indexInfos { + handoffTask.IndexInfos = append(handoffTask.IndexInfos, &querypb.FieldIndexInfo{ + FieldID: hd.meta.GetFieldIDByIndexID(info.CollectionID, indexInfo.IndexID), + EnableIndex: true, + IndexName: hd.meta.GetIndexNameByID(info.CollectionID, indexInfo.IndexID), + IndexID: indexInfo.IndexID, + BuildID: indexInfo.BuildID, + IndexParams: hd.meta.GetIndexParams(info.CollectionID, indexInfo.IndexID), + //IndexFilePaths: nil, + //IndexSize: 0, + }) + } + if err := hd.writeHandoffSegment(handoffTask); err != nil { + log.Ctx(hd.ctx).Warn("write handoff task fail, need to retry", zap.Int64("segID", segID), zap.Error(err)) + return + } + log.Ctx(hd.ctx).Info("write handoff task success", zap.Int64("segID", segID)) + if err := hd.meta.MarkSegmentWriteHandoff(segID); err != nil { + log.Ctx(hd.ctx).Warn("mark segment as write handoff fail, need to retry", zap.Int64("segID", segID), zap.Error(err)) + return + } + log.Ctx(hd.ctx).Info("mark segment as write handoff success, remove task", zap.Int64("segID", segID)) + hd.deleteTask(segID) + return + } + } +} + +func (hd *handoff) Len() int { + hd.taskMutex.RLock() + defer hd.taskMutex.RUnlock() + + return len(hd.tasks) +} + +func (hd *handoff) deleteTask(segID UniqueID) { + hd.taskMutex.Lock() + defer hd.taskMutex.Unlock() + + delete(hd.tasks, segID) +} + +func (hd *handoff) taskDone(segID UniqueID) bool { + hd.taskMutex.RLock() + defer hd.taskMutex.RUnlock() + + _, ok := hd.tasks[segID] + return !ok +} + +func (hd *handoff) allParentsDone(segIDs []UniqueID) bool { + hd.taskMutex.RLock() + defer hd.taskMutex.RUnlock() + + for _, segID := range segIDs { + if _, ok := hd.tasks[segID]; ok { + return false + } + } + return true +} + +func (hd *handoff) writeHandoffSegment(info *querypb.SegmentInfo) error { + key := buildHandoffKey(info.CollectionID, info.PartitionID, info.SegmentID) + value, err := proto.Marshal(info) + if err != nil { + log.Error("IndexCoord marshal handoff task fail", zap.Int64("collID", info.CollectionID), + zap.Int64("partID", info.PartitionID), zap.Int64("segID", info.SegmentID), zap.Error(err)) + return err + } + err = hd.kvClient.Save(key, string(value)) + if err != nil { + log.Error("IndexCoord save handoff task fail", zap.Int64("collID", info.CollectionID), + zap.Int64("partID", info.PartitionID), zap.Int64("segID", info.SegmentID), zap.Error(err)) + return err + } + + log.Info("IndexCoord write handoff task success", zap.Int64("collID", info.CollectionID), + zap.Int64("partID", info.PartitionID), zap.Int64("segID", info.SegmentID)) + return nil +} diff --git a/internal/indexcoord/handoff_test.go b/internal/indexcoord/handoff_test.go new file mode 100644 index 0000000000..0c8d43a0a9 --- /dev/null +++ b/internal/indexcoord/handoff_test.go @@ -0,0 +1,429 @@ +// 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 indexcoord + +import ( + "context" + "errors" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus/api/commonpb" + "github.com/milvus-io/milvus/internal/metastore" + "github.com/milvus-io/milvus/internal/metastore/kv/indexcoord" + "github.com/milvus-io/milvus/internal/metastore/model" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/indexpb" +) + +func createMetaForHandoff(catalog metastore.IndexCoordCatalog) *metaTable { + return &metaTable{ + catalog: catalog, + segmentIndexLock: sync.RWMutex{}, + indexLock: sync.RWMutex{}, + collectionIndexes: map[UniqueID]map[UniqueID]*model.Index{ + collID: { + indexID: { + TenantID: "", + CollectionID: collID, + FieldID: fieldID, + IndexID: indexID, + IndexName: indexName, + IsDeleted: false, + CreateTime: 0, + TypeParams: nil, + IndexParams: nil, + }, + }, + }, + segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ + segID: { + indexID: { + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + BuildID: buildID, + IndexState: 1, + IsDeleted: false, + WriteHandoff: false, + }, + }, + segID + 1: { + indexID: { + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + BuildID: buildID + 1, + IndexState: 1, + IsDeleted: true, + WriteHandoff: false, + }, + }, + segID + 2: { + indexID: { + SegmentID: segID + 2, + CollectionID: collID, + PartitionID: partID, + BuildID: buildID + 2, + IndexState: 1, + IsDeleted: false, + WriteHandoff: true, + }, + }, + }, + buildID2SegmentIndex: map[UniqueID]*model.SegmentIndex{ + buildID: { + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + BuildID: buildID, + IndexID: indexID, + IndexState: 1, + IsDeleted: false, + WriteHandoff: false, + }, + buildID + 1: { + SegmentID: segID + 1, + CollectionID: collID, + PartitionID: partID, + BuildID: buildID + 1, + IndexID: indexID, + IndexState: 1, + IsDeleted: true, + WriteHandoff: false, + }, + buildID + 2: { + SegmentID: segID + 2, + CollectionID: collID, + PartitionID: partID, + BuildID: buildID + 2, + IndexID: indexID, + IndexState: 1, + IsDeleted: false, + WriteHandoff: true, + }, + }, + } +} + +func Test_newHandoff(t *testing.T) { + ctx := context.Background() + hd := newHandoff(ctx, createMetaForHandoff(&indexcoord.Catalog{Txn: NewMockEtcdKV()}), NewMockEtcdKV(), &IndexCoord{dataCoordClient: NewDataCoordMock()}) + assert.NotNil(t, hd) + assert.Equal(t, 1, len(hd.tasks)) + + hd.enqueue(segID) + assert.Equal(t, 1, len(hd.tasks)) + + err := hd.meta.AddIndex(&model.SegmentIndex{ + SegmentID: segID + 3, + CollectionID: collID, + PartitionID: partID, + NumRows: 0, + IndexID: indexID, + BuildID: buildID + 3, + }) + assert.NoError(t, err) + hd.enqueue(segID + 3) + assert.Equal(t, 2, len(hd.tasks)) + + hd.Start() + err = hd.meta.FinishTask(&indexpb.IndexTaskInfo{ + BuildID: buildID, + State: commonpb.IndexState_Finished, + IndexFiles: []string{"file1", "file2"}, + SerializedSize: 100, + FailReason: "", + }) + assert.NoError(t, err) + err = hd.meta.FinishTask(&indexpb.IndexTaskInfo{ + BuildID: buildID + 3, + State: commonpb.IndexState_Failed, + IndexFiles: nil, + SerializedSize: 0, + FailReason: "failed", + }) + assert.NoError(t, err) + + // handle ticker + time.Sleep(time.Second * 2) + for hd.Len() != 0 { + time.Sleep(500 * time.Millisecond) + } + + assert.True(t, hd.taskDone(segID)) + assert.True(t, hd.taskDone(segID+3)) + + hd.Stop() +} + +func Test_handoff_error(t *testing.T) { + t.Run("pullSegmentInfo fail", func(t *testing.T) { + hd := &handoff{ + tasks: map[UniqueID]struct{}{ + segID: {}, + }, + taskMutex: sync.RWMutex{}, + wg: sync.WaitGroup{}, + meta: &metaTable{ + segmentIndexLock: sync.RWMutex{}, + segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ + segID: { + indexID: { + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + IndexID: indexID, + BuildID: buildID, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + WriteHandoff: false, + }, + }, + }, + }, + notifyChan: make(chan struct{}, 1), + scheduleDuration: 0, + kvClient: nil, + ic: &IndexCoord{ + dataCoordClient: &DataCoordMock{ + CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + return nil, errors.New("error") + }, + }, + }, + } + hd.process(segID, true) + assert.Equal(t, 1, hd.Len()) + + hd.ic.dataCoordClient = &DataCoordMock{ + CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + return nil, errSegmentNotFound(segID) + }, + } + hd.process(segID, true) + assert.Equal(t, 0, hd.Len()) + }) + + t.Run("is importing", func(t *testing.T) { + hd := &handoff{ + tasks: map[UniqueID]struct{}{ + segID: {}, + }, + taskMutex: sync.RWMutex{}, + wg: sync.WaitGroup{}, + meta: &metaTable{ + segmentIndexLock: sync.RWMutex{}, + segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ + segID: { + indexID: { + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + IndexID: indexID, + BuildID: buildID, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + WriteHandoff: false, + }, + }, + }, + }, + notifyChan: make(chan struct{}, 1), + scheduleDuration: 0, + kvClient: nil, + ic: &IndexCoord{ + dataCoordClient: &DataCoordMock{ + CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + return &datapb.GetSegmentInfoResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Infos: []*datapb.SegmentInfo{ + { + ID: segID, + CollectionID: collID, + PartitionID: partID, + InsertChannel: "", + NumOfRows: 1024, + State: commonpb.SegmentState_Flushed, + IsImporting: true, + }, + }, + }, nil + }, + }, + }, + } + + hd.process(segID, true) + assert.Equal(t, 1, hd.Len()) + }) + + t.Run("get index info fail", func(t *testing.T) { + hd := &handoff{ + tasks: map[UniqueID]struct{}{ + segID: {}, + }, + taskMutex: sync.RWMutex{}, + wg: sync.WaitGroup{}, + meta: &metaTable{ + segmentIndexLock: sync.RWMutex{}, + segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ + segID: { + indexID: { + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + IndexID: indexID, + BuildID: buildID, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: true, + WriteHandoff: false, + }, + }, + }, + }, + notifyChan: make(chan struct{}, 1), + scheduleDuration: 0, + kvClient: nil, + ic: &IndexCoord{ + dataCoordClient: NewDataCoordMock(), + }, + } + + hd.process(segID, true) + assert.Equal(t, 0, hd.Len()) + }) + + t.Run("write handoff fail", func(t *testing.T) { + hd := &handoff{ + tasks: map[UniqueID]struct{}{ + segID: {}, + }, + taskMutex: sync.RWMutex{}, + wg: sync.WaitGroup{}, + meta: &metaTable{ + catalog: &indexcoord.Catalog{Txn: NewMockEtcdKV()}, + segmentIndexLock: sync.RWMutex{}, + segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ + segID: { + indexID: { + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + IndexID: indexID, + BuildID: buildID, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + WriteHandoff: false, + }, + }, + }, + }, + notifyChan: make(chan struct{}, 1), + scheduleDuration: 0, + kvClient: &mockETCDKV{ + save: func(s string, s2 string) error { + return errors.New("error") + }, + }, + ic: &IndexCoord{ + dataCoordClient: NewDataCoordMock(), + }, + } + + hd.process(segID, true) + assert.Equal(t, 1, hd.Len()) + }) + + t.Run("mark meta as write handoff fail", func(t *testing.T) { + hd := &handoff{ + tasks: map[UniqueID]struct{}{ + segID: {}, + }, + taskMutex: sync.RWMutex{}, + wg: sync.WaitGroup{}, + meta: &metaTable{ + catalog: &indexcoord.Catalog{Txn: &mockETCDKV{ + multiSave: func(m map[string]string) error { + return errors.New("error") + }, + }}, + segmentIndexLock: sync.RWMutex{}, + segmentIndexes: map[UniqueID]map[UniqueID]*model.SegmentIndex{ + segID: { + indexID: { + SegmentID: segID, + CollectionID: collID, + PartitionID: partID, + IndexID: indexID, + BuildID: buildID, + IndexState: commonpb.IndexState_Finished, + FailReason: "", + IsDeleted: false, + WriteHandoff: false, + }, + }, + }, + }, + notifyChan: make(chan struct{}, 1), + scheduleDuration: 0, + kvClient: NewMockEtcdKV(), + ic: &IndexCoord{ + dataCoordClient: NewDataCoordMock(), + }, + } + + hd.process(segID, true) + assert.Equal(t, 1, hd.Len()) + }) +} + +func Test_handoff_allParentsDone(t *testing.T) { + t.Run("done", func(t *testing.T) { + hd := &handoff{ + tasks: map[UniqueID]struct{}{ + segID: {}, + }, + taskMutex: sync.RWMutex{}, + } + + done := hd.allParentsDone([]UniqueID{segID + 1, segID + 2, segID + 3}) + assert.True(t, done) + }) + + t.Run("not done", func(t *testing.T) { + hd := &handoff{ + tasks: map[UniqueID]struct{}{ + segID: {}, + segID + 1: {}, + }, + taskMutex: sync.RWMutex{}, + } + + done := hd.allParentsDone([]UniqueID{segID + 1, segID + 2, segID + 3}) + assert.False(t, done) + }) +} diff --git a/internal/indexcoord/index_coord.go b/internal/indexcoord/index_coord.go index afb883009f..199b7d7d76 100644 --- a/internal/indexcoord/index_coord.go +++ b/internal/indexcoord/index_coord.go @@ -87,6 +87,7 @@ type IndexCoord struct { indexBuilder *indexBuilder garbageCollector *garbageCollector flushedSegmentWatcher *flushedSegmentWatcher + handoff *handoff metricsCacheManager *metricsinfo.MetricsCacheManager @@ -218,7 +219,8 @@ func (i *IndexCoord) Init() error { i.chunkManager = chunkManager i.garbageCollector = newGarbageCollector(i.loopCtx, i.metaTable, i.chunkManager, i) - i.flushedSegmentWatcher, err = newFlushSegmentWatcher(i.loopCtx, i.etcdKV, i.metaTable, i.indexBuilder, i) + i.handoff = newHandoff(i.loopCtx, i.metaTable, i.etcdKV, i) + i.flushedSegmentWatcher, err = newFlushSegmentWatcher(i.loopCtx, i.etcdKV, i.metaTable, i.indexBuilder, i.handoff, i) if err != nil { initErr = err return @@ -254,6 +256,7 @@ func (i *IndexCoord) Start() error { i.indexBuilder.Start() i.garbageCollector.Start() + i.handoff.Start() i.flushedSegmentWatcher.Start() i.UpdateStateCode(internalpb.StateCode_Healthy) @@ -1051,3 +1054,30 @@ func (i *IndexCoord) watchFlushedSegmentLoop() { } } } + +func (i *IndexCoord) pullSegmentInfo(ctx context.Context, segmentID UniqueID) (*datapb.SegmentInfo, error) { + resp, err := i.dataCoordClient.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{ + SegmentIDs: []int64{segmentID}, + IncludeUnHealthy: false, + }) + if err != nil { + log.Error("IndexCoord get segment info fail", zap.Int64("segID", segmentID), zap.Error(err)) + return nil, err + } + if resp.Status.GetErrorCode() != commonpb.ErrorCode_Success { + log.Error("IndexCoord get segment info fail", zap.Int64("segID", segmentID), + zap.String("fail reason", resp.Status.GetReason())) + if resp.Status.GetReason() == msgSegmentNotFound(segmentID) { + return nil, errSegmentNotFound(segmentID) + } + return nil, errors.New(resp.Status.GetReason()) + } + for _, info := range resp.Infos { + if info.ID == segmentID { + return info, nil + } + } + errMsg := msgSegmentNotFound(segmentID) + log.Error(errMsg) + return nil, errSegmentNotFound(segmentID) +} diff --git a/internal/indexcoord/index_coord_test.go b/internal/indexcoord/index_coord_test.go index 30c4667ce9..77b19b1577 100644 --- a/internal/indexcoord/index_coord_test.go +++ b/internal/indexcoord/index_coord_test.go @@ -18,6 +18,7 @@ package indexcoord import ( "context" + "errors" "math/rand" "path" "strconv" @@ -683,3 +684,82 @@ func TestIndexCoord_UnHealthy(t *testing.T) { // assert.Nil(t, err) // assert.Equal(t, commonpb.ErrorCode_Success, status.GetErrorCode()) //} + +func TestIndexCoord_pullSegmentInfo(t *testing.T) { + t.Run("success", func(t *testing.T) { + ic := &IndexCoord{ + dataCoordClient: NewDataCoordMock(), + } + info, err := ic.pullSegmentInfo(context.Background(), segID) + assert.NoError(t, err) + assert.NotNil(t, info) + }) + + t.Run("fail", func(t *testing.T) { + ic := &IndexCoord{ + dataCoordClient: &DataCoordMock{ + CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + return nil, errors.New("error") + }, + }, + } + info, err := ic.pullSegmentInfo(context.Background(), segID) + assert.Error(t, err) + assert.Nil(t, info) + }) + + t.Run("not success", func(t *testing.T) { + ic := &IndexCoord{ + dataCoordClient: &DataCoordMock{ + CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + return &datapb.GetSegmentInfoResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "fail reason", + }, + }, nil + }, + }, + } + info, err := ic.pullSegmentInfo(context.Background(), segID) + assert.Error(t, err) + assert.Nil(t, info) + }) + + t.Run("failed to get segment", func(t *testing.T) { + ic := &IndexCoord{ + dataCoordClient: &DataCoordMock{ + CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + return &datapb.GetSegmentInfoResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: msgSegmentNotFound(segID), + }, + }, nil + }, + }, + } + info, err := ic.pullSegmentInfo(context.Background(), segID) + assert.Error(t, err) + assert.Nil(t, info) + }) + + t.Run("seg not exist", func(t *testing.T) { + ic := &IndexCoord{ + dataCoordClient: &DataCoordMock{ + CallGetSegmentInfo: func(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error) { + return &datapb.GetSegmentInfoResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, + Infos: []*datapb.SegmentInfo{}, + }, nil + }, + }, + } + info, err := ic.pullSegmentInfo(context.Background(), segID) + assert.ErrorIs(t, err, ErrSegmentNotFound) + assert.Nil(t, info) + }) +} diff --git a/internal/indexcoord/meta_table.go b/internal/indexcoord/meta_table.go index a53f254c52..d3dde98776 100644 --- a/internal/indexcoord/meta_table.go +++ b/internal/indexcoord/meta_table.go @@ -1032,3 +1032,18 @@ func (mt *metaTable) MarkSegmentsIndexAsDeletedByBuildID(buildIDs []UniqueID) er log.Info("IndexCoord metaTable MarkSegmentsIndexAsDeletedByBuildID success", zap.Int64s("buildIDs", buildIDs)) return nil } + +func (mt *metaTable) MarkSegmentWriteHandoff(segID UniqueID) error { + mt.segmentIndexLock.Lock() + defer mt.segmentIndexLock.Unlock() + + segIdxes := make([]*model.SegmentIndex, 0) + if segIndexes, ok := mt.segmentIndexes[segID]; ok { + for _, segIdx := range segIndexes { + clonedSegIdx := model.CloneSegmentIndex(segIdx) + clonedSegIdx.WriteHandoff = true + segIdxes = append(segIdxes, clonedSegIdx) + } + } + return mt.alterSegmentIndexes(segIdxes) +} diff --git a/internal/indexcoord/task.go b/internal/indexcoord/task.go index 201231512c..0d5ec83440 100644 --- a/internal/indexcoord/task.go +++ b/internal/indexcoord/task.go @@ -178,6 +178,7 @@ func (cit *CreateIndexTask) Execute(ctx context.Context) error { } buildIDs := make([]UniqueID, 0) + segIDs := make([]UniqueID, 0) for _, segmentInfo := range segmentsInfo.Infos { if segmentInfo.State != commonpb.SegmentState_Flushed { continue @@ -201,6 +202,7 @@ func (cit *CreateIndexTask) Execute(ctx context.Context) error { if have || buildID == 0 { continue } + segIDs = append(segIDs, segmentInfo.ID) buildIDs = append(buildIDs, buildID) } @@ -213,6 +215,9 @@ func (cit *CreateIndexTask) Execute(ctx context.Context) error { for _, buildID := range buildIDs { cit.indexCoordClient.indexBuilder.enqueue(buildID) } + for _, segID := range segIDs { + cit.indexCoordClient.handoff.enqueue(segID) + } return nil } diff --git a/internal/indexcoord/util.go b/internal/indexcoord/util.go index 37c6a86482..9587b87aa5 100644 --- a/internal/indexcoord/util.go +++ b/internal/indexcoord/util.go @@ -18,12 +18,14 @@ package indexcoord import ( "errors" + "fmt" "strconv" "strings" "github.com/milvus-io/milvus/api/schemapb" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/indexpb" + "github.com/milvus-io/milvus/internal/util" ) // getDimension gets the dimension of data from building index request. @@ -66,3 +68,7 @@ func parseBuildIDFromFilePath(key string) (UniqueID, error) { } return strconv.ParseInt(ss[len(ss)-1], 10, 64) } + +func buildHandoffKey(collID, partID, segID UniqueID) string { + return fmt.Sprintf("%s/%d/%d/%d", util.HandoffSegmentPrefix, collID, partID, segID) +} diff --git a/internal/metastore/model/segment_index.go b/internal/metastore/model/segment_index.go index d4abb8d8ca..b32d095ccc 100644 --- a/internal/metastore/model/segment_index.go +++ b/internal/metastore/model/segment_index.go @@ -20,6 +20,7 @@ type SegmentIndex struct { CreateTime uint64 IndexFilePaths []string IndexSize uint64 + WriteHandoff bool } func UnmarshalSegmentIndexModel(segIndex *indexpb.SegmentIndex) *SegmentIndex { @@ -42,6 +43,7 @@ func UnmarshalSegmentIndexModel(segIndex *indexpb.SegmentIndex) *SegmentIndex { CreateTime: segIndex.CreateTime, IndexFilePaths: segIndex.IndexFilesPaths, IndexSize: segIndex.SerializeSize, + WriteHandoff: segIndex.WriteHandoff, } } @@ -65,6 +67,7 @@ func MarshalSegmentIndexModel(segIdx *SegmentIndex) *indexpb.SegmentIndex { Deleted: segIdx.IsDeleted, CreateTime: segIdx.CreateTime, SerializeSize: segIdx.IndexSize, + WriteHandoff: segIdx.WriteHandoff, } } @@ -84,5 +87,6 @@ func CloneSegmentIndex(segIndex *SegmentIndex) *SegmentIndex { CreateTime: segIndex.CreateTime, IndexFilePaths: segIndex.IndexFilePaths, IndexSize: segIndex.IndexSize, + WriteHandoff: segIndex.WriteHandoff, } } diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index 6e73fc6021..be11543c9b 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -260,6 +260,10 @@ message SegmentInfo { bool createdByCompaction = 14; repeated int64 compactionFrom = 15; uint64 dropped_at = 16; // timestamp when segment marked drop + // A flag indicating if: + // (1) this segment is created by bulk load, and + // (2) the bulk load task that creates this segment has not yet reached `ImportCompleted` state. + bool is_importing = 17; } message SegmentStartPosition { diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index c2dcfa7d50..63bd292dba 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -1625,13 +1625,17 @@ type SegmentInfo struct { Binlogs []*FieldBinlog `protobuf:"bytes,11,rep,name=binlogs,proto3" json:"binlogs,omitempty"` Statslogs []*FieldBinlog `protobuf:"bytes,12,rep,name=statslogs,proto3" json:"statslogs,omitempty"` // deltalogs consists of delete binlogs. FieldID is not used yet since delete is always applied on primary key - Deltalogs []*FieldBinlog `protobuf:"bytes,13,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` - CreatedByCompaction bool `protobuf:"varint,14,opt,name=createdByCompaction,proto3" json:"createdByCompaction,omitempty"` - CompactionFrom []int64 `protobuf:"varint,15,rep,packed,name=compactionFrom,proto3" json:"compactionFrom,omitempty"` - DroppedAt uint64 `protobuf:"varint,16,opt,name=dropped_at,json=droppedAt,proto3" json:"dropped_at,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Deltalogs []*FieldBinlog `protobuf:"bytes,13,rep,name=deltalogs,proto3" json:"deltalogs,omitempty"` + CreatedByCompaction bool `protobuf:"varint,14,opt,name=createdByCompaction,proto3" json:"createdByCompaction,omitempty"` + CompactionFrom []int64 `protobuf:"varint,15,rep,packed,name=compactionFrom,proto3" json:"compactionFrom,omitempty"` + DroppedAt uint64 `protobuf:"varint,16,opt,name=dropped_at,json=droppedAt,proto3" json:"dropped_at,omitempty"` + // A flag indicating if: + // (1) this segment is created by bulk load, and + // (2) the bulk load task that creates this segment has not yet reached `ImportCompleted` state. + IsImporting bool `protobuf:"varint,17,opt,name=is_importing,json=isImporting,proto3" json:"is_importing,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *SegmentInfo) Reset() { *m = SegmentInfo{} } @@ -1771,6 +1775,13 @@ func (m *SegmentInfo) GetDroppedAt() uint64 { return 0 } +func (m *SegmentInfo) GetIsImporting() bool { + if m != nil { + return m.IsImporting + } + return false +} + type SegmentStartPosition struct { StartPosition *internalpb.MsgPosition `protobuf:"bytes,1,opt,name=start_position,json=startPosition,proto3" json:"start_position,omitempty"` SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` @@ -4354,249 +4365,250 @@ func init() { func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 3863 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3c, 0xdd, 0x6f, 0x1c, 0xd7, - 0x57, 0x99, 0xfd, 0xf2, 0xee, 0xd9, 0x0f, 0xaf, 0x6f, 0x52, 0x7b, 0xb3, 0xf9, 0x9e, 0x34, 0xa9, - 0x93, 0x5f, 0x9a, 0xb4, 0x0e, 0xd5, 0xaf, 0x22, 0xbf, 0xb6, 0x8a, 0xe3, 0xd8, 0x59, 0xb0, 0x53, - 0x67, 0xec, 0x34, 0x12, 0x45, 0x5a, 0x8d, 0x77, 0xae, 0xd7, 0x53, 0xef, 0xcc, 0x6c, 0x66, 0x66, - 0xe3, 0xb8, 0x3c, 0xb4, 0xa2, 0x12, 0x52, 0x11, 0xa2, 0x08, 0x84, 0x04, 0x0f, 0x48, 0x88, 0xa7, - 0x82, 0x84, 0x84, 0x54, 0xf1, 0x00, 0xa8, 0xef, 0x15, 0x3c, 0x20, 0x1e, 0xf9, 0x07, 0x80, 0x27, - 0xfe, 0x04, 0x84, 0xee, 0xc7, 0xdc, 0xf9, 0xde, 0x1d, 0xef, 0x26, 0x0d, 0xe2, 0xcd, 0xf7, 0xcc, - 0x39, 0xf7, 0x9e, 0x7b, 0xee, 0xf9, 0xbe, 0x77, 0x0d, 0x4d, 0x4d, 0x75, 0xd5, 0x6e, 0xcf, 0xb2, - 0x6c, 0xed, 0xf6, 0xd0, 0xb6, 0x5c, 0x0b, 0x2d, 0x18, 0xfa, 0xe0, 0xc5, 0xc8, 0x61, 0xa3, 0xdb, - 0xe4, 0x73, 0xbb, 0xd6, 0xb3, 0x0c, 0xc3, 0x32, 0x19, 0xa8, 0xdd, 0xd0, 0x4d, 0x17, 0xdb, 0xa6, - 0x3a, 0xe0, 0xe3, 0x5a, 0x90, 0xa0, 0x5d, 0x73, 0x7a, 0x07, 0xd8, 0x50, 0xd9, 0x48, 0x9e, 0x83, - 0xe2, 0x43, 0x63, 0xe8, 0x1e, 0xcb, 0x7f, 0x26, 0x41, 0x6d, 0x7d, 0x30, 0x72, 0x0e, 0x14, 0xfc, - 0x7c, 0x84, 0x1d, 0x17, 0xbd, 0x07, 0x85, 0x3d, 0xd5, 0xc1, 0x2d, 0xe9, 0xb2, 0xb4, 0x5c, 0x5d, - 0x39, 0x7f, 0x3b, 0xb4, 0x2a, 0x5f, 0x6f, 0xcb, 0xe9, 0xaf, 0xaa, 0x0e, 0x56, 0x28, 0x26, 0x42, - 0x50, 0xd0, 0xf6, 0x3a, 0x6b, 0xad, 0xdc, 0x65, 0x69, 0x39, 0xaf, 0xd0, 0xbf, 0xd1, 0x45, 0x00, - 0x07, 0xf7, 0x0d, 0x6c, 0xba, 0x9d, 0x35, 0xa7, 0x95, 0xbf, 0x9c, 0x5f, 0xce, 0x2b, 0x01, 0x08, - 0x92, 0xa1, 0xd6, 0xb3, 0x06, 0x03, 0xdc, 0x73, 0x75, 0xcb, 0xec, 0xac, 0xb5, 0x0a, 0x94, 0x36, - 0x04, 0x93, 0xff, 0x43, 0x82, 0x3a, 0x67, 0xcd, 0x19, 0x5a, 0xa6, 0x83, 0xd1, 0x5d, 0x28, 0x39, - 0xae, 0xea, 0x8e, 0x1c, 0xce, 0xdd, 0xb9, 0x44, 0xee, 0x76, 0x28, 0x8a, 0xc2, 0x51, 0x13, 0xd9, - 0x8b, 0x2e, 0x9f, 0x8f, 0x2f, 0x1f, 0xd9, 0x42, 0x21, 0xb6, 0x85, 0x65, 0x98, 0xdf, 0x27, 0xdc, - 0xed, 0xf8, 0x48, 0x45, 0x8a, 0x14, 0x05, 0x93, 0x99, 0x5c, 0xdd, 0xc0, 0x9f, 0xee, 0xef, 0x60, - 0x75, 0xd0, 0x2a, 0xd1, 0xb5, 0x02, 0x10, 0xf9, 0xdf, 0x24, 0x68, 0x0a, 0x74, 0xef, 0x1c, 0xce, - 0x40, 0xb1, 0x67, 0x8d, 0x4c, 0x97, 0x6e, 0xb5, 0xae, 0xb0, 0x01, 0xba, 0x02, 0xb5, 0xde, 0x81, - 0x6a, 0x9a, 0x78, 0xd0, 0x35, 0x55, 0x03, 0xd3, 0x4d, 0x55, 0x94, 0x2a, 0x87, 0x3d, 0x56, 0x0d, - 0x9c, 0x69, 0x6f, 0x97, 0xa1, 0x3a, 0x54, 0x6d, 0x57, 0x0f, 0x49, 0x3f, 0x08, 0x42, 0x6d, 0x28, - 0xeb, 0x4e, 0xc7, 0x18, 0x5a, 0xb6, 0xdb, 0x2a, 0x5e, 0x96, 0x96, 0xcb, 0x8a, 0x18, 0x93, 0x15, - 0x74, 0xfa, 0xd7, 0xae, 0xea, 0x1c, 0x76, 0xd6, 0xf8, 0x8e, 0x42, 0x30, 0xf9, 0x2f, 0x25, 0x58, - 0xbc, 0xef, 0x38, 0x7a, 0xdf, 0x8c, 0xed, 0x6c, 0x11, 0x4a, 0xa6, 0xa5, 0xe1, 0xce, 0x1a, 0xdd, - 0x5a, 0x5e, 0xe1, 0x23, 0x74, 0x0e, 0x2a, 0x43, 0x8c, 0xed, 0xae, 0x6d, 0x0d, 0xbc, 0x8d, 0x95, - 0x09, 0x40, 0xb1, 0x06, 0x18, 0x3d, 0x81, 0x05, 0x27, 0x32, 0x11, 0xd3, 0xab, 0xea, 0xca, 0xd5, - 0xdb, 0x31, 0xcb, 0xb8, 0x1d, 0x5d, 0x54, 0x89, 0x53, 0xcb, 0x5f, 0xe7, 0xe0, 0xb4, 0xc0, 0x63, - 0xbc, 0x92, 0xbf, 0x89, 0xe4, 0x1d, 0xdc, 0x17, 0xec, 0xb1, 0x41, 0x16, 0xc9, 0x8b, 0x23, 0xcb, - 0x07, 0x8f, 0x2c, 0x83, 0xaa, 0x47, 0xcf, 0xa3, 0x18, 0x3f, 0x8f, 0x4b, 0x50, 0xc5, 0x2f, 0x87, - 0xba, 0x8d, 0xbb, 0x44, 0x71, 0xa8, 0xc8, 0x0b, 0x0a, 0x30, 0xd0, 0xae, 0x6e, 0x04, 0x6d, 0x63, - 0x2e, 0xb3, 0x6d, 0xc8, 0x7f, 0x25, 0xc1, 0x52, 0xec, 0x94, 0xb8, 0xb1, 0x29, 0xd0, 0xa4, 0x3b, - 0xf7, 0x25, 0x43, 0xcc, 0x8e, 0x08, 0xfc, 0xfa, 0x38, 0x81, 0xfb, 0xe8, 0x4a, 0x8c, 0x3e, 0xc0, - 0x64, 0x2e, 0x3b, 0x93, 0x87, 0xb0, 0xb4, 0x81, 0x5d, 0xbe, 0x00, 0xf9, 0x86, 0x9d, 0xe9, 0x9d, - 0x55, 0xd8, 0xaa, 0x73, 0x51, 0xab, 0x96, 0xff, 0x2e, 0x27, 0x6c, 0x91, 0x2e, 0xd5, 0x31, 0xf7, - 0x2d, 0x74, 0x1e, 0x2a, 0x02, 0x85, 0x6b, 0x85, 0x0f, 0x40, 0xbf, 0x84, 0x22, 0xe1, 0x94, 0xa9, - 0x44, 0x63, 0xe5, 0x4a, 0xf2, 0x9e, 0x02, 0x73, 0x2a, 0x0c, 0x1f, 0x75, 0xa0, 0xe1, 0xb8, 0xaa, - 0xed, 0x76, 0x87, 0x96, 0x43, 0xcf, 0x99, 0x2a, 0x4e, 0x75, 0x45, 0x0e, 0xcf, 0x20, 0xdc, 0xfa, - 0x96, 0xd3, 0xdf, 0xe6, 0x98, 0x4a, 0x9d, 0x52, 0x7a, 0x43, 0xf4, 0x10, 0x6a, 0xd8, 0xd4, 0xfc, - 0x89, 0x0a, 0x99, 0x27, 0xaa, 0x62, 0x53, 0x13, 0xd3, 0xf8, 0xe7, 0x53, 0xcc, 0x7e, 0x3e, 0x7f, - 0x20, 0x41, 0x2b, 0x7e, 0x40, 0xb3, 0xb8, 0xec, 0x7b, 0x8c, 0x08, 0xb3, 0x03, 0x1a, 0x6b, 0xe1, - 0xe2, 0x90, 0x14, 0x4e, 0x22, 0xff, 0xa9, 0x04, 0x6f, 0xf9, 0xec, 0xd0, 0x4f, 0xaf, 0x4b, 0x5b, - 0xd0, 0x4d, 0x68, 0xea, 0x66, 0x6f, 0x30, 0xd2, 0xf0, 0x53, 0xf3, 0x11, 0x56, 0x07, 0xee, 0xc1, - 0x31, 0x3d, 0xc3, 0xb2, 0x12, 0x83, 0xcb, 0xdf, 0x48, 0xb0, 0x18, 0xe5, 0x6b, 0x16, 0x21, 0xfd, - 0x1a, 0x14, 0x75, 0x73, 0xdf, 0xf2, 0x64, 0x74, 0x71, 0x8c, 0x51, 0x92, 0xb5, 0x18, 0xb2, 0x6c, - 0xc0, 0xb9, 0x0d, 0xec, 0x76, 0x4c, 0x07, 0xdb, 0xee, 0xaa, 0x6e, 0x0e, 0xac, 0xfe, 0xb6, 0xea, - 0x1e, 0xcc, 0x60, 0x50, 0x21, 0xdb, 0xc8, 0x45, 0x6c, 0x43, 0xfe, 0x5e, 0x82, 0xf3, 0xc9, 0xeb, - 0xf1, 0xad, 0xb7, 0xa1, 0xbc, 0xaf, 0xe3, 0x81, 0x46, 0xe4, 0x2b, 0x51, 0xf9, 0x8a, 0x31, 0x31, - 0xac, 0x21, 0x41, 0xe6, 0x3b, 0xbc, 0x92, 0xa2, 0xcd, 0x3b, 0xae, 0xad, 0x9b, 0xfd, 0x4d, 0xdd, - 0x71, 0x15, 0x86, 0x1f, 0x90, 0x67, 0x3e, 0xbb, 0x1a, 0xff, 0xbe, 0x04, 0x17, 0x37, 0xb0, 0xfb, - 0x40, 0xf8, 0x65, 0xf2, 0x5d, 0x77, 0x5c, 0xbd, 0xe7, 0xbc, 0xda, 0xdc, 0x28, 0x43, 0x80, 0x96, - 0xbf, 0x93, 0xe0, 0x52, 0x2a, 0x33, 0x5c, 0x74, 0xdc, 0xef, 0x78, 0x5e, 0x39, 0xd9, 0xef, 0xfc, - 0x26, 0x3e, 0xfe, 0x4c, 0x1d, 0x8c, 0xf0, 0xb6, 0xaa, 0xdb, 0xcc, 0xef, 0x4c, 0xe9, 0x85, 0xff, - 0x56, 0x82, 0x0b, 0x1b, 0xd8, 0xdd, 0xf6, 0x62, 0xd2, 0x1b, 0x94, 0x0e, 0xc1, 0x09, 0xc4, 0x46, - 0x2f, 0x39, 0x0b, 0xc1, 0xe4, 0x3f, 0x64, 0xc7, 0x99, 0xc8, 0xef, 0x1b, 0x11, 0xe0, 0x45, 0x6a, - 0x09, 0x01, 0x93, 0x7c, 0xc0, 0x52, 0x07, 0x2e, 0x3e, 0xf9, 0x2f, 0x24, 0x38, 0x7b, 0xbf, 0xf7, - 0x7c, 0xa4, 0xdb, 0x98, 0x23, 0x6d, 0x5a, 0xbd, 0xc3, 0xe9, 0x85, 0xeb, 0xa7, 0x59, 0xb9, 0x50, - 0x9a, 0x35, 0x29, 0x35, 0x5f, 0x84, 0x92, 0xcb, 0xf2, 0x3a, 0x96, 0xa9, 0xf0, 0x11, 0xe5, 0x4f, - 0xc1, 0x03, 0xac, 0x3a, 0xff, 0x37, 0xf9, 0xfb, 0xae, 0x00, 0xb5, 0xcf, 0x78, 0x3a, 0x46, 0xa3, - 0x76, 0x54, 0x93, 0xa4, 0xe4, 0xc4, 0x2b, 0x90, 0xc1, 0x25, 0x25, 0x75, 0x1b, 0x50, 0x77, 0x30, - 0x3e, 0x9c, 0x26, 0x46, 0xd7, 0x08, 0xa1, 0x88, 0xad, 0x9b, 0xb0, 0x30, 0x32, 0x69, 0x69, 0x80, - 0x35, 0x2e, 0x40, 0xa6, 0xb9, 0x93, 0x7d, 0x77, 0x9c, 0x10, 0x3d, 0xe2, 0xd5, 0x47, 0x60, 0xae, - 0x62, 0xa6, 0xb9, 0xa2, 0x64, 0xa8, 0x03, 0x4d, 0xcd, 0xb6, 0x86, 0x43, 0xac, 0x75, 0x1d, 0x6f, - 0xaa, 0x52, 0xb6, 0xa9, 0x38, 0x9d, 0x98, 0xea, 0x3d, 0x38, 0x1d, 0xe5, 0xb4, 0xa3, 0x91, 0x84, - 0x94, 0x9c, 0x61, 0xd2, 0x27, 0x74, 0x0b, 0x16, 0xe2, 0xf8, 0x65, 0x8a, 0x1f, 0xff, 0x80, 0xde, - 0x05, 0x14, 0x61, 0x95, 0xa0, 0x57, 0x18, 0x7a, 0x98, 0x99, 0x8e, 0xe6, 0xc8, 0xdf, 0x4a, 0xb0, - 0xf8, 0x4c, 0x75, 0x7b, 0x07, 0x6b, 0x06, 0xb7, 0xb5, 0x19, 0x7c, 0xd5, 0x47, 0x50, 0x79, 0xc1, - 0xf5, 0xc2, 0x0b, 0x48, 0x97, 0x12, 0xe4, 0x13, 0xd4, 0x40, 0xc5, 0xa7, 0x90, 0x7f, 0x92, 0xe0, - 0xcc, 0x7a, 0xa0, 0x2e, 0x7c, 0x03, 0x5e, 0x73, 0x52, 0x41, 0x7b, 0x1d, 0x1a, 0x86, 0x6a, 0x1f, - 0xc6, 0xea, 0xd9, 0x08, 0x54, 0x7e, 0x09, 0xc0, 0x47, 0x5b, 0x4e, 0x7f, 0x0a, 0xfe, 0x3f, 0x84, - 0x39, 0xbe, 0x2a, 0x77, 0x9f, 0x93, 0xf4, 0xcc, 0x43, 0x97, 0xff, 0x59, 0x82, 0x86, 0x1f, 0x12, - 0xa9, 0x91, 0x37, 0x20, 0x27, 0x4c, 0x3b, 0xd7, 0x59, 0x43, 0x1f, 0x41, 0x89, 0x35, 0x3a, 0xf8, - 0xdc, 0xd7, 0xc2, 0x73, 0xf3, 0x26, 0x48, 0x20, 0xae, 0x52, 0x80, 0xc2, 0x89, 0x88, 0x8c, 0x44, - 0x14, 0x11, 0xce, 0xc7, 0x87, 0xa0, 0x0e, 0xcc, 0x87, 0x53, 0x76, 0xcf, 0x84, 0x2f, 0xa7, 0x05, - 0x8f, 0x35, 0xd5, 0x55, 0x69, 0xec, 0x68, 0x84, 0x32, 0x76, 0x47, 0xfe, 0xef, 0x22, 0x54, 0x03, - 0xbb, 0x8c, 0xed, 0x24, 0x7a, 0xa4, 0xb9, 0xc9, 0x75, 0x63, 0x3e, 0x5e, 0x37, 0x5e, 0x83, 0x86, - 0x4e, 0x93, 0xaf, 0x2e, 0x57, 0x45, 0xea, 0x35, 0x2b, 0x4a, 0x9d, 0x41, 0xb9, 0x5d, 0xa0, 0x8b, - 0x50, 0x35, 0x47, 0x46, 0xd7, 0xda, 0xef, 0xda, 0xd6, 0x91, 0xc3, 0x0b, 0xd0, 0x8a, 0x39, 0x32, - 0x3e, 0xdd, 0x57, 0xac, 0x23, 0xc7, 0xaf, 0x71, 0x4a, 0x27, 0xac, 0x71, 0x2e, 0x42, 0xd5, 0x50, - 0x5f, 0x92, 0x59, 0xbb, 0xe6, 0xc8, 0xa0, 0xb5, 0x69, 0x5e, 0xa9, 0x18, 0xea, 0x4b, 0xc5, 0x3a, - 0x7a, 0x3c, 0x32, 0xd0, 0x32, 0x34, 0x07, 0xaa, 0xe3, 0x76, 0x83, 0xc5, 0x6d, 0x99, 0x16, 0xb7, - 0x0d, 0x02, 0x7f, 0xe8, 0x17, 0xb8, 0xf1, 0x6a, 0xa9, 0x32, 0x43, 0xb5, 0xa4, 0x19, 0x03, 0x7f, - 0x22, 0xc8, 0x5e, 0x2d, 0x69, 0xc6, 0x40, 0x4c, 0xf3, 0x21, 0xcc, 0xed, 0xd1, 0x94, 0xd6, 0x69, - 0x55, 0x53, 0x1d, 0xe6, 0x3a, 0xc9, 0x66, 0x59, 0xe6, 0xab, 0x78, 0xe8, 0xe8, 0x57, 0x50, 0xa1, - 0x99, 0x04, 0xa5, 0xad, 0x65, 0xa2, 0xf5, 0x09, 0x08, 0xb5, 0x86, 0x07, 0xae, 0x4a, 0xa9, 0xeb, - 0xd9, 0xa8, 0x05, 0x01, 0x71, 0xd2, 0x3d, 0x1b, 0xab, 0x2e, 0xd6, 0x56, 0x8f, 0x1f, 0x58, 0xc6, - 0x50, 0xa5, 0xca, 0xd4, 0x6a, 0xd0, 0xb2, 0x25, 0xe9, 0x13, 0x71, 0x0c, 0x3d, 0x31, 0x5a, 0xb7, - 0x2d, 0xa3, 0x35, 0xcf, 0x1c, 0x43, 0x18, 0x8a, 0x2e, 0x00, 0x78, 0xee, 0x59, 0x75, 0x5b, 0x4d, - 0x7a, 0x8a, 0x15, 0x0e, 0xb9, 0xef, 0xca, 0x5f, 0xc1, 0x19, 0x5f, 0x43, 0x02, 0xa7, 0x11, 0x3f, - 0x58, 0x69, 0xda, 0x83, 0x1d, 0x5f, 0x8c, 0xfc, 0x6b, 0x01, 0x16, 0x77, 0xd4, 0x17, 0xf8, 0xf5, - 0xd7, 0x3d, 0x99, 0xfc, 0xf1, 0x26, 0x2c, 0xd0, 0x52, 0x67, 0x25, 0xc0, 0xcf, 0x98, 0x84, 0x20, - 0x78, 0x9c, 0x71, 0x42, 0xf4, 0x09, 0xc9, 0x64, 0x70, 0xef, 0x70, 0xdb, 0xd2, 0xfd, 0x64, 0xe0, - 0x42, 0xc2, 0x3c, 0x0f, 0x04, 0x96, 0x12, 0xa4, 0x40, 0xdb, 0x71, 0xd7, 0xc6, 0xd2, 0x80, 0x77, - 0xc6, 0x56, 0xdf, 0xbe, 0xf4, 0xa3, 0x1e, 0x0e, 0xb5, 0x60, 0x8e, 0xc7, 0x70, 0x6a, 0xf7, 0x65, - 0xc5, 0x1b, 0xa2, 0x6d, 0x38, 0xcd, 0x76, 0xb0, 0xc3, 0x95, 0x9a, 0x6d, 0xbe, 0x9c, 0x69, 0xf3, - 0x49, 0xa4, 0x61, 0x9b, 0xa8, 0x9c, 0xd4, 0x26, 0x5a, 0x30, 0xc7, 0xf5, 0x94, 0xfa, 0x82, 0xb2, - 0xe2, 0x0d, 0xc9, 0x31, 0xb3, 0xbe, 0xa6, 0x6e, 0xf6, 0x5b, 0x55, 0xfa, 0xcd, 0x07, 0x90, 0x9a, - 0x11, 0x7c, 0x79, 0x4e, 0xe8, 0x13, 0x7d, 0x0c, 0x65, 0xa1, 0xe1, 0xb9, 0xcc, 0x1a, 0x2e, 0x68, - 0xa2, 0x3e, 0x3a, 0x1f, 0xf1, 0xd1, 0xf2, 0xbf, 0x48, 0x50, 0x5b, 0x23, 0x5b, 0xda, 0xb4, 0xfa, - 0x34, 0xa2, 0x5c, 0x83, 0x86, 0x8d, 0x7b, 0x96, 0xad, 0x75, 0xb1, 0xe9, 0xda, 0x3a, 0x66, 0xed, - 0x85, 0x82, 0x52, 0x67, 0xd0, 0x87, 0x0c, 0x48, 0xd0, 0x88, 0xdb, 0x75, 0x5c, 0xd5, 0x18, 0x76, - 0xf7, 0x89, 0x79, 0xe7, 0x18, 0x9a, 0x80, 0x52, 0xeb, 0xbe, 0x02, 0x35, 0x1f, 0xcd, 0xb5, 0xe8, - 0xfa, 0x05, 0xa5, 0x2a, 0x60, 0xbb, 0x16, 0x7a, 0x1b, 0x1a, 0x54, 0xa6, 0xdd, 0x81, 0xd5, 0xef, - 0x92, 0x52, 0x9c, 0x07, 0x9b, 0x9a, 0xc6, 0xd9, 0x22, 0x67, 0x15, 0xc6, 0x72, 0xf4, 0x2f, 0x31, - 0x0f, 0x37, 0x02, 0x6b, 0x47, 0xff, 0x12, 0x93, 0x58, 0x5f, 0x27, 0xb1, 0xf3, 0xb1, 0xa5, 0xe1, - 0xdd, 0x29, 0x33, 0x8d, 0x0c, 0x3d, 0xdb, 0xf3, 0x50, 0x11, 0x3b, 0xe0, 0x5b, 0xf2, 0x01, 0x68, - 0x1d, 0x1a, 0x5e, 0x4e, 0xdc, 0x65, 0xa5, 0x62, 0x21, 0x35, 0xf3, 0x0b, 0x44, 0x3f, 0x47, 0xa9, - 0x7b, 0x64, 0x74, 0x28, 0xaf, 0x43, 0x2d, 0xf8, 0x99, 0xac, 0xba, 0x13, 0x55, 0x14, 0x01, 0x20, - 0xda, 0xf8, 0x78, 0x64, 0x90, 0x33, 0xe5, 0x8e, 0xc5, 0x1b, 0xca, 0xdf, 0x48, 0x50, 0xe7, 0x21, - 0x7b, 0x47, 0xdc, 0x6e, 0xd0, 0xad, 0x49, 0x74, 0x6b, 0xf4, 0x6f, 0xf4, 0xeb, 0xe1, 0x86, 0xe4, - 0xdb, 0x89, 0x4e, 0x80, 0x4e, 0x42, 0xb3, 0xe3, 0x50, 0xbc, 0xce, 0xd2, 0x9c, 0xf8, 0x9a, 0x28, - 0x1a, 0x3f, 0x1a, 0xaa, 0x68, 0x2d, 0x98, 0x53, 0x35, 0xcd, 0xc6, 0x8e, 0xc3, 0xf9, 0xf0, 0x86, - 0xe4, 0xcb, 0x0b, 0x6c, 0x3b, 0x9e, 0xca, 0xe7, 0x15, 0x6f, 0x88, 0x7e, 0x05, 0x65, 0x91, 0x4e, - 0xe7, 0x93, 0x52, 0xa8, 0x20, 0x9f, 0xbc, 0x94, 0x16, 0x14, 0xf2, 0xdf, 0xe7, 0xa0, 0xc1, 0x05, - 0xb6, 0xca, 0x63, 0xea, 0x78, 0xe3, 0x5b, 0x85, 0xda, 0xbe, 0x6f, 0xfb, 0xe3, 0x9a, 0x66, 0x41, - 0x17, 0x11, 0xa2, 0x99, 0x64, 0x80, 0xe1, 0xa8, 0x5e, 0x98, 0x29, 0xaa, 0x17, 0x4f, 0xea, 0xc1, - 0xe2, 0x79, 0x5e, 0x29, 0x21, 0xcf, 0x93, 0x7f, 0x1b, 0xaa, 0x81, 0x09, 0xa8, 0x87, 0x66, 0xdd, - 0x36, 0x2e, 0x31, 0x6f, 0x88, 0xee, 0xfa, 0xb9, 0x0d, 0x13, 0xd5, 0xd9, 0x04, 0x5e, 0x22, 0x69, - 0x8d, 0xfc, 0xd7, 0x12, 0x94, 0xf8, 0xcc, 0x97, 0xa0, 0xca, 0x9d, 0x0e, 0xcd, 0xfb, 0xd8, 0xec, - 0xc0, 0x41, 0x24, 0xf1, 0x7b, 0x75, 0x5e, 0xe7, 0x2c, 0x94, 0x23, 0xfe, 0x66, 0x8e, 0x87, 0x05, - 0xef, 0x53, 0xc0, 0xc9, 0x90, 0x4f, 0xd4, 0xbf, 0xfc, 0x24, 0xd1, 0x6b, 0x05, 0x05, 0xf7, 0xac, - 0x17, 0xd8, 0x3e, 0x9e, 0xbd, 0x1f, 0x7b, 0x2f, 0xa0, 0xd0, 0x19, 0xeb, 0x43, 0x41, 0x80, 0xee, - 0xf9, 0xe2, 0xce, 0x27, 0x35, 0xa3, 0x82, 0x1e, 0x86, 0xab, 0xa3, 0x2f, 0xf6, 0x3f, 0x62, 0x9d, - 0xe5, 0xf0, 0x56, 0xa6, 0xcd, 0x6b, 0x5e, 0x49, 0xd9, 0x21, 0xff, 0x89, 0x04, 0x67, 0x37, 0xb0, - 0xbb, 0x1e, 0xee, 0x35, 0xbc, 0x69, 0xae, 0x0c, 0x68, 0x27, 0x31, 0x35, 0xcb, 0xa9, 0xb7, 0xa1, - 0x2c, 0xba, 0x26, 0xec, 0x7e, 0x40, 0x8c, 0xe5, 0xdf, 0x93, 0xa0, 0xc5, 0x57, 0xa1, 0x6b, 0x92, - 0x94, 0x7a, 0x80, 0x5d, 0xac, 0xfd, 0xdc, 0x75, 0xf3, 0x8f, 0x12, 0x34, 0x83, 0x1e, 0x9f, 0x3a, - 0xed, 0x0f, 0xa0, 0x48, 0xdb, 0x13, 0x9c, 0x83, 0x89, 0xca, 0xca, 0xb0, 0x89, 0xcb, 0xa0, 0x69, - 0xde, 0xae, 0x08, 0x4e, 0x7c, 0xe8, 0x87, 0x9d, 0xfc, 0xc9, 0xc3, 0x0e, 0x0f, 0xc3, 0xd6, 0x88, - 0xcc, 0xcb, 0xfa, 0x7a, 0x3e, 0x40, 0xfe, 0x0d, 0x58, 0xf4, 0xcb, 0x11, 0x46, 0x37, 0xad, 0x26, - 0xc9, 0x3f, 0xe4, 0xa0, 0x15, 0x98, 0xec, 0xe7, 0x8e, 0x21, 0x29, 0x99, 0x6f, 0xfe, 0x15, 0x65, - 0xbe, 0x85, 0xd9, 0xe3, 0x46, 0x31, 0x29, 0x6e, 0xfc, 0x53, 0x0e, 0x1a, 0xbe, 0xd4, 0xb6, 0x07, - 0xaa, 0x89, 0x16, 0xa1, 0x34, 0x1c, 0xa8, 0x7e, 0x63, 0x95, 0x8f, 0xd0, 0x8e, 0xc8, 0x99, 0xc2, - 0x72, 0xfa, 0x45, 0x92, 0x3e, 0xa4, 0x1c, 0x84, 0x12, 0x99, 0x82, 0x94, 0x96, 0xac, 0x38, 0xa1, - 0x0d, 0x02, 0x9e, 0xa7, 0x31, 0xc5, 0xd3, 0x0d, 0x8c, 0x6e, 0x01, 0xe2, 0xda, 0xd2, 0xd5, 0xcd, - 0xae, 0x83, 0x7b, 0x96, 0xa9, 0x31, 0x3d, 0x2a, 0x2a, 0x4d, 0xfe, 0xa5, 0x63, 0xee, 0x30, 0x38, - 0xfa, 0x00, 0x0a, 0xee, 0xf1, 0x90, 0x45, 0x84, 0x46, 0xa2, 0xa7, 0xf5, 0xf9, 0xda, 0x3d, 0x1e, - 0x62, 0x85, 0xa2, 0x7b, 0xcf, 0x38, 0x5c, 0x5b, 0x7d, 0xc1, 0xc3, 0x6b, 0x41, 0x09, 0x40, 0x88, - 0x65, 0x78, 0x32, 0x9c, 0x63, 0x61, 0x88, 0x0f, 0xe5, 0x7f, 0xc8, 0x41, 0xd3, 0x9f, 0x52, 0xc1, - 0xce, 0x68, 0xe0, 0xa6, 0xca, 0x6f, 0x7c, 0x61, 0x39, 0x29, 0x07, 0xf9, 0x04, 0xaa, 0xfc, 0x3c, - 0x4f, 0xa0, 0x0f, 0xc0, 0x48, 0x36, 0xc7, 0x28, 0x68, 0xf1, 0x15, 0x29, 0x68, 0xe9, 0x84, 0x0a, - 0x2a, 0x7f, 0x2f, 0xc1, 0x5b, 0x31, 0xe3, 0x1f, 0x2b, 0xc0, 0xf1, 0xe9, 0x2f, 0x77, 0x0a, 0xd1, - 0x29, 0xb9, 0x1f, 0xba, 0x07, 0x25, 0x9b, 0xce, 0xce, 0xdb, 0xfc, 0x57, 0xc7, 0x2a, 0x07, 0x63, - 0x44, 0xe1, 0x24, 0xf2, 0x1f, 0x4b, 0xb0, 0x14, 0x67, 0x75, 0x86, 0xe0, 0xb2, 0x0a, 0x73, 0x6c, - 0x6a, 0xcf, 0x86, 0x96, 0xc7, 0xdb, 0x90, 0x2f, 0x1c, 0xc5, 0x23, 0x94, 0x77, 0x60, 0xd1, 0x8b, - 0x41, 0xbe, 0x80, 0xb7, 0xb0, 0xab, 0x8e, 0x49, 0xfe, 0x2e, 0x41, 0x95, 0xe5, 0x16, 0x2c, 0xa9, - 0x62, 0x65, 0x13, 0xec, 0x89, 0x6e, 0x03, 0x49, 0xf4, 0xce, 0x50, 0x27, 0x1e, 0xed, 0xab, 0x67, - 0xb9, 0x73, 0x91, 0x45, 0x55, 0x46, 0x2a, 0x30, 0xb6, 0xb5, 0x8a, 0x12, 0x82, 0x25, 0xf5, 0x59, - 0xf3, 0x53, 0xf6, 0x59, 0x37, 0xe1, 0xad, 0x08, 0xab, 0x33, 0x1c, 0x09, 0xd9, 0xf9, 0xe2, 0x4e, - 0xf8, 0xb1, 0xc3, 0xf4, 0x59, 0xcd, 0x05, 0xd1, 0x91, 0xef, 0xea, 0x5a, 0xd4, 0xd6, 0x35, 0xf4, - 0x31, 0x54, 0x4c, 0x7c, 0xd4, 0x0d, 0x06, 0xd5, 0x0c, 0x8d, 0xd7, 0xb2, 0x89, 0x8f, 0xe8, 0x5f, - 0xf2, 0x63, 0x58, 0x8a, 0xb1, 0x3a, 0xcb, 0xde, 0xff, 0x51, 0x82, 0xb3, 0x6b, 0xb6, 0x35, 0xfc, - 0x4c, 0xb7, 0xdd, 0x91, 0x3a, 0x08, 0xdf, 0x5f, 0xbe, 0x9e, 0xf2, 0xfc, 0x51, 0x20, 0xbd, 0x62, - 0x0a, 0x70, 0x2b, 0xc1, 0x04, 0xe2, 0x4c, 0xf1, 0x4d, 0x07, 0x92, 0xb1, 0xff, 0xcc, 0x27, 0x31, - 0xcf, 0xf1, 0x26, 0x04, 0xfe, 0x2c, 0xd9, 0x67, 0x62, 0x37, 0x2f, 0x3f, 0x6d, 0x37, 0x2f, 0xc5, - 0x0b, 0x17, 0x5e, 0x91, 0x17, 0x3e, 0x71, 0x79, 0xf9, 0x08, 0xc2, 0x9d, 0x56, 0x1a, 0xfe, 0xa6, - 0x6a, 0xd1, 0xae, 0x02, 0xf8, 0x5d, 0x47, 0xfe, 0x56, 0x2d, 0xcb, 0x34, 0x01, 0x2a, 0x72, 0x5a, - 0x22, 0xe2, 0xd1, 0xdb, 0x82, 0x50, 0x1f, 0xec, 0x09, 0xb4, 0x93, 0xb4, 0x74, 0x16, 0xcd, 0xff, - 0x21, 0x07, 0xd0, 0x11, 0xcf, 0x1b, 0xa7, 0x73, 0xe6, 0x57, 0xa1, 0xee, 0x2b, 0x8c, 0x6f, 0xef, - 0x41, 0x2d, 0xd2, 0x88, 0x49, 0x88, 0x82, 0x85, 0xe0, 0xc4, 0x8a, 0x18, 0x8d, 0xce, 0x13, 0xb0, - 0x1a, 0xa6, 0x14, 0x51, 0xff, 0x79, 0x0e, 0x2a, 0xb6, 0x75, 0xd4, 0x25, 0x66, 0xa6, 0x79, 0xef, - 0x37, 0x6d, 0xeb, 0x88, 0x18, 0x9f, 0x86, 0x96, 0x60, 0xce, 0x55, 0x9d, 0x43, 0x32, 0x7f, 0x29, - 0x70, 0x85, 0xae, 0xa1, 0x33, 0x50, 0xdc, 0xd7, 0x07, 0x98, 0xdd, 0xd8, 0x56, 0x14, 0x36, 0x40, - 0xbf, 0xf4, 0x1e, 0x1a, 0x95, 0x33, 0x3f, 0x93, 0x60, 0x6f, 0x8d, 0x7e, 0x92, 0x60, 0xde, 0x97, - 0x1a, 0x75, 0x40, 0xc4, 0xa7, 0x51, 0x7f, 0xf6, 0xc0, 0xd2, 0x98, 0xab, 0x68, 0xa4, 0xb8, 0x74, - 0x46, 0xc8, 0xbc, 0x96, 0x4f, 0x32, 0xae, 0xde, 0x22, 0xfb, 0x22, 0x9b, 0xd6, 0x35, 0xef, 0xe6, - 0xae, 0x64, 0x5b, 0x47, 0x1d, 0x4d, 0x48, 0x83, 0x3d, 0xce, 0x64, 0xd5, 0x05, 0x91, 0xc6, 0x03, - 0xfa, 0x3e, 0xf3, 0x2a, 0xd4, 0xb1, 0x6d, 0x5b, 0x76, 0xd7, 0xc0, 0x8e, 0xa3, 0xf6, 0x31, 0x4f, - 0x80, 0x6b, 0x14, 0xb8, 0xc5, 0x60, 0xf2, 0x8f, 0x79, 0x68, 0xf8, 0x5b, 0xf1, 0xee, 0xeb, 0x74, - 0xcd, 0xbb, 0xaf, 0xd3, 0xc9, 0xd1, 0x81, 0xcd, 0x5c, 0xa1, 0x38, 0xdc, 0xd5, 0x5c, 0x4b, 0x52, - 0x2a, 0x1c, 0xda, 0xd1, 0x48, 0x5c, 0x25, 0x46, 0x66, 0x5a, 0x1a, 0xf6, 0x0f, 0x17, 0x3c, 0x10, - 0x3f, 0xdb, 0x90, 0x8e, 0x14, 0x32, 0xe8, 0x48, 0x31, 0x83, 0x8e, 0x94, 0x12, 0x74, 0x64, 0x11, - 0x4a, 0x7b, 0xa3, 0xde, 0x21, 0x76, 0x79, 0xba, 0xca, 0x47, 0x61, 0xdd, 0x29, 0x47, 0x74, 0x47, - 0xa8, 0x48, 0x25, 0xa8, 0x22, 0xe7, 0xa0, 0xc2, 0x2e, 0x8e, 0xba, 0xae, 0x43, 0x3b, 0xe8, 0x79, - 0xa5, 0xcc, 0x00, 0xbb, 0x0e, 0xfa, 0xd0, 0xcb, 0xc7, 0xaa, 0x49, 0xc6, 0x4e, 0xbd, 0x4e, 0x44, - 0x4b, 0xbc, 0x6c, 0xec, 0x1a, 0x34, 0xe8, 0x33, 0xf8, 0xe7, 0x23, 0x6c, 0x1f, 0xab, 0x7b, 0x03, - 0xdc, 0xaa, 0x51, 0x76, 0xea, 0x04, 0xfa, 0xc4, 0x03, 0x12, 0x81, 0x50, 0x34, 0xdd, 0xd4, 0xf0, - 0x4b, 0xac, 0xb5, 0xea, 0x14, 0x89, 0x8a, 0xba, 0xc3, 0x40, 0xf2, 0x17, 0x80, 0xfc, 0x35, 0x66, - 0x4b, 0xca, 0x22, 0x87, 0x98, 0x8b, 0x1e, 0xa2, 0xfc, 0x37, 0x12, 0x2c, 0x04, 0x17, 0x9b, 0x36, - 0x3c, 0x7e, 0x0c, 0x55, 0x76, 0xd3, 0xd0, 0x25, 0xe6, 0xc9, 0x6b, 0xfe, 0x0b, 0x63, 0xa5, 0xa7, - 0x80, 0xff, 0x08, 0x9b, 0x28, 0xc1, 0x91, 0x65, 0x1f, 0xea, 0x66, 0xbf, 0x4b, 0x38, 0xf3, 0x8c, - 0xa2, 0xc6, 0x81, 0x8f, 0x09, 0x4c, 0xfe, 0x56, 0x82, 0x8b, 0x4f, 0x87, 0x9a, 0xea, 0xe2, 0x40, - 0x9e, 0x30, 0xeb, 0xbb, 0xae, 0x0f, 0xbc, 0x87, 0x55, 0xb9, 0x6c, 0xdd, 0x72, 0x86, 0x2d, 0x6f, - 0xc1, 0x59, 0x05, 0x3b, 0xd8, 0xd4, 0x42, 0x1f, 0xa7, 0xae, 0xf4, 0x87, 0xd0, 0x4e, 0x9a, 0x6e, - 0x96, 0xb3, 0x67, 0x09, 0x5b, 0xd7, 0x26, 0xd3, 0xba, 0xdc, 0xff, 0x90, 0x3c, 0x81, 0xae, 0xe3, - 0xca, 0xff, 0x25, 0xc1, 0xc2, 0x7d, 0xcd, 0x5b, 0xef, 0xb5, 0xe5, 0x85, 0xd1, 0xbc, 0x29, 0x1f, - 0xcf, 0x9b, 0x5e, 0x95, 0x23, 0xe1, 0x2e, 0xd5, 0x1c, 0x19, 0x5e, 0xa8, 0xb0, 0xe9, 0xbd, 0xbd, - 0xbc, 0x2f, 0x2e, 0x73, 0x15, 0xbc, 0x8f, 0x6d, 0x6c, 0xf6, 0xf0, 0xa6, 0xd5, 0x3b, 0x0c, 0xbc, - 0xce, 0x92, 0x82, 0xaf, 0xb3, 0xa6, 0x7d, 0xed, 0x75, 0xf3, 0xcf, 0x25, 0x58, 0x88, 0x75, 0x8d, - 0x50, 0x03, 0xe0, 0xa9, 0xd9, 0xe3, 0xed, 0xb4, 0xe6, 0x29, 0x54, 0x83, 0xb2, 0xd7, 0x5c, 0x6b, - 0x4a, 0xa8, 0x0a, 0x73, 0xbb, 0x16, 0xc5, 0x6e, 0xe6, 0x50, 0x13, 0x6a, 0x8c, 0x70, 0xd4, 0xeb, - 0x61, 0xc7, 0x69, 0xe6, 0x05, 0x64, 0x5d, 0xd5, 0x07, 0x23, 0x1b, 0x37, 0x0b, 0xa8, 0x0e, 0x95, - 0x5d, 0x8b, 0xbf, 0x6d, 0x6b, 0x16, 0x11, 0x82, 0x86, 0xf7, 0xd0, 0x8d, 0x13, 0x95, 0x02, 0x30, - 0x8f, 0x6c, 0xee, 0xe6, 0xb3, 0x60, 0x4f, 0x64, 0xf7, 0x78, 0x88, 0xd1, 0x12, 0x9c, 0x7e, 0x6a, - 0x6a, 0x78, 0x5f, 0x37, 0xb1, 0xe6, 0x7f, 0x6a, 0x9e, 0x42, 0xa7, 0x61, 0x7e, 0x0b, 0xdb, 0x7d, - 0x1c, 0x00, 0xe6, 0xd0, 0x02, 0xd4, 0xb7, 0xf4, 0x97, 0x01, 0x50, 0x5e, 0x2e, 0x94, 0xa5, 0xa6, - 0xb4, 0xf2, 0x3f, 0x4b, 0x50, 0x21, 0xf5, 0xcc, 0x03, 0xcb, 0xb2, 0x35, 0x34, 0x04, 0x44, 0x9f, - 0x82, 0x1a, 0x43, 0xcb, 0x14, 0x0f, 0xac, 0xd1, 0x7b, 0x29, 0x39, 0x53, 0x1c, 0x95, 0xeb, 0x61, - 0xfb, 0x7a, 0x0a, 0x45, 0x04, 0x5d, 0x3e, 0x85, 0x0c, 0xba, 0xe2, 0xae, 0x6e, 0xe0, 0x5d, 0xbd, - 0x77, 0xe8, 0xbd, 0x11, 0x19, 0xb3, 0x62, 0x04, 0xd5, 0x5b, 0x31, 0x52, 0x3d, 0xf3, 0x01, 0x7b, - 0xaf, 0xeb, 0x19, 0xa2, 0x7c, 0x0a, 0x3d, 0x87, 0x33, 0x1b, 0x38, 0xe0, 0x78, 0xbc, 0x05, 0x57, - 0xd2, 0x17, 0x8c, 0x21, 0x9f, 0x70, 0xc9, 0x4d, 0x28, 0xd2, 0x96, 0x2c, 0x4a, 0xf2, 0x4d, 0xc1, - 0xdf, 0x43, 0xb5, 0x2f, 0xa7, 0x23, 0x88, 0xd9, 0xbe, 0x80, 0xf9, 0xc8, 0xaf, 0x28, 0xd0, 0x8d, - 0x04, 0xb2, 0xe4, 0xdf, 0xc3, 0xb4, 0x6f, 0x66, 0x41, 0x15, 0x6b, 0xf5, 0xa1, 0x11, 0x7e, 0x46, - 0x8a, 0x92, 0x9a, 0x02, 0x89, 0x0f, 0xe0, 0xdb, 0x37, 0x32, 0x60, 0x8a, 0x85, 0x0c, 0x68, 0x46, - 0x5f, 0xf5, 0xa3, 0x9b, 0x63, 0x27, 0x08, 0xab, 0xdb, 0x2f, 0x32, 0xe1, 0x8a, 0xe5, 0x8e, 0xa9, - 0x12, 0xc4, 0x1e, 0x8a, 0xa3, 0xdb, 0xc9, 0xd3, 0xa4, 0xbd, 0x60, 0x6f, 0xdf, 0xc9, 0x8c, 0x2f, - 0x96, 0xfe, 0x5d, 0x76, 0x15, 0x94, 0xf4, 0xd8, 0x1a, 0xbd, 0x9f, 0x3c, 0xdd, 0x98, 0x57, 0xe2, - 0xed, 0x95, 0x93, 0x90, 0x08, 0x26, 0xbe, 0xa2, 0x77, 0x38, 0x09, 0xcf, 0x95, 0xa3, 0x76, 0xe7, - 0xcd, 0x97, 0xfe, 0x12, 0xbb, 0xfd, 0xfe, 0x09, 0x28, 0x04, 0x03, 0x56, 0xf4, 0x67, 0x13, 0x9e, - 0x19, 0xde, 0x99, 0xa8, 0x35, 0xd3, 0xd9, 0xe0, 0xe7, 0x30, 0x1f, 0x79, 0x8d, 0x93, 0x68, 0x35, - 0xc9, 0x2f, 0x76, 0xda, 0xe3, 0xe2, 0x35, 0x33, 0xc9, 0xc8, 0x95, 0x18, 0x4a, 0xd1, 0xfe, 0x84, - 0x6b, 0xb3, 0xf6, 0xcd, 0x2c, 0xa8, 0x62, 0x23, 0x0e, 0x75, 0x97, 0x91, 0x6b, 0x25, 0x74, 0x2b, - 0x79, 0x8e, 0xe4, 0x2b, 0xb1, 0xf6, 0xbb, 0x19, 0xb1, 0xc5, 0xa2, 0xbf, 0x03, 0x68, 0xe7, 0x80, - 0xd4, 0x30, 0xe6, 0xbe, 0xde, 0x1f, 0xd9, 0x2a, 0x7b, 0x72, 0x93, 0xe6, 0xa3, 0xe3, 0xa8, 0x29, - 0xba, 0x32, 0x96, 0x42, 0x2c, 0xde, 0x05, 0xd8, 0xc0, 0xee, 0x16, 0x76, 0x6d, 0xa2, 0xa0, 0xd7, - 0x13, 0xcf, 0xdb, 0x47, 0xf0, 0x96, 0x7a, 0x67, 0x22, 0x5e, 0x20, 0x24, 0x34, 0xb7, 0x54, 0x93, - 0x94, 0xef, 0xfe, 0x33, 0xb4, 0x5b, 0x89, 0xe4, 0x51, 0xb4, 0x14, 0x81, 0xa6, 0x62, 0x8b, 0x25, - 0x8f, 0x44, 0x98, 0x0d, 0x74, 0x53, 0xa3, 0xee, 0xc7, 0xe7, 0x39, 0xf9, 0x42, 0x2a, 0xea, 0x7e, - 0xc6, 0xe0, 0x8b, 0x85, 0xbf, 0x96, 0xe8, 0x6f, 0x72, 0x22, 0x08, 0xcf, 0x74, 0xf7, 0x60, 0x7b, - 0xa0, 0x9a, 0x4e, 0x16, 0x16, 0x28, 0xe2, 0x09, 0x58, 0xe0, 0xf8, 0x82, 0x05, 0x0d, 0xea, 0xa1, - 0x1e, 0x29, 0x4a, 0x7a, 0xf3, 0x95, 0xd4, 0xf0, 0x6d, 0x2f, 0x4f, 0x46, 0x14, 0xab, 0x1c, 0x40, - 0xdd, 0x53, 0x69, 0x26, 0xdc, 0x1b, 0x69, 0x9c, 0xfa, 0x38, 0x29, 0x16, 0x99, 0x8c, 0x1a, 0xb4, - 0xc8, 0x78, 0x0b, 0x08, 0x65, 0x6b, 0x1d, 0x8e, 0xb3, 0xc8, 0xf4, 0xbe, 0x12, 0x73, 0x39, 0x91, - 0x76, 0x6b, 0xb2, 0x3f, 0x4b, 0xec, 0x1e, 0x27, 0xba, 0x9c, 0x94, 0xee, 0xad, 0x7c, 0x0a, 0x3d, - 0x83, 0x12, 0xff, 0x31, 0xee, 0xdb, 0xe3, 0x0b, 0x42, 0x3e, 0xfb, 0xb5, 0x09, 0x58, 0x62, 0xe2, - 0x43, 0x58, 0x4a, 0x29, 0x07, 0x13, 0x43, 0xe1, 0xf8, 0xd2, 0x71, 0x92, 0x93, 0x56, 0x01, 0xc5, - 0x7f, 0xf1, 0x92, 0x78, 0x4c, 0xa9, 0x3f, 0x8c, 0xc9, 0xb0, 0x44, 0xfc, 0x47, 0x2b, 0x89, 0x4b, - 0xa4, 0xfe, 0xb6, 0x65, 0xd2, 0x12, 0x4f, 0x00, 0xfc, 0xa2, 0x2f, 0xf1, 0x3c, 0x62, 0x35, 0xe1, - 0x84, 0x29, 0x57, 0xfe, 0xbd, 0x0c, 0x65, 0xef, 0x85, 0xd5, 0x1b, 0xc8, 0xff, 0xdf, 0x40, 0x42, - 0xfe, 0x39, 0xcc, 0x47, 0x7e, 0xaa, 0x91, 0x68, 0x3c, 0xc9, 0x3f, 0xe7, 0x98, 0x74, 0x42, 0xcf, - 0xf8, 0x3f, 0x12, 0x10, 0xb1, 0xf9, 0x9d, 0xb4, 0xa4, 0x3e, 0x1a, 0x96, 0x27, 0x4c, 0xfc, 0xff, - 0x3b, 0x08, 0x3f, 0x06, 0x08, 0x84, 0xdf, 0xf1, 0xf7, 0xe4, 0x24, 0xa2, 0x4c, 0x92, 0x96, 0x91, - 0x18, 0x61, 0x6f, 0x64, 0xb9, 0xd3, 0x4c, 0xf7, 0x91, 0xe9, 0x71, 0x75, 0xeb, 0x84, 0x3e, 0x72, - 0x02, 0xf7, 0x0e, 0xf1, 0x24, 0xd1, 0x76, 0x52, 0x8a, 0x27, 0x49, 0x69, 0x62, 0x25, 0xc6, 0x94, - 0xf4, 0x1e, 0xd5, 0x6b, 0xf1, 0x2d, 0xab, 0x77, 0x7f, 0xeb, 0xfd, 0xbe, 0xee, 0x1e, 0x8c, 0xf6, - 0xc8, 0x97, 0x3b, 0x0c, 0xf5, 0x5d, 0xdd, 0xe2, 0x7f, 0xdd, 0xf1, 0x74, 0xef, 0x0e, 0xa5, 0xbe, - 0x43, 0xd6, 0x18, 0xee, 0xed, 0x95, 0xe8, 0xe8, 0xee, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x48, - 0x17, 0x7f, 0x8d, 0x97, 0x44, 0x00, 0x00, + // 3878 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x3c, 0x5b, 0x6f, 0x1b, 0x47, + 0x77, 0x5e, 0xde, 0x44, 0x1e, 0x5e, 0x44, 0x8d, 0x1d, 0x89, 0xa6, 0xef, 0xeb, 0xd8, 0x91, 0xfd, + 0x39, 0x76, 0x22, 0x37, 0xf8, 0x82, 0xfa, 0x4b, 0x02, 0xcb, 0xb2, 0x64, 0xb6, 0x92, 0x23, 0xaf, + 0xe4, 0x18, 0x68, 0x0a, 0x10, 0x2b, 0xee, 0x88, 0xda, 0x88, 0xbb, 0x4b, 0xef, 0x2e, 0x2d, 0x2b, + 0x7d, 0x48, 0xd0, 0x00, 0x05, 0x52, 0x14, 0x4d, 0x2f, 0x28, 0xd0, 0x3e, 0x14, 0x28, 0xfa, 0x94, + 0x16, 0x28, 0x50, 0x20, 0xe8, 0x43, 0x5b, 0xe4, 0x3d, 0x68, 0x1f, 0x8a, 0x3e, 0xf6, 0x0f, 0xb4, + 0xfd, 0x15, 0x45, 0x31, 0x97, 0x9d, 0xbd, 0x93, 0x2b, 0xd2, 0x8e, 0x8b, 0xbe, 0x69, 0xce, 0x9e, + 0x73, 0xe6, 0xcc, 0xcc, 0xb9, 0xcf, 0x50, 0xd0, 0xd4, 0x54, 0x57, 0xed, 0xf6, 0x2c, 0xcb, 0xd6, + 0x6e, 0x0f, 0x6d, 0xcb, 0xb5, 0xd0, 0x82, 0xa1, 0x0f, 0x5e, 0x8c, 0x1c, 0x36, 0xba, 0x4d, 0x3e, + 0xb7, 0x6b, 0x3d, 0xcb, 0x30, 0x2c, 0x93, 0x81, 0xda, 0x0d, 0xdd, 0x74, 0xb1, 0x6d, 0xaa, 0x03, + 0x3e, 0xae, 0x05, 0x09, 0xda, 0x35, 0xa7, 0x77, 0x80, 0x0d, 0x95, 0x8d, 0xe4, 0x39, 0x28, 0x3e, + 0x34, 0x86, 0xee, 0xb1, 0xfc, 0xe7, 0x12, 0xd4, 0xd6, 0x07, 0x23, 0xe7, 0x40, 0xc1, 0xcf, 0x47, + 0xd8, 0x71, 0xd1, 0x7b, 0x50, 0xd8, 0x53, 0x1d, 0xdc, 0x92, 0x2e, 0x4b, 0xcb, 0xd5, 0x95, 0xf3, + 0xb7, 0x43, 0xb3, 0xf2, 0xf9, 0xb6, 0x9c, 0xfe, 0xaa, 0xea, 0x60, 0x85, 0x62, 0x22, 0x04, 0x05, + 0x6d, 0xaf, 0xb3, 0xd6, 0xca, 0x5d, 0x96, 0x96, 0xf3, 0x0a, 0xfd, 0x1b, 0x5d, 0x04, 0x70, 0x70, + 0xdf, 0xc0, 0xa6, 0xdb, 0x59, 0x73, 0x5a, 0xf9, 0xcb, 0xf9, 0xe5, 0xbc, 0x12, 0x80, 0x20, 0x19, + 0x6a, 0x3d, 0x6b, 0x30, 0xc0, 0x3d, 0x57, 0xb7, 0xcc, 0xce, 0x5a, 0xab, 0x40, 0x69, 0x43, 0x30, + 0xf9, 0x3f, 0x25, 0xa8, 0x73, 0xd1, 0x9c, 0xa1, 0x65, 0x3a, 0x18, 0xdd, 0x85, 0x92, 0xe3, 0xaa, + 0xee, 0xc8, 0xe1, 0xd2, 0x9d, 0x4b, 0x94, 0x6e, 0x87, 0xa2, 0x28, 0x1c, 0x35, 0x51, 0xbc, 0xe8, + 0xf4, 0xf9, 0xf8, 0xf4, 0x91, 0x25, 0x14, 0x62, 0x4b, 0x58, 0x86, 0xf9, 0x7d, 0x22, 0xdd, 0x8e, + 0x8f, 0x54, 0xa4, 0x48, 0x51, 0x30, 0xe1, 0xe4, 0xea, 0x06, 0xfe, 0x74, 0x7f, 0x07, 0xab, 0x83, + 0x56, 0x89, 0xce, 0x15, 0x80, 0xc8, 0xff, 0x2e, 0x41, 0x53, 0xa0, 0x7b, 0xe7, 0x70, 0x06, 0x8a, + 0x3d, 0x6b, 0x64, 0xba, 0x74, 0xa9, 0x75, 0x85, 0x0d, 0xd0, 0x15, 0xa8, 0xf5, 0x0e, 0x54, 0xd3, + 0xc4, 0x83, 0xae, 0xa9, 0x1a, 0x98, 0x2e, 0xaa, 0xa2, 0x54, 0x39, 0xec, 0xb1, 0x6a, 0xe0, 0x4c, + 0x6b, 0xbb, 0x0c, 0xd5, 0xa1, 0x6a, 0xbb, 0x7a, 0x68, 0xf7, 0x83, 0x20, 0xd4, 0x86, 0xb2, 0xee, + 0x74, 0x8c, 0xa1, 0x65, 0xbb, 0xad, 0xe2, 0x65, 0x69, 0xb9, 0xac, 0x88, 0x31, 0x99, 0x41, 0xa7, + 0x7f, 0xed, 0xaa, 0xce, 0x61, 0x67, 0x8d, 0xaf, 0x28, 0x04, 0x93, 0xff, 0x4a, 0x82, 0xc5, 0xfb, + 0x8e, 0xa3, 0xf7, 0xcd, 0xd8, 0xca, 0x16, 0xa1, 0x64, 0x5a, 0x1a, 0xee, 0xac, 0xd1, 0xa5, 0xe5, + 0x15, 0x3e, 0x42, 0xe7, 0xa0, 0x32, 0xc4, 0xd8, 0xee, 0xda, 0xd6, 0xc0, 0x5b, 0x58, 0x99, 0x00, + 0x14, 0x6b, 0x80, 0xd1, 0x13, 0x58, 0x70, 0x22, 0x8c, 0x98, 0x5e, 0x55, 0x57, 0xae, 0xde, 0x8e, + 0x59, 0xc6, 0xed, 0xe8, 0xa4, 0x4a, 0x9c, 0x5a, 0xfe, 0x3a, 0x07, 0xa7, 0x05, 0x1e, 0x93, 0x95, + 0xfc, 0x4d, 0x76, 0xde, 0xc1, 0x7d, 0x21, 0x1e, 0x1b, 0x64, 0xd9, 0x79, 0x71, 0x64, 0xf9, 0xe0, + 0x91, 0x65, 0x50, 0xf5, 0xe8, 0x79, 0x14, 0xe3, 0xe7, 0x71, 0x09, 0xaa, 0xf8, 0xe5, 0x50, 0xb7, + 0x71, 0x97, 0x28, 0x0e, 0xdd, 0xf2, 0x82, 0x02, 0x0c, 0xb4, 0xab, 0x1b, 0x41, 0xdb, 0x98, 0xcb, + 0x6c, 0x1b, 0xf2, 0x5f, 0x4b, 0xb0, 0x14, 0x3b, 0x25, 0x6e, 0x6c, 0x0a, 0x34, 0xe9, 0xca, 0xfd, + 0x9d, 0x21, 0x66, 0x47, 0x36, 0xfc, 0xfa, 0xb8, 0x0d, 0xf7, 0xd1, 0x95, 0x18, 0x7d, 0x40, 0xc8, + 0x5c, 0x76, 0x21, 0x0f, 0x61, 0x69, 0x03, 0xbb, 0x7c, 0x02, 0xf2, 0x0d, 0x3b, 0xd3, 0x3b, 0xab, + 0xb0, 0x55, 0xe7, 0xa2, 0x56, 0x2d, 0xff, 0x7d, 0x4e, 0xd8, 0x22, 0x9d, 0xaa, 0x63, 0xee, 0x5b, + 0xe8, 0x3c, 0x54, 0x04, 0x0a, 0xd7, 0x0a, 0x1f, 0x80, 0x7e, 0x09, 0x45, 0x22, 0x29, 0x53, 0x89, + 0xc6, 0xca, 0x95, 0xe4, 0x35, 0x05, 0x78, 0x2a, 0x0c, 0x1f, 0x75, 0xa0, 0xe1, 0xb8, 0xaa, 0xed, + 0x76, 0x87, 0x96, 0x43, 0xcf, 0x99, 0x2a, 0x4e, 0x75, 0x45, 0x0e, 0x73, 0x10, 0x6e, 0x7d, 0xcb, + 0xe9, 0x6f, 0x73, 0x4c, 0xa5, 0x4e, 0x29, 0xbd, 0x21, 0x7a, 0x08, 0x35, 0x6c, 0x6a, 0x3e, 0xa3, + 0x42, 0x66, 0x46, 0x55, 0x6c, 0x6a, 0x82, 0x8d, 0x7f, 0x3e, 0xc5, 0xec, 0xe7, 0xf3, 0x07, 0x12, + 0xb4, 0xe2, 0x07, 0x34, 0x8b, 0xcb, 0xbe, 0xc7, 0x88, 0x30, 0x3b, 0xa0, 0xb1, 0x16, 0x2e, 0x0e, + 0x49, 0xe1, 0x24, 0xf2, 0x9f, 0x49, 0xf0, 0x96, 0x2f, 0x0e, 0xfd, 0xf4, 0xba, 0xb4, 0x05, 0xdd, + 0x84, 0xa6, 0x6e, 0xf6, 0x06, 0x23, 0x0d, 0x3f, 0x35, 0x1f, 0x61, 0x75, 0xe0, 0x1e, 0x1c, 0xd3, + 0x33, 0x2c, 0x2b, 0x31, 0xb8, 0xfc, 0x8d, 0x04, 0x8b, 0x51, 0xb9, 0x66, 0xd9, 0xa4, 0x5f, 0x83, + 0xa2, 0x6e, 0xee, 0x5b, 0xde, 0x1e, 0x5d, 0x1c, 0x63, 0x94, 0x64, 0x2e, 0x86, 0x2c, 0x1b, 0x70, + 0x6e, 0x03, 0xbb, 0x1d, 0xd3, 0xc1, 0xb6, 0xbb, 0xaa, 0x9b, 0x03, 0xab, 0xbf, 0xad, 0xba, 0x07, + 0x33, 0x18, 0x54, 0xc8, 0x36, 0x72, 0x11, 0xdb, 0x90, 0xbf, 0x97, 0xe0, 0x7c, 0xf2, 0x7c, 0x7c, + 0xe9, 0x6d, 0x28, 0xef, 0xeb, 0x78, 0xa0, 0x91, 0xfd, 0x95, 0xe8, 0xfe, 0x8a, 0x31, 0x31, 0xac, + 0x21, 0x41, 0xe6, 0x2b, 0xbc, 0x92, 0xa2, 0xcd, 0x3b, 0xae, 0xad, 0x9b, 0xfd, 0x4d, 0xdd, 0x71, + 0x15, 0x86, 0x1f, 0xd8, 0xcf, 0x7c, 0x76, 0x35, 0xfe, 0x7d, 0x09, 0x2e, 0x6e, 0x60, 0xf7, 0x81, + 0xf0, 0xcb, 0xe4, 0xbb, 0xee, 0xb8, 0x7a, 0xcf, 0x79, 0xb5, 0xb9, 0x51, 0x86, 0x00, 0x2d, 0x7f, + 0x27, 0xc1, 0xa5, 0x54, 0x61, 0xf8, 0xd6, 0x71, 0xbf, 0xe3, 0x79, 0xe5, 0x64, 0xbf, 0xf3, 0x9b, + 0xf8, 0xf8, 0x33, 0x75, 0x30, 0xc2, 0xdb, 0xaa, 0x6e, 0x33, 0xbf, 0x33, 0xa5, 0x17, 0xfe, 0x3b, + 0x09, 0x2e, 0x6c, 0x60, 0x77, 0xdb, 0x8b, 0x49, 0x6f, 0x70, 0x77, 0x08, 0x4e, 0x20, 0x36, 0x7a, + 0xc9, 0x59, 0x08, 0x26, 0xff, 0x21, 0x3b, 0xce, 0x44, 0x79, 0xdf, 0xc8, 0x06, 0x5e, 0xa4, 0x96, + 0x10, 0x30, 0xc9, 0x07, 0x2c, 0x75, 0xe0, 0xdb, 0x27, 0xff, 0xa5, 0x04, 0x67, 0xef, 0xf7, 0x9e, + 0x8f, 0x74, 0x1b, 0x73, 0xa4, 0x4d, 0xab, 0x77, 0x38, 0xfd, 0xe6, 0xfa, 0x69, 0x56, 0x2e, 0x94, + 0x66, 0x4d, 0x4a, 0xcd, 0x17, 0xa1, 0xe4, 0xb2, 0xbc, 0x8e, 0x65, 0x2a, 0x7c, 0x44, 0xe5, 0x53, + 0xf0, 0x00, 0xab, 0xce, 0xff, 0x4d, 0xf9, 0xbe, 0x2b, 0x40, 0xed, 0x33, 0x9e, 0x8e, 0xd1, 0xa8, + 0x1d, 0xd5, 0x24, 0x29, 0x39, 0xf1, 0x0a, 0x64, 0x70, 0x49, 0x49, 0xdd, 0x06, 0xd4, 0x1d, 0x8c, + 0x0f, 0xa7, 0x89, 0xd1, 0x35, 0x42, 0x28, 0x62, 0xeb, 0x26, 0x2c, 0x8c, 0x4c, 0x5a, 0x1a, 0x60, + 0x8d, 0x6f, 0x20, 0xd3, 0xdc, 0xc9, 0xbe, 0x3b, 0x4e, 0x88, 0x1e, 0xf1, 0xea, 0x23, 0xc0, 0xab, + 0x98, 0x89, 0x57, 0x94, 0x0c, 0x75, 0xa0, 0xa9, 0xd9, 0xd6, 0x70, 0x88, 0xb5, 0xae, 0xe3, 0xb1, + 0x2a, 0x65, 0x63, 0xc5, 0xe9, 0x04, 0xab, 0xf7, 0xe0, 0x74, 0x54, 0xd2, 0x8e, 0x46, 0x12, 0x52, + 0x72, 0x86, 0x49, 0x9f, 0xd0, 0x2d, 0x58, 0x88, 0xe3, 0x97, 0x29, 0x7e, 0xfc, 0x03, 0x7a, 0x17, + 0x50, 0x44, 0x54, 0x82, 0x5e, 0x61, 0xe8, 0x61, 0x61, 0x3a, 0x9a, 0x23, 0x7f, 0x2b, 0xc1, 0xe2, + 0x33, 0xd5, 0xed, 0x1d, 0xac, 0x19, 0xdc, 0xd6, 0x66, 0xf0, 0x55, 0x1f, 0x41, 0xe5, 0x05, 0xd7, + 0x0b, 0x2f, 0x20, 0x5d, 0x4a, 0xd8, 0x9f, 0xa0, 0x06, 0x2a, 0x3e, 0x85, 0xfc, 0x93, 0x04, 0x67, + 0xd6, 0x03, 0x75, 0xe1, 0x1b, 0xf0, 0x9a, 0x93, 0x0a, 0xda, 0xeb, 0xd0, 0x30, 0x54, 0xfb, 0x30, + 0x56, 0xcf, 0x46, 0xa0, 0xf2, 0x4b, 0x00, 0x3e, 0xda, 0x72, 0xfa, 0x53, 0xc8, 0xff, 0x21, 0xcc, + 0xf1, 0x59, 0xb9, 0xfb, 0x9c, 0xa4, 0x67, 0x1e, 0xba, 0xfc, 0x2f, 0x12, 0x34, 0xfc, 0x90, 0x48, + 0x8d, 0xbc, 0x01, 0x39, 0x61, 0xda, 0xb9, 0xce, 0x1a, 0xfa, 0x08, 0x4a, 0xac, 0xd1, 0xc1, 0x79, + 0x5f, 0x0b, 0xf3, 0xe6, 0x4d, 0x90, 0x40, 0x5c, 0xa5, 0x00, 0x85, 0x13, 0x91, 0x3d, 0x12, 0x51, + 0x44, 0x38, 0x1f, 0x1f, 0x82, 0x3a, 0x30, 0x1f, 0x4e, 0xd9, 0x3d, 0x13, 0xbe, 0x9c, 0x16, 0x3c, + 0xd6, 0x54, 0x57, 0xa5, 0xb1, 0xa3, 0x11, 0xca, 0xd8, 0x1d, 0xf9, 0x8f, 0x4b, 0x50, 0x0d, 0xac, + 0x32, 0xb6, 0x92, 0xe8, 0x91, 0xe6, 0x26, 0xd7, 0x8d, 0xf9, 0x78, 0xdd, 0x78, 0x0d, 0x1a, 0x3a, + 0x4d, 0xbe, 0xba, 0x5c, 0x15, 0xa9, 0xd7, 0xac, 0x28, 0x75, 0x06, 0xe5, 0x76, 0x81, 0x2e, 0x42, + 0xd5, 0x1c, 0x19, 0x5d, 0x6b, 0xbf, 0x6b, 0x5b, 0x47, 0x0e, 0x2f, 0x40, 0x2b, 0xe6, 0xc8, 0xf8, + 0x74, 0x5f, 0xb1, 0x8e, 0x1c, 0xbf, 0xc6, 0x29, 0x9d, 0xb0, 0xc6, 0xb9, 0x08, 0x55, 0x43, 0x7d, + 0x49, 0xb8, 0x76, 0xcd, 0x91, 0x41, 0x6b, 0xd3, 0xbc, 0x52, 0x31, 0xd4, 0x97, 0x8a, 0x75, 0xf4, + 0x78, 0x64, 0xa0, 0x65, 0x68, 0x0e, 0x54, 0xc7, 0xed, 0x06, 0x8b, 0xdb, 0x32, 0x2d, 0x6e, 0x1b, + 0x04, 0xfe, 0xd0, 0x2f, 0x70, 0xe3, 0xd5, 0x52, 0x65, 0x86, 0x6a, 0x49, 0x33, 0x06, 0x3e, 0x23, + 0xc8, 0x5e, 0x2d, 0x69, 0xc6, 0x40, 0xb0, 0xf9, 0x10, 0xe6, 0xf6, 0x68, 0x4a, 0xeb, 0xb4, 0xaa, + 0xa9, 0x0e, 0x73, 0x9d, 0x64, 0xb3, 0x2c, 0xf3, 0x55, 0x3c, 0x74, 0xf4, 0x2b, 0xa8, 0xd0, 0x4c, + 0x82, 0xd2, 0xd6, 0x32, 0xd1, 0xfa, 0x04, 0x84, 0x5a, 0xc3, 0x03, 0x57, 0xa5, 0xd4, 0xf5, 0x6c, + 0xd4, 0x82, 0x80, 0x38, 0xe9, 0x9e, 0x8d, 0x55, 0x17, 0x6b, 0xab, 0xc7, 0x0f, 0x2c, 0x63, 0xa8, + 0x52, 0x65, 0x6a, 0x35, 0x68, 0xd9, 0x92, 0xf4, 0x89, 0x38, 0x86, 0x9e, 0x18, 0xad, 0xdb, 0x96, + 0xd1, 0x9a, 0x67, 0x8e, 0x21, 0x0c, 0x45, 0x17, 0x00, 0x3c, 0xf7, 0xac, 0xba, 0xad, 0x26, 0x3d, + 0xc5, 0x0a, 0x87, 0xdc, 0xa7, 0xbd, 0x2b, 0xdd, 0xe9, 0xb2, 0x2e, 0x91, 0x6e, 0xf6, 0x5b, 0x0b, + 0x74, 0xc6, 0xaa, 0xd7, 0x56, 0xd2, 0xcd, 0xbe, 0xfc, 0x15, 0x9c, 0xf1, 0x95, 0x28, 0x70, 0x60, + 0xf1, 0xb3, 0x97, 0xa6, 0x3d, 0xfb, 0xf1, 0xf5, 0xca, 0xbf, 0x15, 0x60, 0x71, 0x47, 0x7d, 0x81, + 0x5f, 0x7f, 0x69, 0x94, 0xc9, 0x65, 0x6f, 0xc2, 0x02, 0xad, 0x86, 0x56, 0x02, 0xf2, 0x8c, 0xc9, + 0x19, 0x82, 0x27, 0x1e, 0x27, 0x44, 0x9f, 0x90, 0x64, 0x07, 0xf7, 0x0e, 0xb7, 0x2d, 0xdd, 0xcf, + 0x17, 0x2e, 0x24, 0xf0, 0x79, 0x20, 0xb0, 0x94, 0x20, 0x05, 0xda, 0x8e, 0x7b, 0x3f, 0x96, 0x29, + 0xbc, 0x33, 0xb6, 0x40, 0xf7, 0x77, 0x3f, 0xea, 0x04, 0x51, 0x0b, 0xe6, 0x78, 0x98, 0xa7, 0xae, + 0xa1, 0xac, 0x78, 0x43, 0xb4, 0x0d, 0xa7, 0xd9, 0x0a, 0x76, 0xb8, 0xde, 0xb3, 0xc5, 0x97, 0x33, + 0x2d, 0x3e, 0x89, 0x34, 0x6c, 0x36, 0x95, 0x93, 0x9a, 0x4d, 0x0b, 0xe6, 0xb8, 0x2a, 0x53, 0x77, + 0x51, 0x56, 0xbc, 0x21, 0x39, 0x66, 0x5f, 0xa9, 0xab, 0xf4, 0x9b, 0x0f, 0x20, 0x65, 0x25, 0xf8, + 0xfb, 0x39, 0xa1, 0x95, 0xf4, 0x31, 0x94, 0x85, 0x86, 0xe7, 0x32, 0x6b, 0xb8, 0xa0, 0x89, 0xba, + 0xf1, 0x7c, 0xc4, 0x8d, 0xcb, 0xff, 0x2a, 0x41, 0x6d, 0x8d, 0x2c, 0x69, 0xd3, 0xea, 0xd3, 0xa0, + 0x73, 0x0d, 0x1a, 0x36, 0xee, 0x59, 0xb6, 0xd6, 0xc5, 0xa6, 0x6b, 0xeb, 0x98, 0x75, 0x20, 0x0a, + 0x4a, 0x9d, 0x41, 0x1f, 0x32, 0x20, 0x41, 0x23, 0x9e, 0xd9, 0x71, 0x55, 0x63, 0xd8, 0xdd, 0x27, + 0x1e, 0x20, 0xc7, 0xd0, 0x04, 0x94, 0x3a, 0x80, 0x2b, 0x50, 0xf3, 0xd1, 0x5c, 0x8b, 0xce, 0x5f, + 0x50, 0xaa, 0x02, 0xb6, 0x6b, 0xa1, 0xb7, 0xa1, 0x41, 0xf7, 0xb4, 0x3b, 0xb0, 0xfa, 0x5d, 0x52, + 0xad, 0xf3, 0x78, 0x54, 0xd3, 0xb8, 0x58, 0xe4, 0xac, 0xc2, 0x58, 0x8e, 0xfe, 0x25, 0xe6, 0x11, + 0x49, 0x60, 0xed, 0xe8, 0x5f, 0x62, 0x92, 0x0e, 0xd4, 0x49, 0x78, 0x7d, 0x6c, 0x69, 0x78, 0x77, + 0xca, 0x64, 0x24, 0x43, 0x5b, 0xf7, 0x3c, 0x54, 0xc4, 0x0a, 0xf8, 0x92, 0x7c, 0x00, 0x5a, 0x87, + 0x86, 0x97, 0x36, 0x77, 0x59, 0x35, 0x59, 0x48, 0x4d, 0x0e, 0x03, 0x01, 0xd2, 0x51, 0xea, 0x1e, + 0x19, 0x1d, 0xca, 0xeb, 0x50, 0x0b, 0x7e, 0x26, 0xb3, 0xee, 0x44, 0x15, 0x45, 0x00, 0x88, 0x36, + 0x3e, 0x1e, 0x19, 0xe4, 0x4c, 0xb9, 0x63, 0xf1, 0x86, 0xf2, 0x37, 0x12, 0xd4, 0x79, 0x54, 0xdf, + 0x11, 0x17, 0x20, 0x74, 0x69, 0x12, 0x5d, 0x1a, 0xfd, 0x1b, 0xfd, 0x7a, 0xb8, 0x67, 0xf9, 0x76, + 0xa2, 0x13, 0xa0, 0x4c, 0x68, 0x02, 0x1d, 0x0a, 0xe9, 0x59, 0xfa, 0x17, 0x5f, 0x13, 0x45, 0xe3, + 0x47, 0x43, 0x15, 0xad, 0x05, 0x73, 0xaa, 0xa6, 0xd9, 0xd8, 0x71, 0xb8, 0x1c, 0xde, 0x90, 0x7c, + 0x79, 0x81, 0x6d, 0xc7, 0x53, 0xf9, 0xbc, 0xe2, 0x0d, 0xd1, 0xaf, 0xa0, 0x2c, 0x32, 0xee, 0x7c, + 0x52, 0x96, 0x15, 0x94, 0x93, 0x57, 0xdb, 0x82, 0x42, 0xfe, 0x87, 0x1c, 0x34, 0xf8, 0x86, 0xad, + 0xf2, 0xb0, 0x3b, 0xde, 0xf8, 0x56, 0xa1, 0xb6, 0xef, 0xdb, 0xfe, 0xb8, 0xbe, 0x5a, 0xd0, 0x45, + 0x84, 0x68, 0x26, 0x19, 0x60, 0x38, 0xf0, 0x17, 0x66, 0x0a, 0xfc, 0xc5, 0x93, 0x7a, 0xb0, 0x78, + 0x2a, 0x58, 0x4a, 0x48, 0x05, 0xe5, 0xdf, 0x86, 0x6a, 0x80, 0x01, 0xf5, 0xd0, 0xac, 0x21, 0xc7, + 0x77, 0xcc, 0x1b, 0xa2, 0xbb, 0x7e, 0xfa, 0xc3, 0xb6, 0xea, 0x6c, 0x82, 0x2c, 0x91, 0xcc, 0x47, + 0xfe, 0x1b, 0x09, 0x4a, 0x9c, 0xf3, 0x25, 0xa8, 0x72, 0xa7, 0x43, 0x53, 0x43, 0xc6, 0x1d, 0x38, + 0x88, 0xe4, 0x86, 0xaf, 0xce, 0xeb, 0x9c, 0x85, 0x72, 0xc4, 0xdf, 0xcc, 0xf1, 0xb0, 0xe0, 0x7d, + 0x0a, 0x38, 0x19, 0xf2, 0x89, 0xfa, 0x97, 0x9f, 0x24, 0x7a, 0xf3, 0xa0, 0xe0, 0x9e, 0xf5, 0x02, + 0xdb, 0xc7, 0xb3, 0xb7, 0x6c, 0xef, 0x05, 0x14, 0x3a, 0x63, 0x09, 0x29, 0x08, 0xd0, 0x3d, 0x7f, + 0xbb, 0xf3, 0x49, 0xfd, 0xaa, 0xa0, 0x87, 0xe1, 0xea, 0xe8, 0x6f, 0xfb, 0x1f, 0xb1, 0xe6, 0x73, + 0x78, 0x29, 0xd3, 0xe6, 0x35, 0xaf, 0xa4, 0x32, 0x91, 0xff, 0x54, 0x82, 0xb3, 0x1b, 0xd8, 0x5d, + 0x0f, 0xb7, 0x23, 0xde, 0xb4, 0x54, 0x06, 0xb4, 0x93, 0x84, 0x9a, 0xe5, 0xd4, 0xdb, 0x50, 0x16, + 0x8d, 0x15, 0x76, 0x85, 0x20, 0xc6, 0xf2, 0xef, 0x49, 0xd0, 0xe2, 0xb3, 0xd0, 0x39, 0x49, 0xd6, + 0x3d, 0xc0, 0x2e, 0xd6, 0x7e, 0xee, 0xd2, 0xfa, 0x47, 0x09, 0x9a, 0x41, 0x8f, 0x4f, 0x9d, 0xf6, + 0x07, 0x50, 0xa4, 0x1d, 0x0c, 0x2e, 0xc1, 0x44, 0x65, 0x65, 0xd8, 0xc4, 0x65, 0xd0, 0x34, 0x6f, + 0x57, 0x04, 0x27, 0x3e, 0xf4, 0xc3, 0x4e, 0xfe, 0xe4, 0x61, 0x87, 0x87, 0x61, 0x6b, 0x44, 0xf8, + 0xb2, 0xd6, 0x9f, 0x0f, 0x90, 0x7f, 0x03, 0x16, 0xfd, 0x8a, 0x85, 0xd1, 0x4d, 0xab, 0x49, 0xf2, + 0x0f, 0x39, 0x68, 0x05, 0x98, 0xfd, 0xdc, 0x31, 0x24, 0x25, 0xf3, 0xcd, 0xbf, 0xa2, 0xcc, 0xb7, + 0x30, 0x7b, 0xdc, 0x28, 0x26, 0xc5, 0x8d, 0x7f, 0xce, 0x41, 0xc3, 0xdf, 0xb5, 0xed, 0x81, 0x6a, + 0xa2, 0x45, 0x28, 0x0d, 0x07, 0xaa, 0xdf, 0x7b, 0xe5, 0x23, 0xb4, 0x23, 0x72, 0xa6, 0xf0, 0x3e, + 0xfd, 0x22, 0x49, 0x1f, 0x52, 0x0e, 0x42, 0x89, 0xb0, 0x20, 0xd5, 0x27, 0x2b, 0x4e, 0x68, 0x0f, + 0x81, 0xe7, 0x69, 0x4c, 0xf1, 0x74, 0x03, 0xa3, 0x5b, 0x80, 0xb8, 0xb6, 0x74, 0x75, 0xb3, 0xeb, + 0xe0, 0x9e, 0x65, 0x6a, 0x4c, 0x8f, 0x8a, 0x4a, 0x93, 0x7f, 0xe9, 0x98, 0x3b, 0x0c, 0x8e, 0x3e, + 0x80, 0x82, 0x7b, 0x3c, 0x64, 0x11, 0xa1, 0x91, 0xe8, 0x69, 0x7d, 0xb9, 0x76, 0x8f, 0x87, 0x58, + 0xa1, 0xe8, 0xde, 0x4b, 0x0f, 0xd7, 0x56, 0x5f, 0xf0, 0xf0, 0x5a, 0x50, 0x02, 0x10, 0x62, 0x19, + 0xde, 0x1e, 0xce, 0xb1, 0x30, 0xc4, 0x87, 0xf2, 0x3f, 0xe6, 0xa0, 0xe9, 0xb3, 0x54, 0xb0, 0x33, + 0x1a, 0xb8, 0xa9, 0xfb, 0x37, 0xbe, 0xb0, 0x9c, 0x94, 0x83, 0x7c, 0x02, 0x55, 0x7e, 0x9e, 0x27, + 0xd0, 0x07, 0x60, 0x24, 0x9b, 0x63, 0x14, 0xb4, 0xf8, 0x8a, 0x14, 0xb4, 0x74, 0x42, 0x05, 0x95, + 0xbf, 0x97, 0xe0, 0xad, 0x98, 0xf1, 0x8f, 0xdd, 0xc0, 0xf1, 0xe9, 0x2f, 0x77, 0x0a, 0x51, 0x96, + 0xdc, 0x0f, 0xdd, 0x83, 0x92, 0x4d, 0xb9, 0xf3, 0x9b, 0x80, 0xab, 0x63, 0x95, 0x83, 0x09, 0xa2, + 0x70, 0x12, 0xf9, 0x4f, 0x24, 0x58, 0x8a, 0x8b, 0x3a, 0x43, 0x70, 0x59, 0x85, 0x39, 0xc6, 0xda, + 0xb3, 0xa1, 0xe5, 0xf1, 0x36, 0xe4, 0x6f, 0x8e, 0xe2, 0x11, 0xca, 0x3b, 0xb0, 0xe8, 0xc5, 0x20, + 0x7f, 0x83, 0xb7, 0xb0, 0xab, 0x8e, 0x49, 0xfe, 0x2e, 0x41, 0x95, 0xe5, 0x16, 0x2c, 0xa9, 0x62, + 0x65, 0x13, 0xec, 0x89, 0x6e, 0x03, 0x49, 0xf4, 0xce, 0x50, 0x27, 0x1e, 0x6d, 0xbd, 0x67, 0xb9, + 0x96, 0x91, 0x45, 0x55, 0x46, 0x2a, 0x30, 0xb6, 0xb4, 0x8a, 0x12, 0x82, 0x25, 0xb5, 0x62, 0xf3, + 0x53, 0xb6, 0x62, 0x37, 0xe1, 0xad, 0x88, 0xa8, 0x33, 0x1c, 0x09, 0x59, 0xf9, 0xe2, 0x4e, 0xf8, + 0x3d, 0xc4, 0xf4, 0x59, 0xcd, 0x05, 0xd1, 0xb4, 0xef, 0xea, 0x5a, 0xd4, 0xd6, 0x35, 0xf4, 0x31, + 0x54, 0x4c, 0x7c, 0xd4, 0x0d, 0x06, 0xd5, 0x0c, 0xbd, 0xd9, 0xb2, 0x89, 0x8f, 0xe8, 0x5f, 0xf2, + 0x63, 0x58, 0x8a, 0x89, 0x3a, 0xcb, 0xda, 0xff, 0x49, 0x82, 0xb3, 0x6b, 0xb6, 0x35, 0xfc, 0x4c, + 0xb7, 0xdd, 0x91, 0x3a, 0x08, 0x5f, 0x71, 0xbe, 0x9e, 0xf2, 0xfc, 0x51, 0x20, 0xbd, 0x62, 0x0a, + 0x70, 0x2b, 0xc1, 0x04, 0xe2, 0x42, 0xf1, 0x45, 0x07, 0x92, 0xb1, 0xff, 0xca, 0x27, 0x09, 0xcf, + 0xf1, 0x26, 0x04, 0xfe, 0x2c, 0xd9, 0x67, 0x62, 0x37, 0x2f, 0x3f, 0x6d, 0x37, 0x2f, 0xc5, 0x0b, + 0x17, 0x5e, 0x91, 0x17, 0x3e, 0x71, 0x79, 0xf9, 0x08, 0xc2, 0x9d, 0x56, 0x1a, 0xfe, 0xa6, 0x6a, + 0xd1, 0xae, 0x02, 0xf8, 0x5d, 0x47, 0xfe, 0x9c, 0x2d, 0x0b, 0x9b, 0x00, 0x15, 0x39, 0x2d, 0x11, + 0xf1, 0xe8, 0x85, 0x42, 0xa8, 0x0f, 0xf6, 0x04, 0xda, 0x49, 0x5a, 0x3a, 0x8b, 0xe6, 0xff, 0x90, + 0x03, 0xe8, 0x88, 0x17, 0x90, 0xd3, 0x39, 0xf3, 0xab, 0x50, 0xf7, 0x15, 0xc6, 0xb7, 0xf7, 0xa0, + 0x16, 0x69, 0xc4, 0x24, 0x44, 0xc1, 0x42, 0x70, 0x62, 0x45, 0x8c, 0x46, 0xf9, 0x04, 0xac, 0x86, + 0x29, 0x45, 0xd4, 0x7f, 0x9e, 0x83, 0x8a, 0x6d, 0x1d, 0x75, 0x89, 0x99, 0x69, 0xde, 0x13, 0x4f, + 0xdb, 0x3a, 0x22, 0xc6, 0xa7, 0xa1, 0x25, 0x98, 0x73, 0x55, 0xe7, 0x90, 0xf0, 0x2f, 0x05, 0x6e, + 0xd9, 0x35, 0x74, 0x06, 0x8a, 0xfb, 0xfa, 0x00, 0xb3, 0x4b, 0xdd, 0x8a, 0xc2, 0x06, 0xe8, 0x97, + 0xde, 0x5b, 0xa4, 0x72, 0xe6, 0x97, 0x14, 0xec, 0x39, 0xd2, 0x4f, 0x12, 0xcc, 0xfb, 0xbb, 0x46, + 0x1d, 0x10, 0xf1, 0x69, 0xd4, 0x9f, 0x3d, 0xb0, 0x34, 0xe6, 0x2a, 0x1a, 0x29, 0x2e, 0x9d, 0x11, + 0x32, 0xaf, 0xe5, 0x93, 0x8c, 0xab, 0xb7, 0xc8, 0xba, 0xc8, 0xa2, 0x75, 0xcd, 0xbb, 0xdc, 0x2b, + 0xd9, 0xd6, 0x51, 0x47, 0x13, 0xbb, 0xc1, 0xde, 0x6f, 0xb2, 0xea, 0x82, 0xec, 0xc6, 0x03, 0xfa, + 0x84, 0xf3, 0x2a, 0xd4, 0xb1, 0x6d, 0x5b, 0x76, 0xd7, 0xc0, 0x8e, 0xa3, 0xf6, 0x31, 0x4f, 0x80, + 0x6b, 0x14, 0xb8, 0xc5, 0x60, 0xf2, 0x8f, 0x79, 0x68, 0xf8, 0x4b, 0xf1, 0xae, 0xf4, 0x74, 0xcd, + 0xbb, 0xd2, 0xd3, 0xc9, 0xd1, 0x81, 0xcd, 0x5c, 0xa1, 0x38, 0xdc, 0xd5, 0x5c, 0x4b, 0x52, 0x2a, + 0x1c, 0xda, 0xd1, 0x48, 0x5c, 0x25, 0x46, 0x66, 0x5a, 0x1a, 0xf6, 0x0f, 0x17, 0x3c, 0x10, 0x3f, + 0xdb, 0x90, 0x8e, 0x14, 0x32, 0xe8, 0x48, 0x31, 0x83, 0x8e, 0x94, 0x12, 0x74, 0x64, 0x11, 0x4a, + 0x7b, 0xa3, 0xde, 0x21, 0x76, 0x79, 0xba, 0xca, 0x47, 0x61, 0xdd, 0x29, 0x47, 0x74, 0x47, 0xa8, + 0x48, 0x25, 0xa8, 0x22, 0xe7, 0xa0, 0xc2, 0xee, 0x96, 0xba, 0xae, 0x43, 0x3b, 0xe8, 0x79, 0xa5, + 0xcc, 0x00, 0xbb, 0x0e, 0xfa, 0xd0, 0xcb, 0xc7, 0xaa, 0x49, 0xc6, 0x4e, 0xbd, 0x4e, 0x44, 0x4b, + 0xbc, 0x6c, 0xec, 0x1a, 0x34, 0xe8, 0x4b, 0xf9, 0xe7, 0x23, 0x6c, 0x1f, 0xab, 0x7b, 0x03, 0xdc, + 0xaa, 0x51, 0x71, 0xea, 0x04, 0xfa, 0xc4, 0x03, 0x92, 0x0d, 0xa1, 0x68, 0xba, 0xa9, 0xe1, 0x97, + 0x58, 0x6b, 0xd5, 0xd9, 0xdd, 0x13, 0x81, 0x75, 0x18, 0x48, 0xfe, 0x02, 0x90, 0x3f, 0xc7, 0x6c, + 0x49, 0x59, 0xe4, 0x10, 0x73, 0xd1, 0x43, 0x94, 0xff, 0x56, 0x82, 0x85, 0xe0, 0x64, 0xd3, 0x86, + 0xc7, 0x8f, 0xa1, 0xca, 0x6e, 0x1a, 0xba, 0xc4, 0x3c, 0x79, 0xcd, 0x7f, 0x61, 0xec, 0xee, 0x29, + 0xe0, 0xbf, 0xd3, 0x26, 0x4a, 0x70, 0x64, 0xd9, 0x87, 0xba, 0xd9, 0xef, 0x12, 0xc9, 0x3c, 0xa3, + 0xa8, 0x71, 0xe0, 0x63, 0x02, 0x93, 0xbf, 0x95, 0xe0, 0xe2, 0xd3, 0xa1, 0xa6, 0xba, 0x38, 0x90, + 0x27, 0xcc, 0xfa, 0xf4, 0xeb, 0x03, 0xef, 0xed, 0x55, 0x2e, 0x5b, 0xb7, 0x9c, 0x61, 0xcb, 0x5b, + 0x70, 0x56, 0xc1, 0x0e, 0x36, 0xb5, 0xd0, 0xc7, 0xa9, 0x2b, 0xfd, 0x21, 0xb4, 0x93, 0xd8, 0xcd, + 0x72, 0xf6, 0x2c, 0x61, 0xeb, 0xda, 0x84, 0xad, 0xcb, 0xfd, 0x0f, 0xc9, 0x13, 0xe8, 0x3c, 0xae, + 0xfc, 0xdf, 0x12, 0x2c, 0xdc, 0xd7, 0xbc, 0xf9, 0x5e, 0x5b, 0x5e, 0x18, 0xcd, 0x9b, 0xf2, 0xf1, + 0xbc, 0xe9, 0x55, 0x39, 0x12, 0xee, 0x52, 0xcd, 0x91, 0xe1, 0x85, 0x0a, 0x9b, 0x5e, 0xed, 0xcb, + 0xfb, 0xe2, 0x32, 0x57, 0xc1, 0xfb, 0xd8, 0xc6, 0x66, 0x0f, 0x6f, 0x5a, 0xbd, 0xc3, 0xc0, 0x03, + 0x2e, 0x29, 0xf8, 0x80, 0x6b, 0xda, 0x07, 0x61, 0x37, 0xff, 0x42, 0x82, 0x85, 0x58, 0xd7, 0x08, + 0x35, 0x00, 0x9e, 0x9a, 0x3d, 0xde, 0x4e, 0x6b, 0x9e, 0x42, 0x35, 0x28, 0x7b, 0xcd, 0xb5, 0xa6, + 0x84, 0xaa, 0x30, 0xb7, 0x6b, 0x51, 0xec, 0x66, 0x0e, 0x35, 0xa1, 0xc6, 0x08, 0x47, 0xbd, 0x1e, + 0x76, 0x9c, 0x66, 0x5e, 0x40, 0xd6, 0x55, 0x7d, 0x30, 0xb2, 0x71, 0xb3, 0x80, 0xea, 0x50, 0xd9, + 0xb5, 0xf8, 0xf3, 0xb7, 0x66, 0x11, 0x21, 0x68, 0x78, 0x6f, 0xe1, 0x38, 0x51, 0x29, 0x00, 0xf3, + 0xc8, 0xe6, 0x6e, 0x3e, 0x0b, 0xf6, 0x44, 0x76, 0x8f, 0x87, 0x18, 0x2d, 0xc1, 0xe9, 0xa7, 0xa6, + 0x86, 0xf7, 0x75, 0x13, 0x6b, 0xfe, 0xa7, 0xe6, 0x29, 0x74, 0x1a, 0xe6, 0xb7, 0xb0, 0xdd, 0xc7, + 0x01, 0x60, 0x0e, 0x2d, 0x40, 0x7d, 0x4b, 0x7f, 0x19, 0x00, 0xe5, 0xe5, 0x42, 0x59, 0x6a, 0x4a, + 0x2b, 0xff, 0xb3, 0x04, 0x15, 0x52, 0xcf, 0x3c, 0xb0, 0x2c, 0x5b, 0x43, 0x43, 0x40, 0xf4, 0xb5, + 0xa8, 0x31, 0xb4, 0x4c, 0xf1, 0x06, 0x1b, 0xbd, 0x97, 0x92, 0x33, 0xc5, 0x51, 0xb9, 0x1e, 0xb6, + 0xaf, 0xa7, 0x50, 0x44, 0xd0, 0xe5, 0x53, 0xc8, 0xa0, 0x33, 0xee, 0xea, 0x06, 0xde, 0xd5, 0x7b, + 0x87, 0xde, 0x33, 0x92, 0x31, 0x33, 0x46, 0x50, 0xbd, 0x19, 0x23, 0xd5, 0x33, 0x1f, 0xb0, 0x27, + 0xbd, 0x9e, 0x21, 0xca, 0xa7, 0xd0, 0x73, 0x38, 0xb3, 0x81, 0x03, 0x8e, 0xc7, 0x9b, 0x70, 0x25, + 0x7d, 0xc2, 0x18, 0xf2, 0x09, 0xa7, 0xdc, 0x84, 0x22, 0x6d, 0xc9, 0xa2, 0x24, 0xdf, 0x14, 0xfc, + 0xc9, 0x54, 0xfb, 0x72, 0x3a, 0x82, 0xe0, 0xf6, 0x05, 0xcc, 0x47, 0x7e, 0x68, 0x81, 0x6e, 0x24, + 0x90, 0x25, 0xff, 0x64, 0xa6, 0x7d, 0x33, 0x0b, 0xaa, 0x98, 0xab, 0x0f, 0x8d, 0xf0, 0x4b, 0x53, + 0x94, 0xd4, 0x14, 0x48, 0x7c, 0x23, 0xdf, 0xbe, 0x91, 0x01, 0x53, 0x4c, 0x64, 0x40, 0x33, 0xfa, + 0xf0, 0x1f, 0xdd, 0x1c, 0xcb, 0x20, 0xac, 0x6e, 0xbf, 0xc8, 0x84, 0x2b, 0xa6, 0x3b, 0xa6, 0x4a, + 0x10, 0x7b, 0x4b, 0x8e, 0x6e, 0x27, 0xb3, 0x49, 0x7b, 0xe4, 0xde, 0xbe, 0x93, 0x19, 0x5f, 0x4c, + 0xfd, 0xbb, 0xec, 0x2a, 0x28, 0xe9, 0x3d, 0x36, 0x7a, 0x3f, 0x99, 0xdd, 0x98, 0x87, 0xe4, 0xed, + 0x95, 0x93, 0x90, 0x08, 0x21, 0xbe, 0xa2, 0x77, 0x38, 0x09, 0x2f, 0x9a, 0xa3, 0x76, 0xe7, 0xf1, + 0x4b, 0x7f, 0xac, 0xdd, 0x7e, 0xff, 0x04, 0x14, 0x42, 0x00, 0x2b, 0xfa, 0xcb, 0x0a, 0xcf, 0x0c, + 0xef, 0x4c, 0xd4, 0x9a, 0xe9, 0x6c, 0xf0, 0x73, 0x98, 0x8f, 0xbc, 0xc6, 0x49, 0xb4, 0x9a, 0xe4, + 0x17, 0x3b, 0xed, 0x71, 0xf1, 0x9a, 0x99, 0x64, 0xe4, 0x4a, 0x0c, 0xa5, 0x68, 0x7f, 0xc2, 0xb5, + 0x59, 0xfb, 0x66, 0x16, 0x54, 0xb1, 0x10, 0x87, 0xba, 0xcb, 0xc8, 0xb5, 0x12, 0xba, 0x95, 0xcc, + 0x23, 0xf9, 0x4a, 0xac, 0xfd, 0x6e, 0x46, 0x6c, 0x31, 0xe9, 0xef, 0x00, 0xda, 0x39, 0x20, 0x35, + 0x8c, 0xb9, 0xaf, 0xf7, 0x47, 0xb6, 0xca, 0x9e, 0xdc, 0xa4, 0xf9, 0xe8, 0x38, 0x6a, 0x8a, 0xae, + 0x8c, 0xa5, 0x10, 0x93, 0x77, 0x01, 0x36, 0xb0, 0xbb, 0x85, 0x5d, 0x9b, 0x28, 0xe8, 0xf5, 0xc4, + 0xf3, 0xf6, 0x11, 0xbc, 0xa9, 0xde, 0x99, 0x88, 0x17, 0x08, 0x09, 0xcd, 0x2d, 0xd5, 0x24, 0xe5, + 0xbb, 0xff, 0x52, 0xed, 0x56, 0x22, 0x79, 0x14, 0x2d, 0x65, 0x43, 0x53, 0xb1, 0xc5, 0x94, 0x47, + 0x22, 0xcc, 0x06, 0xba, 0xa9, 0x51, 0xf7, 0xe3, 0xcb, 0x9c, 0x7c, 0x21, 0x15, 0x75, 0x3f, 0x63, + 0xf0, 0xc5, 0xc4, 0x5f, 0x4b, 0xf4, 0x67, 0x3b, 0x11, 0x84, 0x67, 0xba, 0x7b, 0xb0, 0x3d, 0x50, + 0x4d, 0x27, 0x8b, 0x08, 0x14, 0xf1, 0x04, 0x22, 0x70, 0x7c, 0x21, 0x82, 0x06, 0xf5, 0x50, 0x8f, + 0x14, 0x25, 0xbd, 0xf9, 0x4a, 0x6a, 0xf8, 0xb6, 0x97, 0x27, 0x23, 0x8a, 0x59, 0x0e, 0xa0, 0xee, + 0xa9, 0x34, 0xdb, 0xdc, 0x1b, 0x69, 0x92, 0xfa, 0x38, 0x29, 0x16, 0x99, 0x8c, 0x1a, 0xb4, 0xc8, + 0x78, 0x0b, 0x08, 0x65, 0x6b, 0x1d, 0x8e, 0xb3, 0xc8, 0xf4, 0xbe, 0x12, 0x73, 0x39, 0x91, 0x76, + 0x6b, 0xb2, 0x3f, 0x4b, 0xec, 0x1e, 0x27, 0xba, 0x9c, 0x94, 0xee, 0xad, 0x7c, 0x0a, 0x3d, 0x83, + 0x12, 0xff, 0xbd, 0xee, 0xdb, 0xe3, 0x0b, 0x42, 0xce, 0xfd, 0xda, 0x04, 0x2c, 0xc1, 0xf8, 0x10, + 0x96, 0x52, 0xca, 0xc1, 0xc4, 0x50, 0x38, 0xbe, 0x74, 0x9c, 0xe4, 0xa4, 0x55, 0x40, 0xf1, 0x1f, + 0xc5, 0x24, 0x1e, 0x53, 0xea, 0x6f, 0x67, 0x32, 0x4c, 0x11, 0xff, 0x5d, 0x4b, 0xe2, 0x14, 0xa9, + 0x3f, 0x7f, 0x99, 0x34, 0xc5, 0x13, 0x00, 0xbf, 0xe8, 0x4b, 0x3c, 0x8f, 0x58, 0x4d, 0x38, 0x81, + 0xe5, 0xca, 0x7f, 0x94, 0xa1, 0xec, 0xbd, 0xb0, 0x7a, 0x03, 0xf9, 0xff, 0x1b, 0x48, 0xc8, 0x3f, + 0x87, 0xf9, 0xc8, 0xaf, 0x39, 0x12, 0x8d, 0x27, 0xf9, 0x17, 0x1f, 0x93, 0x4e, 0xe8, 0x19, 0xff, + 0x5f, 0x03, 0x22, 0x36, 0xbf, 0x93, 0x96, 0xd4, 0x47, 0xc3, 0xf2, 0x04, 0xc6, 0xff, 0xbf, 0x83, + 0xf0, 0x63, 0x80, 0x40, 0xf8, 0x1d, 0x7f, 0x4f, 0x4e, 0x22, 0xca, 0xa4, 0xdd, 0x32, 0x12, 0x23, + 0xec, 0x8d, 0x2c, 0x77, 0x9a, 0xe9, 0x3e, 0x32, 0x3d, 0xae, 0x6e, 0x9d, 0xd0, 0x47, 0x4e, 0x90, + 0xde, 0x21, 0x9e, 0x24, 0xda, 0x4e, 0x4a, 0xf1, 0x24, 0x29, 0x4d, 0xac, 0xc4, 0x98, 0x92, 0xde, + 0xa3, 0x7a, 0x2d, 0xbe, 0x65, 0xf5, 0xee, 0x6f, 0xbd, 0xdf, 0xd7, 0xdd, 0x83, 0xd1, 0x1e, 0xf9, + 0x72, 0x87, 0xa1, 0xbe, 0xab, 0x5b, 0xfc, 0xaf, 0x3b, 0x9e, 0xee, 0xdd, 0xa1, 0xd4, 0x77, 0xc8, + 0x1c, 0xc3, 0xbd, 0xbd, 0x12, 0x1d, 0xdd, 0xfd, 0xdf, 0x00, 0x00, 0x00, 0xff, 0xff, 0xae, 0xee, + 0xce, 0x3f, 0xba, 0x44, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/internal/proto/index_coord.proto b/internal/proto/index_coord.proto index b82955d457..df7de6950d 100644 --- a/internal/proto/index_coord.proto +++ b/internal/proto/index_coord.proto @@ -67,6 +67,7 @@ message SegmentIndex { bool deleted = 12; uint64 create_time = 13; uint64 serialize_size = 14; + bool write_handoff = 15; } message RegisterNodeRequest { diff --git a/internal/proto/indexpb/index_coord.pb.go b/internal/proto/indexpb/index_coord.pb.go index 241dc9c906..d59978bf50 100644 --- a/internal/proto/indexpb/index_coord.pb.go +++ b/internal/proto/indexpb/index_coord.pb.go @@ -176,6 +176,7 @@ type SegmentIndex struct { Deleted bool `protobuf:"varint,12,opt,name=deleted,proto3" json:"deleted,omitempty"` CreateTime uint64 `protobuf:"varint,13,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"` SerializeSize uint64 `protobuf:"varint,14,opt,name=serialize_size,json=serializeSize,proto3" json:"serialize_size,omitempty"` + WriteHandoff bool `protobuf:"varint,15,opt,name=write_handoff,json=writeHandoff,proto3" json:"write_handoff,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -304,6 +305,13 @@ func (m *SegmentIndex) GetSerializeSize() uint64 { return 0 } +func (m *SegmentIndex) GetWriteHandoff() bool { + if m != nil { + return m.WriteHandoff + } + return false +} + type RegisterNodeRequest struct { Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` Address *commonpb.Address `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"` @@ -1922,133 +1930,134 @@ func init() { func init() { proto.RegisterFile("index_coord.proto", fileDescriptor_f9e019eb3fda53c2) } var fileDescriptor_f9e019eb3fda53c2 = []byte{ - // 2007 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x59, 0xdd, 0x6f, 0x1b, 0x59, - 0x15, 0xcf, 0x78, 0xf2, 0xe1, 0x39, 0x13, 0x27, 0xe9, 0x6d, 0x17, 0x79, 0xdd, 0x96, 0x26, 0x53, - 0xda, 0x1a, 0xa4, 0x4d, 0x4a, 0x16, 0xd0, 0x82, 0x00, 0x29, 0x1f, 0xdb, 0xe2, 0x74, 0x13, 0x85, - 0x71, 0xb5, 0x12, 0x2b, 0xa4, 0x61, 0xec, 0xb9, 0x4e, 0xef, 0xc6, 0x9e, 0xeb, 0xce, 0xbd, 0xd3, - 0x36, 0x45, 0x42, 0xbc, 0xf0, 0x00, 0x5a, 0x09, 0x89, 0x07, 0x10, 0x2f, 0x3c, 0xf1, 0xb4, 0x48, - 0xbc, 0x22, 0xf1, 0x2f, 0xc0, 0x5f, 0x81, 0xf8, 0x43, 0xd0, 0xfd, 0x98, 0xf1, 0xcc, 0x78, 0x1c, - 0xbb, 0x71, 0x78, 0xda, 0x37, 0xdf, 0x33, 0xe7, 0x7e, 0xfd, 0xce, 0xb9, 0xe7, 0xf7, 0xbb, 0xd7, - 0x70, 0x83, 0x84, 0x01, 0x7e, 0xe3, 0x75, 0x29, 0x8d, 0x82, 0xed, 0x61, 0x44, 0x39, 0x45, 0x68, - 0x40, 0xfa, 0xaf, 0x62, 0xa6, 0x5a, 0xdb, 0xf2, 0x7b, 0x63, 0xb5, 0x4b, 0x07, 0x03, 0x1a, 0x2a, - 0x5b, 0x63, 0x8d, 0x84, 0x1c, 0x47, 0xa1, 0xdf, 0xd7, 0xed, 0xd5, 0x6c, 0x0f, 0xe7, 0x8b, 0x0a, - 0x58, 0x2d, 0xd1, 0xab, 0x15, 0xf6, 0x28, 0x72, 0x60, 0xb5, 0x4b, 0xfb, 0x7d, 0xdc, 0xe5, 0x84, - 0x86, 0xad, 0xc3, 0xba, 0xb1, 0x69, 0x34, 0x4d, 0x37, 0x67, 0x43, 0x75, 0x58, 0xe9, 0x11, 0xdc, - 0x0f, 0x5a, 0x87, 0xf5, 0x8a, 0xfc, 0x9c, 0x34, 0xd1, 0x5d, 0x00, 0xb5, 0xc0, 0xd0, 0x1f, 0xe0, - 0xba, 0xb9, 0x69, 0x34, 0x2d, 0xd7, 0x92, 0x96, 0x13, 0x7f, 0x80, 0x45, 0x47, 0xd9, 0x68, 0x1d, - 0xd6, 0x17, 0x55, 0x47, 0xdd, 0x44, 0xfb, 0x60, 0xf3, 0x8b, 0x21, 0xf6, 0x86, 0x7e, 0xe4, 0x0f, - 0x58, 0x7d, 0x69, 0xd3, 0x6c, 0xda, 0xbb, 0x5b, 0xdb, 0xb9, 0xad, 0xe9, 0x3d, 0x3d, 0xc3, 0x17, - 0x9f, 0xfa, 0xfd, 0x18, 0x9f, 0xfa, 0x24, 0x72, 0x41, 0xf4, 0x3a, 0x95, 0x9d, 0xd0, 0x21, 0xac, - 0xaa, 0xc9, 0xf5, 0x20, 0xcb, 0xb3, 0x0e, 0x62, 0xcb, 0x6e, 0x6a, 0x14, 0xe7, 0x37, 0x06, 0xc0, - 0x13, 0xb9, 0x1d, 0x61, 0x44, 0x3f, 0x4c, 0x76, 0x44, 0xc2, 0x1e, 0x95, 0x68, 0xd8, 0xbb, 0x77, - 0xb7, 0xc7, 0x21, 0xdf, 0x4e, 0x21, 0xd4, 0x1b, 0x96, 0x68, 0xd6, 0x61, 0x25, 0xc0, 0x7d, 0xcc, - 0x71, 0x20, 0x91, 0xaa, 0xba, 0x49, 0x13, 0xdd, 0x03, 0xbb, 0x1b, 0x61, 0x9f, 0x63, 0x8f, 0x13, - 0x0d, 0xd5, 0xa2, 0x0b, 0xca, 0xf4, 0x9c, 0x0c, 0xb0, 0xf3, 0x5f, 0x13, 0x56, 0xdb, 0xf8, 0x6c, - 0x80, 0x43, 0xae, 0x56, 0x32, 0x4b, 0x64, 0x36, 0xc1, 0x1e, 0xfa, 0x11, 0x27, 0xda, 0x45, 0x45, - 0x27, 0x6b, 0x42, 0x77, 0xc0, 0x62, 0x7a, 0xd4, 0x43, 0x39, 0xab, 0xe9, 0x8e, 0x0c, 0xe8, 0x7d, - 0xa8, 0x86, 0xf1, 0xc0, 0x8b, 0xe8, 0x6b, 0x96, 0x44, 0x28, 0x8c, 0x07, 0x2e, 0x7d, 0xcd, 0xb2, - 0xb1, 0x5b, 0xca, 0xc7, 0xae, 0x0e, 0x2b, 0x9d, 0x98, 0xc8, 0x74, 0x58, 0x56, 0x5f, 0x74, 0x13, - 0x7d, 0x0d, 0x96, 0x43, 0x1a, 0xe0, 0xd6, 0x61, 0x7d, 0x45, 0x7e, 0xd0, 0x2d, 0x74, 0x1f, 0x6a, - 0x0a, 0xd4, 0x57, 0x38, 0x62, 0x84, 0x86, 0xf5, 0xaa, 0xda, 0x8b, 0x34, 0x7e, 0xaa, 0x6c, 0xe8, - 0xbb, 0xb0, 0xc4, 0xb8, 0xcf, 0x71, 0xdd, 0xda, 0x34, 0x9a, 0x6b, 0xbb, 0xf7, 0x4a, 0xe3, 0x28, - 0xa1, 0x69, 0x0b, 0x37, 0x57, 0x79, 0x0b, 0x60, 0x7b, 0x3e, 0xe9, 0x7b, 0x11, 0xf6, 0x19, 0x0d, - 0xeb, 0x20, 0x73, 0x10, 0x84, 0xc9, 0x95, 0x16, 0xf4, 0xad, 0xe4, 0x10, 0xf5, 0x48, 0x1f, 0x33, - 0x6f, 0xe8, 0xf3, 0x17, 0xac, 0x6e, 0x6f, 0x9a, 0x4d, 0xcb, 0x5d, 0x97, 0x1f, 0x9e, 0x08, 0xfb, - 0xa9, 0x30, 0x67, 0xe3, 0xb7, 0x7a, 0x69, 0xfc, 0x6a, 0xc5, 0xf8, 0xa1, 0x07, 0xb0, 0xc6, 0x70, - 0x44, 0xfc, 0x3e, 0x79, 0x8b, 0x3d, 0x46, 0xde, 0xe2, 0xfa, 0x9a, 0xf4, 0xa9, 0xa5, 0xd6, 0x36, - 0x79, 0x8b, 0x9d, 0x3f, 0x19, 0x70, 0xd3, 0xc5, 0x67, 0x84, 0x71, 0x1c, 0x9d, 0xd0, 0x00, 0xbb, - 0xf8, 0x65, 0x8c, 0x19, 0x47, 0x8f, 0x61, 0xb1, 0xe3, 0x33, 0xac, 0x33, 0xee, 0x4e, 0xe9, 0xe6, - 0x8f, 0xd9, 0xd9, 0xbe, 0xcf, 0xb0, 0x2b, 0x3d, 0xd1, 0xf7, 0x60, 0xc5, 0x0f, 0x82, 0x08, 0x33, - 0x26, 0xe3, 0x3e, 0xa9, 0xd3, 0x9e, 0xf2, 0x71, 0x13, 0xe7, 0x4c, 0x90, 0xcc, 0x6c, 0x90, 0x9c, - 0xdf, 0x1b, 0x70, 0x2b, 0xbf, 0x32, 0x36, 0xa4, 0x21, 0xc3, 0xe8, 0x43, 0x58, 0x16, 0x50, 0xc7, - 0x4c, 0x2f, 0xee, 0x76, 0xe9, 0x3c, 0x6d, 0xe9, 0xe2, 0x6a, 0x57, 0x71, 0xc0, 0x49, 0x48, 0x78, - 0x72, 0x36, 0xd5, 0x0a, 0xb7, 0x8a, 0x07, 0x49, 0x97, 0xa9, 0x56, 0x48, 0xb8, 0x3a, 0x8e, 0x2e, - 0x90, 0xf4, 0xb7, 0xf3, 0x33, 0xb8, 0xf5, 0x14, 0xf3, 0x4c, 0xc8, 0x35, 0x56, 0xb3, 0x9c, 0x8c, - 0x7c, 0x65, 0xaa, 0x14, 0x2a, 0x93, 0xf3, 0x57, 0x03, 0xde, 0x2b, 0x8c, 0x3d, 0xcf, 0x6e, 0xd3, - 0xdc, 0xad, 0xcc, 0x93, 0xbb, 0x66, 0x31, 0x77, 0x9d, 0x5f, 0x1b, 0x70, 0xfb, 0x29, 0xe6, 0xd9, - 0xba, 0x70, 0xcd, 0x48, 0xa0, 0xaf, 0x03, 0xa4, 0xf5, 0x80, 0xd5, 0xcd, 0x4d, 0xb3, 0x69, 0xba, - 0x19, 0x8b, 0xf3, 0x5b, 0x03, 0x6e, 0x8c, 0xcd, 0x9f, 0x2f, 0x2b, 0x46, 0xb1, 0xac, 0xfc, 0xbf, - 0xe0, 0xf8, 0x83, 0x01, 0x77, 0xca, 0xe1, 0x98, 0x27, 0x78, 0x3f, 0x52, 0x9d, 0xb0, 0xc8, 0x52, - 0xc1, 0x20, 0x0f, 0xca, 0xca, 0xfd, 0xf8, 0x9c, 0xba, 0x93, 0xf3, 0xe7, 0x0a, 0xa0, 0x03, 0x59, - 0x07, 0xe4, 0xc7, 0x77, 0x09, 0xcd, 0x95, 0x89, 0xb5, 0x40, 0x9f, 0x8b, 0xd7, 0x41, 0x9f, 0x4b, - 0x57, 0xa1, 0x4f, 0x91, 0x08, 0xa2, 0x20, 0x32, 0xee, 0x0f, 0x86, 0x92, 0x0e, 0x16, 0xdd, 0x91, - 0xc1, 0x79, 0x03, 0x37, 0x93, 0x53, 0x26, 0xa9, 0xf2, 0x1d, 0xb0, 0xc9, 0xe7, 0x65, 0xa5, 0x98, - 0x97, 0x53, 0x10, 0x72, 0xfe, 0x51, 0x81, 0x1b, 0xad, 0xa4, 0xba, 0x8b, 0xe2, 0x2e, 0xf9, 0xf9, - 0xf2, 0xb4, 0x9d, 0x1c, 0x8e, 0x0c, 0x19, 0x9a, 0x13, 0xc9, 0x70, 0x31, 0x4f, 0x86, 0xf9, 0x05, - 0x2e, 0x15, 0x43, 0x78, 0x2d, 0xea, 0x05, 0x35, 0x61, 0x63, 0x44, 0x6e, 0x9a, 0xdb, 0x56, 0x24, - 0xb7, 0xad, 0x91, 0xec, 0xee, 0x19, 0x7a, 0x04, 0xeb, 0x29, 0x13, 0x05, 0x8a, 0xa0, 0xaa, 0x32, - 0x5c, 0x23, 0xda, 0x0a, 0x24, 0x43, 0xfd, 0xd3, 0x00, 0x3b, 0xcd, 0xf6, 0x19, 0x15, 0x62, 0x0e, - 0xd7, 0x4a, 0x11, 0xd7, 0x2d, 0x58, 0xc5, 0xa1, 0xdf, 0xe9, 0x63, 0x4f, 0xae, 0x49, 0x42, 0x58, - 0x75, 0x6d, 0x65, 0x53, 0x62, 0xe7, 0x09, 0xd8, 0x23, 0xd9, 0x95, 0x24, 0xf4, 0x83, 0x89, 0xba, - 0x2b, 0x1b, 0x54, 0x17, 0x52, 0xfd, 0xc5, 0x9c, 0xdf, 0x55, 0x46, 0x9c, 0xa1, 0x32, 0x6e, 0x9e, - 0xca, 0xf0, 0x73, 0x58, 0xd5, 0xbb, 0x50, 0x72, 0x50, 0xd5, 0x87, 0xef, 0x97, 0x2d, 0xab, 0x6c, - 0xd2, 0xed, 0x0c, 0x8c, 0x1f, 0x87, 0x3c, 0xba, 0x70, 0x6d, 0x36, 0xb2, 0x34, 0x3c, 0xd8, 0x28, - 0x3a, 0xa0, 0x0d, 0x30, 0xcf, 0xf1, 0x85, 0xc6, 0x58, 0xfc, 0x14, 0xb5, 0xf4, 0x95, 0x88, 0xbd, - 0xa6, 0xd0, 0x7b, 0x97, 0x16, 0xa7, 0x1e, 0x75, 0x95, 0xf7, 0x0f, 0x2a, 0x1f, 0x19, 0x0e, 0x85, - 0x8d, 0xc3, 0x88, 0x0e, 0xdf, 0xb9, 0x2c, 0x4d, 0x57, 0xf5, 0xe5, 0xc7, 0x44, 0x10, 0xf6, 0x21, - 0x66, 0xdd, 0x88, 0x74, 0xf0, 0x9c, 0x93, 0x8e, 0x11, 0xf6, 0x17, 0x06, 0xbc, 0x57, 0x18, 0x7b, - 0x9e, 0xc8, 0xfe, 0x38, 0x9f, 0x6f, 0x2a, 0xb0, 0x53, 0x74, 0x7e, 0x36, 0xcf, 0x7c, 0x49, 0x44, - 0xf2, 0xdb, 0xbe, 0x38, 0xef, 0xa7, 0x11, 0x3d, 0x93, 0x32, 0xeb, 0xfa, 0x76, 0xfc, 0x47, 0x03, - 0xee, 0x4e, 0x98, 0x63, 0x9e, 0x9d, 0x6f, 0xe9, 0xba, 0x83, 0x03, 0x25, 0xfb, 0xf5, 0x9d, 0x41, - 0xdb, 0xa4, 0xf4, 0xbf, 0x0b, 0xc0, 0x29, 0xf7, 0xfb, 0xca, 0x41, 0x5f, 0x1a, 0xa4, 0x45, 0x7c, - 0x76, 0xfe, 0x56, 0x81, 0x5a, 0x9b, 0xd3, 0xc8, 0x3f, 0xc3, 0x07, 0x34, 0xec, 0x91, 0x33, 0x91, - 0x11, 0x89, 0x14, 0x35, 0xe4, 0x36, 0x52, 0xb1, 0xb9, 0x05, 0xab, 0x7e, 0xb7, 0x8b, 0x19, 0xf3, - 0xce, 0xf1, 0x85, 0x4e, 0x18, 0xcb, 0xb5, 0x95, 0xed, 0x99, 0x30, 0x09, 0x7d, 0xce, 0x70, 0x37, - 0xc2, 0xdc, 0x1b, 0x79, 0xea, 0xa4, 0x5b, 0x57, 0x1f, 0xf6, 0x12, 0x6f, 0xa1, 0x5d, 0x63, 0x86, - 0xdb, 0xed, 0x4f, 0x64, 0xb1, 0xad, 0xba, 0xba, 0x25, 0x94, 0x43, 0x27, 0xee, 0x9e, 0x63, 0x9e, - 0x2d, 0xb6, 0xa0, 0x4c, 0x32, 0x67, 0x6f, 0x83, 0x15, 0x51, 0xca, 0x65, 0x85, 0x94, 0x34, 0x65, - 0xb9, 0x55, 0x61, 0x10, 0x45, 0x44, 0x8f, 0xda, 0xda, 0x3b, 0x96, 0xd7, 0x16, 0x35, 0x6a, 0x6b, - 0xef, 0x58, 0xdc, 0xae, 0x5a, 0x7b, 0xc7, 0x1f, 0x87, 0xc1, 0x90, 0x92, 0x90, 0xcb, 0x72, 0x69, - 0xb9, 0x59, 0x93, 0xd8, 0x1e, 0x53, 0x48, 0x78, 0x82, 0x59, 0xe5, 0xd5, 0xc5, 0x72, 0x6d, 0x6d, - 0x7b, 0x7e, 0x31, 0xc4, 0xce, 0x7f, 0x4c, 0xd8, 0x50, 0xf2, 0xe0, 0x88, 0x76, 0x92, 0xf4, 0xb8, - 0x03, 0x56, 0xb7, 0x1f, 0x0b, 0xa5, 0xad, 0x73, 0xc3, 0x72, 0x47, 0x86, 0xfc, 0x8d, 0xc5, 0x1b, - 0x46, 0xb8, 0x47, 0xde, 0x68, 0xe4, 0x46, 0x37, 0x96, 0x53, 0x69, 0xce, 0xf2, 0x8f, 0x39, 0xc6, - 0x3f, 0x81, 0xcf, 0x7d, 0x4d, 0x0a, 0x8b, 0x92, 0x14, 0x2c, 0x61, 0x51, 0x7c, 0x30, 0x76, 0x27, - 0x5b, 0x2a, 0xb9, 0x93, 0x65, 0x78, 0x6f, 0x39, 0xcf, 0x7b, 0xf9, 0xe4, 0x5d, 0x29, 0xd6, 0x88, - 0x9f, 0xc0, 0x5a, 0x02, 0x4c, 0x57, 0xe6, 0x88, 0x44, 0xaf, 0xe4, 0x06, 0x20, 0xcb, 0x57, 0x36, - 0x99, 0xdc, 0x1a, 0xcb, 0xe5, 0x56, 0x91, 0x27, 0xad, 0x2b, 0xf1, 0x64, 0x41, 0x30, 0xc1, 0x55, - 0x04, 0x53, 0xf6, 0xb2, 0x6c, 0xe7, 0x2e, 0xcb, 0xce, 0x27, 0xb0, 0xf1, 0xd3, 0x18, 0x47, 0x17, - 0x47, 0xb4, 0xc3, 0x66, 0x8b, 0x71, 0x03, 0xaa, 0x3a, 0x50, 0x89, 0xb8, 0x49, 0xdb, 0xce, 0xbf, - 0x0c, 0xa8, 0xc9, 0x63, 0xff, 0xdc, 0x67, 0xe7, 0xc9, 0xbb, 0x42, 0x12, 0x65, 0x23, 0x1f, 0xe5, - 0xab, 0x4b, 0xed, 0xcc, 0xa5, 0x58, 0xca, 0x7e, 0x4b, 0x17, 0x38, 0x79, 0x1d, 0x2e, 0x93, 0x0b, - 0x8b, 0x65, 0x72, 0xa1, 0x28, 0xda, 0x97, 0xc6, 0x44, 0xfb, 0x97, 0x06, 0xdc, 0xc8, 0x80, 0x33, - 0x4f, 0xed, 0xca, 0x41, 0x5a, 0x29, 0x42, 0xba, 0x9f, 0xaf, 0xe9, 0x66, 0x59, 0x8c, 0x33, 0x35, - 0x3d, 0x01, 0x37, 0x57, 0xd7, 0x9f, 0xc1, 0xba, 0xa0, 0xcc, 0xeb, 0x89, 0xe3, 0xbf, 0x0d, 0x58, - 0x39, 0xa2, 0x1d, 0x19, 0xc1, 0x6c, 0xf2, 0x18, 0xf9, 0x97, 0x96, 0x0d, 0x30, 0x03, 0x32, 0xd0, - 0x85, 0x58, 0xfc, 0x14, 0x87, 0x8b, 0x71, 0x3f, 0xe2, 0xa3, 0xb7, 0x22, 0xa1, 0xa7, 0x84, 0x45, - 0x3e, 0x35, 0xbc, 0x0f, 0x55, 0x1c, 0x06, 0xea, 0xa3, 0x16, 0x9d, 0x38, 0x0c, 0xe4, 0xa7, 0xeb, - 0x11, 0xf5, 0xb7, 0x60, 0x69, 0x48, 0x47, 0xef, 0x3b, 0xaa, 0xe1, 0xdc, 0x02, 0xf4, 0x14, 0xf3, - 0x23, 0xda, 0x11, 0x51, 0x49, 0xe0, 0x71, 0xfe, 0x52, 0x91, 0x1a, 0x7f, 0x64, 0x9e, 0x27, 0xc0, - 0x0e, 0xd4, 0x14, 0xf3, 0x7c, 0x4e, 0x3b, 0x5e, 0x18, 0x27, 0xa0, 0xd8, 0xd2, 0x78, 0x44, 0x3b, - 0x27, 0xf1, 0x00, 0x7d, 0x00, 0x37, 0x49, 0xe8, 0x0d, 0x35, 0x19, 0xa6, 0x9e, 0x0a, 0xa5, 0x0d, - 0x12, 0x26, 0x34, 0xa9, 0xdd, 0x1f, 0xc2, 0x3a, 0x0e, 0x5f, 0xc6, 0x38, 0xc6, 0xa9, 0xab, 0xc2, - 0xac, 0xa6, 0xcd, 0xda, 0x4f, 0x90, 0x9e, 0xcf, 0xce, 0x3d, 0xd6, 0xa7, 0x9c, 0xe9, 0x62, 0x68, - 0x09, 0x4b, 0x5b, 0x18, 0xd0, 0x47, 0x60, 0x89, 0xee, 0x2a, 0xb5, 0x94, 0x56, 0xbf, 0x5d, 0x96, - 0x5a, 0x3a, 0xde, 0x6e, 0xf5, 0x73, 0xf5, 0x83, 0xed, 0x7e, 0x69, 0x01, 0xc8, 0x84, 0x3b, 0xa0, - 0x34, 0x0a, 0xd0, 0x50, 0xa2, 0x78, 0x40, 0x07, 0x43, 0x1a, 0xe2, 0x90, 0xcb, 0x53, 0xc9, 0xd0, - 0xe3, 0x09, 0x2f, 0x23, 0xe3, 0xae, 0x1a, 0xf7, 0xc6, 0xc3, 0x09, 0x3d, 0x0a, 0xee, 0xce, 0x02, - 0x7a, 0x29, 0x25, 0xb1, 0x68, 0x12, 0xc6, 0x49, 0x97, 0x1d, 0xbc, 0xf0, 0xc3, 0x10, 0xf7, 0xd1, - 0xee, 0xe4, 0x39, 0xc7, 0x9c, 0x93, 0x59, 0xef, 0xe7, 0xfb, 0xe8, 0x46, 0x9b, 0x47, 0x24, 0x3c, - 0x4b, 0x42, 0xef, 0x2c, 0xa0, 0xe7, 0x60, 0x67, 0xae, 0xc4, 0xe8, 0x61, 0x19, 0x52, 0xe3, 0x77, - 0xe6, 0xc6, 0x65, 0x39, 0xe2, 0x2c, 0xa0, 0x1e, 0xd4, 0x72, 0x6f, 0x36, 0xa8, 0x79, 0x99, 0x12, - 0xcf, 0x3e, 0x94, 0x34, 0xbe, 0x39, 0x83, 0x67, 0xba, 0xfa, 0x5f, 0x2a, 0xc0, 0xc6, 0x1e, 0x3d, - 0x76, 0x26, 0x0c, 0x32, 0xe9, 0x79, 0xa6, 0xf1, 0x78, 0xf6, 0x0e, 0xe9, 0xe4, 0xc1, 0x68, 0x93, - 0x32, 0x7f, 0xd0, 0xa3, 0xe9, 0xd7, 0x0d, 0x35, 0x5b, 0x73, 0xd6, 0x7b, 0x89, 0xb3, 0x80, 0x4e, - 0xc1, 0x4a, 0xaf, 0x06, 0xe8, 0x1b, 0x65, 0x1d, 0x8b, 0x37, 0x87, 0x19, 0x82, 0x93, 0xd3, 0xe7, - 0xe5, 0xc1, 0x29, 0xbb, 0x1e, 0x94, 0x07, 0xa7, 0x54, 0xec, 0x3b, 0x0b, 0xe8, 0x57, 0xa3, 0x87, - 0xbb, 0x9c, 0x2a, 0x46, 0x8f, 0x2f, 0xdb, 0x7e, 0x99, 0x48, 0x6f, 0x7c, 0xfb, 0x1d, 0x7a, 0x64, - 0x92, 0x03, 0xb5, 0x5f, 0xd0, 0xd7, 0x4a, 0x9d, 0xc4, 0x91, 0x2f, 0xc4, 0xfc, 0xe4, 0xf3, 0x3b, - 0xee, 0x3a, 0x71, 0xf2, 0x4b, 0x7a, 0xa4, 0x93, 0x7b, 0x00, 0x4f, 0x31, 0x3f, 0xc6, 0x3c, 0x22, - 0x5d, 0x56, 0x3c, 0x56, 0xba, 0x31, 0x72, 0x48, 0xa6, 0x7a, 0x34, 0xd5, 0x2f, 0x99, 0x60, 0xf7, - 0xef, 0xcb, 0xfa, 0xcf, 0xa1, 0x13, 0x1a, 0xe0, 0xaf, 0x46, 0xad, 0x3a, 0x05, 0x2b, 0xd5, 0xe7, - 0xe5, 0x47, 0xa1, 0x28, 0xdf, 0xa7, 0x1d, 0x85, 0xcf, 0xc0, 0x4a, 0x05, 0x4f, 0xf9, 0x88, 0x45, - 0xb1, 0xd8, 0x78, 0x30, 0xc5, 0x2b, 0x5d, 0xed, 0x09, 0x54, 0x13, 0x81, 0x82, 0xee, 0x4f, 0x3a, - 0xb7, 0xd9, 0x91, 0xa7, 0xac, 0xf5, 0x17, 0x60, 0x67, 0xd8, 0xbb, 0xbc, 0x52, 0x8f, 0xb3, 0x7e, - 0xe3, 0xd1, 0x54, 0xbf, 0xaf, 0xc6, 0x81, 0xd9, 0xff, 0xce, 0x67, 0xbb, 0x67, 0x84, 0xbf, 0x88, - 0x3b, 0x02, 0xd9, 0x1d, 0xe5, 0xf9, 0x01, 0xa1, 0xfa, 0xd7, 0x4e, 0xb2, 0xca, 0x1d, 0x39, 0xd2, - 0x8e, 0xc4, 0x69, 0xd8, 0xe9, 0x2c, 0xcb, 0xe6, 0x87, 0xff, 0x0b, 0x00, 0x00, 0xff, 0xff, 0x4a, - 0x31, 0x02, 0x5e, 0xdf, 0x1d, 0x00, 0x00, + // 2030 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x59, 0x4f, 0x6f, 0xdb, 0xc8, + 0x15, 0x37, 0x25, 0xff, 0x11, 0x9f, 0x24, 0xdb, 0x99, 0x64, 0x0b, 0xad, 0x92, 0x34, 0x36, 0xd3, + 0x24, 0x6a, 0x81, 0xb5, 0x53, 0x6f, 0x5b, 0x6c, 0x8b, 0xb6, 0x80, 0xff, 0x6c, 0xb2, 0x72, 0xd6, + 0x86, 0x4b, 0x05, 0x0b, 0x74, 0x51, 0x80, 0xa5, 0xc4, 0x91, 0x3d, 0x6b, 0x69, 0x46, 0xe1, 0x8c, + 0x92, 0x38, 0x05, 0x8a, 0x5e, 0x7a, 0xd8, 0x62, 0x81, 0x02, 0x3d, 0xb4, 0xe8, 0xa5, 0xa7, 0x9e, + 0xb6, 0x40, 0xaf, 0x05, 0xfa, 0x15, 0xda, 0x4f, 0xd1, 0x4f, 0x52, 0xcc, 0x1f, 0x52, 0x24, 0x45, + 0x59, 0x8a, 0xe5, 0x3d, 0xe5, 0xa6, 0x79, 0x7c, 0xc3, 0x99, 0xf9, 0xbd, 0x37, 0xef, 0xf7, 0x7b, + 0x14, 0xdc, 0x20, 0x34, 0xc0, 0xaf, 0xbd, 0x0e, 0x63, 0x61, 0xb0, 0x35, 0x08, 0x99, 0x60, 0x08, + 0xf5, 0x49, 0xef, 0xe5, 0x90, 0xeb, 0xd1, 0x96, 0x7a, 0x5e, 0xaf, 0x74, 0x58, 0xbf, 0xcf, 0xa8, + 0xb6, 0xd5, 0x57, 0x09, 0x15, 0x38, 0xa4, 0x7e, 0xcf, 0x8c, 0x2b, 0xc9, 0x19, 0xce, 0x57, 0x05, + 0xb0, 0x9b, 0x72, 0x56, 0x93, 0x76, 0x19, 0x72, 0xa0, 0xd2, 0x61, 0xbd, 0x1e, 0xee, 0x08, 0xc2, + 0x68, 0xf3, 0xa0, 0x66, 0x6d, 0x58, 0x8d, 0xa2, 0x9b, 0xb2, 0xa1, 0x1a, 0xac, 0x74, 0x09, 0xee, + 0x05, 0xcd, 0x83, 0x5a, 0x41, 0x3d, 0x8e, 0x86, 0xe8, 0x2e, 0x80, 0xde, 0x20, 0xf5, 0xfb, 0xb8, + 0x56, 0xdc, 0xb0, 0x1a, 0xb6, 0x6b, 0x2b, 0xcb, 0xb1, 0xdf, 0xc7, 0x72, 0xa2, 0x1a, 0x34, 0x0f, + 0x6a, 0x8b, 0x7a, 0xa2, 0x19, 0xa2, 0x3d, 0x28, 0x8b, 0x8b, 0x01, 0xf6, 0x06, 0x7e, 0xe8, 0xf7, + 0x79, 0x6d, 0x69, 0xa3, 0xd8, 0x28, 0xef, 0x6c, 0x6e, 0xa5, 0x8e, 0x66, 0xce, 0xf4, 0x0c, 0x5f, + 0x7c, 0xe6, 0xf7, 0x86, 0xf8, 0xc4, 0x27, 0xa1, 0x0b, 0x72, 0xd6, 0x89, 0x9a, 0x84, 0x0e, 0xa0, + 0xa2, 0x17, 0x37, 0x2f, 0x59, 0x9e, 0xf5, 0x25, 0x65, 0x35, 0x4d, 0xbf, 0xc5, 0xf9, 0xbd, 0x05, + 0xf0, 0x44, 0x1d, 0x47, 0x1a, 0xd1, 0x4f, 0xa3, 0x13, 0x11, 0xda, 0x65, 0x0a, 0x8d, 0xf2, 0xce, + 0xdd, 0xad, 0x71, 0xc8, 0xb7, 0x62, 0x08, 0xcd, 0x81, 0x15, 0x9a, 0x35, 0x58, 0x09, 0x70, 0x0f, + 0x0b, 0x1c, 0x28, 0xa4, 0x4a, 0x6e, 0x34, 0x44, 0xf7, 0xa0, 0xdc, 0x09, 0xb1, 0x2f, 0xb0, 0x27, + 0x88, 0x81, 0x6a, 0xd1, 0x05, 0x6d, 0x7a, 0x4e, 0xfa, 0xd8, 0xf9, 0x72, 0x11, 0x2a, 0x2d, 0x7c, + 0xda, 0xc7, 0x54, 0xe8, 0x9d, 0xcc, 0x12, 0x99, 0x0d, 0x28, 0x0f, 0xfc, 0x50, 0x10, 0xe3, 0xa2, + 0xa3, 0x93, 0x34, 0xa1, 0x3b, 0x60, 0x73, 0xf3, 0xd6, 0x03, 0xb5, 0x6a, 0xd1, 0x1d, 0x19, 0xd0, + 0xfb, 0x50, 0xa2, 0xc3, 0xbe, 0x17, 0xb2, 0x57, 0x3c, 0x8a, 0x10, 0x1d, 0xf6, 0x5d, 0xf6, 0x8a, + 0x27, 0x63, 0xb7, 0x94, 0x8e, 0x5d, 0x0d, 0x56, 0xda, 0x43, 0xa2, 0xd2, 0x61, 0x59, 0x3f, 0x31, + 0x43, 0xf4, 0x2d, 0x58, 0xa6, 0x2c, 0xc0, 0xcd, 0x83, 0xda, 0x8a, 0x7a, 0x60, 0x46, 0xe8, 0x3e, + 0x54, 0x35, 0xa8, 0x2f, 0x71, 0xc8, 0x09, 0xa3, 0xb5, 0x92, 0x3e, 0x8b, 0x32, 0x7e, 0xa6, 0x6d, + 0xe8, 0x87, 0xb0, 0xc4, 0x85, 0x2f, 0x70, 0xcd, 0xde, 0xb0, 0x1a, 0xab, 0x3b, 0xf7, 0x72, 0xe3, + 0xa8, 0xa0, 0x69, 0x49, 0x37, 0x57, 0x7b, 0x4b, 0x60, 0xbb, 0x3e, 0xe9, 0x79, 0x21, 0xf6, 0x39, + 0xa3, 0x35, 0x50, 0x39, 0x08, 0xd2, 0xe4, 0x2a, 0x0b, 0xfa, 0x5e, 0x74, 0x89, 0xba, 0xa4, 0x87, + 0xb9, 0x37, 0xf0, 0xc5, 0x19, 0xaf, 0x95, 0x37, 0x8a, 0x0d, 0xdb, 0x5d, 0x53, 0x0f, 0x9e, 0x48, + 0xfb, 0x89, 0x34, 0x27, 0xe3, 0x57, 0xb9, 0x34, 0x7e, 0xd5, 0x6c, 0xfc, 0xd0, 0x03, 0x58, 0xe5, + 0x38, 0x24, 0x7e, 0x8f, 0xbc, 0xc1, 0x1e, 0x27, 0x6f, 0x70, 0x6d, 0x55, 0xf9, 0x54, 0x63, 0x6b, + 0x8b, 0xbc, 0xc1, 0x12, 0x8a, 0x57, 0x21, 0x11, 0xd8, 0x3b, 0xf3, 0x69, 0xc0, 0xba, 0xdd, 0xda, + 0x9a, 0x5a, 0xa7, 0xa2, 0x8c, 0x9f, 0x68, 0x9b, 0xf3, 0x17, 0x0b, 0x6e, 0xba, 0xf8, 0x94, 0x70, + 0x81, 0xc3, 0x63, 0x16, 0x60, 0x17, 0xbf, 0x18, 0x62, 0x2e, 0xd0, 0x63, 0x58, 0x6c, 0xfb, 0x1c, + 0x9b, 0xb4, 0xbc, 0x93, 0x8b, 0xd0, 0x11, 0x3f, 0xdd, 0xf3, 0x39, 0x76, 0x95, 0x27, 0xfa, 0x11, + 0xac, 0xf8, 0x41, 0x10, 0x62, 0xce, 0x55, 0x72, 0x4c, 0x9a, 0xb4, 0xab, 0x7d, 0xdc, 0xc8, 0x39, + 0x11, 0xc9, 0x62, 0x32, 0x92, 0xce, 0x1f, 0x2d, 0xb8, 0x95, 0xde, 0x19, 0x1f, 0x30, 0xca, 0x31, + 0xfa, 0x10, 0x96, 0x65, 0x3c, 0x86, 0xdc, 0x6c, 0xee, 0x76, 0xee, 0x3a, 0x2d, 0xe5, 0xe2, 0x1a, + 0x57, 0x59, 0x05, 0x08, 0x25, 0x22, 0xba, 0xc0, 0x7a, 0x87, 0x9b, 0xd9, 0xdb, 0x66, 0x6a, 0x59, + 0x93, 0x12, 0xa1, 0xef, 0xac, 0x0b, 0x24, 0xfe, 0xed, 0xfc, 0x12, 0x6e, 0x3d, 0xc5, 0x22, 0x91, + 0x17, 0x06, 0xab, 0x59, 0xae, 0x4f, 0xba, 0x7c, 0x15, 0x32, 0xe5, 0xcb, 0xf9, 0xbb, 0x05, 0xef, + 0x65, 0xde, 0x3d, 0xcf, 0x69, 0xe3, 0x04, 0x2f, 0xcc, 0x93, 0xe0, 0xc5, 0x6c, 0x82, 0x3b, 0xbf, + 0xb3, 0xe0, 0xf6, 0x53, 0x2c, 0x92, 0xc5, 0xe3, 0x9a, 0x91, 0x40, 0xdf, 0x06, 0x88, 0x8b, 0x06, + 0xaf, 0x15, 0x37, 0x8a, 0x8d, 0xa2, 0x9b, 0xb0, 0x38, 0x5f, 0x5a, 0x70, 0x63, 0x6c, 0xfd, 0x74, + 0xed, 0xb1, 0xb2, 0xb5, 0xe7, 0x9b, 0x82, 0xe3, 0x4f, 0x16, 0xdc, 0xc9, 0x87, 0x63, 0x9e, 0xe0, + 0xfd, 0x4c, 0x4f, 0xc2, 0x32, 0x4b, 0x25, 0xcd, 0x3c, 0xc8, 0xe3, 0x84, 0xf1, 0x35, 0xcd, 0x24, + 0xe7, 0xaf, 0x05, 0x40, 0xfb, 0xaa, 0x58, 0xa8, 0x87, 0x6f, 0x13, 0x9a, 0x2b, 0xb3, 0x6f, 0x86, + 0x63, 0x17, 0xaf, 0x83, 0x63, 0x97, 0xae, 0xc2, 0xb1, 0x32, 0x11, 0x64, 0xd5, 0xe4, 0xc2, 0xef, + 0x0f, 0x14, 0x67, 0x2c, 0xba, 0x23, 0x83, 0xf3, 0x1a, 0x6e, 0x46, 0xb7, 0x4c, 0xf1, 0xe9, 0x5b, + 0x60, 0x93, 0xce, 0xcb, 0x42, 0x36, 0x2f, 0xa7, 0x20, 0xe4, 0xfc, 0xab, 0x00, 0x37, 0x9a, 0x11, + 0x05, 0x48, 0x06, 0x50, 0x24, 0x7e, 0x79, 0xda, 0x4e, 0x0e, 0x47, 0x82, 0x31, 0x8b, 0x13, 0x19, + 0x73, 0x31, 0xcd, 0x98, 0xe9, 0x0d, 0x2e, 0x65, 0x43, 0x78, 0x2d, 0x12, 0x07, 0x35, 0x60, 0x7d, + 0xc4, 0x80, 0x86, 0x00, 0x57, 0x14, 0x01, 0xae, 0x92, 0xe4, 0xe9, 0x39, 0x7a, 0x04, 0x6b, 0x31, + 0x5d, 0x05, 0x9a, 0xc5, 0x4a, 0x2a, 0x5c, 0x23, 0x6e, 0x0b, 0x24, 0x8d, 0x39, 0xff, 0xb6, 0xa0, + 0x1c, 0x67, 0xfb, 0x8c, 0x32, 0x32, 0x85, 0x6b, 0x21, 0x8b, 0xeb, 0x26, 0x54, 0x30, 0xf5, 0xdb, + 0x3d, 0xec, 0xa9, 0x3d, 0x29, 0x08, 0x4b, 0x6e, 0x59, 0xdb, 0xb4, 0x22, 0x7a, 0x02, 0xe5, 0x91, + 0x36, 0x8b, 0x12, 0xfa, 0xc1, 0x44, 0x71, 0x96, 0x0c, 0xaa, 0x0b, 0xb1, 0x48, 0xe3, 0xce, 0x1f, + 0x0a, 0x23, 0xce, 0xd0, 0x19, 0x37, 0x4f, 0x65, 0xf8, 0x15, 0x54, 0xcc, 0x29, 0xb4, 0x66, 0xd4, + 0xf5, 0xe1, 0xc7, 0x79, 0xdb, 0xca, 0x5b, 0x74, 0x2b, 0x01, 0xe3, 0xc7, 0x54, 0x84, 0x17, 0x6e, + 0x99, 0x8f, 0x2c, 0x75, 0x0f, 0xd6, 0xb3, 0x0e, 0x68, 0x1d, 0x8a, 0xe7, 0xf8, 0xc2, 0x60, 0x2c, + 0x7f, 0xca, 0x5a, 0xfa, 0x52, 0xc6, 0xde, 0x50, 0xe8, 0xbd, 0x4b, 0x8b, 0x53, 0x97, 0xb9, 0xda, + 0xfb, 0x27, 0x85, 0x8f, 0x2c, 0x87, 0xc1, 0xfa, 0x41, 0xc8, 0x06, 0x6f, 0x5d, 0x96, 0xa6, 0x4b, + 0xff, 0xfc, 0x6b, 0x22, 0x09, 0xfb, 0x00, 0xf3, 0x4e, 0x48, 0xda, 0x78, 0xce, 0x45, 0xc7, 0x08, + 0xfb, 0x2b, 0x0b, 0xde, 0xcb, 0xbc, 0x7b, 0x9e, 0xc8, 0xfe, 0x3c, 0x9d, 0x6f, 0x3a, 0xb0, 0x53, + 0x9a, 0x81, 0x64, 0x9e, 0xf9, 0x8a, 0x88, 0xd4, 0xb3, 0x3d, 0x79, 0xdf, 0x4f, 0x42, 0x76, 0xaa, + 0x64, 0xd6, 0xf5, 0x9d, 0xf8, 0xcf, 0x16, 0xdc, 0x9d, 0xb0, 0xc6, 0x3c, 0x27, 0xdf, 0x34, 0x75, + 0x07, 0x07, 0xba, 0x37, 0x30, 0x8d, 0x85, 0xb1, 0xa9, 0xfe, 0xe0, 0x2e, 0x80, 0x60, 0xc2, 0xef, + 0x69, 0x07, 0xd3, 0x59, 0x28, 0x8b, 0x7c, 0xec, 0xfc, 0xa3, 0x00, 0xd5, 0x96, 0x60, 0xa1, 0x7f, + 0x8a, 0xf7, 0x19, 0xed, 0x92, 0x53, 0x99, 0x11, 0x91, 0x14, 0xb5, 0xd4, 0x31, 0x62, 0xb1, 0xb9, + 0x09, 0x15, 0xbf, 0xd3, 0xc1, 0x9c, 0x7b, 0xe7, 0xf8, 0xc2, 0x24, 0x8c, 0xed, 0x96, 0xb5, 0xed, + 0x99, 0x34, 0x49, 0x11, 0xcf, 0x71, 0x27, 0xc4, 0xc2, 0x1b, 0x79, 0x9a, 0xa4, 0x5b, 0xd3, 0x0f, + 0x76, 0x23, 0x6f, 0xa9, 0x5d, 0x87, 0x1c, 0xb7, 0x5a, 0x9f, 0xaa, 0x62, 0x5b, 0x72, 0xcd, 0x48, + 0x2a, 0x87, 0xf6, 0xb0, 0x73, 0x8e, 0x45, 0xb2, 0xd8, 0x82, 0x36, 0xa9, 0x9c, 0xbd, 0x0d, 0x76, + 0xc8, 0x98, 0x50, 0x15, 0x52, 0xd1, 0x94, 0xed, 0x96, 0xa4, 0x41, 0x16, 0x11, 0xf3, 0xd6, 0xe6, + 0xee, 0x91, 0xea, 0x6d, 0xf4, 0x5b, 0x9b, 0xbb, 0x47, 0xb2, 0x05, 0x6b, 0xee, 0x1e, 0x7d, 0x4c, + 0x83, 0x01, 0x23, 0x54, 0xa8, 0x72, 0x69, 0xbb, 0x49, 0x93, 0x3c, 0x1e, 0xd7, 0x48, 0x78, 0x92, + 0x59, 0x55, 0x7f, 0x63, 0xbb, 0x65, 0x63, 0x7b, 0x7e, 0x31, 0xc0, 0xce, 0xff, 0x8a, 0xb0, 0xae, + 0xe5, 0xc1, 0x21, 0x6b, 0x47, 0xe9, 0x71, 0x07, 0xec, 0x4e, 0x6f, 0x28, 0x95, 0xb6, 0xc9, 0x0d, + 0xdb, 0x1d, 0x19, 0xd2, 0x6d, 0x8d, 0x37, 0x08, 0x71, 0x97, 0xbc, 0x36, 0xc8, 0x8d, 0xda, 0x9a, + 0x13, 0x65, 0x4e, 0xf2, 0x4f, 0x71, 0x8c, 0x7f, 0x02, 0x5f, 0xf8, 0x86, 0x14, 0x16, 0x15, 0x29, + 0xd8, 0xd2, 0xa2, 0xf9, 0x60, 0xac, 0x71, 0x5b, 0xca, 0x69, 0xdc, 0x12, 0xbc, 0xb7, 0x9c, 0xe6, + 0xbd, 0x74, 0xf2, 0xae, 0x64, 0x6b, 0xc4, 0x27, 0xb0, 0x1a, 0x01, 0xd3, 0x51, 0x39, 0xa2, 0xd0, + 0xcb, 0xe9, 0x00, 0x54, 0xf9, 0x4a, 0x26, 0x93, 0x5b, 0xe5, 0xa9, 0xdc, 0xca, 0xf2, 0xa4, 0x7d, + 0x25, 0x9e, 0xcc, 0x08, 0x26, 0xb8, 0x8a, 0x60, 0x4a, 0x76, 0xd4, 0xe5, 0x54, 0x47, 0xed, 0x7c, + 0x0a, 0xeb, 0xbf, 0x18, 0xe2, 0xf0, 0xe2, 0x90, 0xb5, 0xf9, 0x6c, 0x31, 0xae, 0x43, 0xc9, 0x04, + 0x2a, 0x12, 0x37, 0xf1, 0xd8, 0xf9, 0x8f, 0x05, 0x55, 0x75, 0xed, 0x9f, 0xfb, 0xfc, 0x3c, 0xfa, + 0xf8, 0x10, 0x45, 0xd9, 0x4a, 0x47, 0xf9, 0xea, 0x52, 0x3b, 0xd1, 0x39, 0x2b, 0xd9, 0x6f, 0x9b, + 0x02, 0xa7, 0x7a, 0xe6, 0x3c, 0xb9, 0xb0, 0x98, 0x27, 0x17, 0xb2, 0xa2, 0x7d, 0x69, 0x4c, 0xb4, + 0x7f, 0x6d, 0xc1, 0x8d, 0x04, 0x38, 0xf3, 0xd4, 0xae, 0x14, 0xa4, 0x85, 0x2c, 0xa4, 0x7b, 0xe9, + 0x9a, 0x5e, 0xcc, 0x8b, 0x71, 0xa2, 0xa6, 0x47, 0xe0, 0xa6, 0xea, 0xfa, 0x33, 0x58, 0x93, 0x94, + 0x79, 0x3d, 0x71, 0xfc, 0xaf, 0x05, 0x2b, 0x87, 0xac, 0xad, 0x22, 0x98, 0x4c, 0x1e, 0x2b, 0xfd, + 0x39, 0x66, 0x1d, 0x8a, 0x01, 0xe9, 0x9b, 0x42, 0x2c, 0x7f, 0xca, 0xcb, 0xc5, 0x85, 0x1f, 0x8a, + 0xd1, 0x07, 0x25, 0xa9, 0xa7, 0xa4, 0x45, 0x7d, 0x8f, 0x78, 0x1f, 0x4a, 0x98, 0x06, 0xfa, 0xa1, + 0x11, 0x9d, 0x98, 0x06, 0xea, 0xd1, 0xf5, 0x88, 0xfa, 0x5b, 0xb0, 0x34, 0x60, 0xa3, 0x8f, 0x40, + 0x7a, 0xe0, 0xdc, 0x02, 0xf4, 0x14, 0x8b, 0x43, 0xd6, 0x96, 0x51, 0x89, 0xe0, 0x71, 0xfe, 0x56, + 0x50, 0x1a, 0x7f, 0x64, 0x9e, 0x27, 0xc0, 0x0e, 0x54, 0x35, 0xf3, 0x7c, 0xc1, 0xda, 0x1e, 0x1d, + 0x46, 0xa0, 0x94, 0x95, 0xf1, 0x90, 0xb5, 0x8f, 0x87, 0x7d, 0xf4, 0x01, 0xdc, 0x24, 0xd4, 0x1b, + 0x18, 0x32, 0x8c, 0x3d, 0x35, 0x4a, 0xeb, 0x84, 0x46, 0x34, 0x69, 0xdc, 0x1f, 0xc2, 0x1a, 0xa6, + 0x2f, 0x86, 0x78, 0x88, 0x63, 0x57, 0x8d, 0x59, 0xd5, 0x98, 0x8d, 0x9f, 0x24, 0x3d, 0x9f, 0x9f, + 0x7b, 0xbc, 0xc7, 0x04, 0x37, 0xc5, 0xd0, 0x96, 0x96, 0x96, 0x34, 0xa0, 0x8f, 0xc0, 0x96, 0xd3, + 0x75, 0x6a, 0x69, 0xad, 0x7e, 0x3b, 0x2f, 0xb5, 0x4c, 0xbc, 0xdd, 0xd2, 0x17, 0xfa, 0x07, 0xdf, + 0xf9, 0xda, 0x06, 0x50, 0x09, 0xb7, 0xcf, 0x58, 0x18, 0xa0, 0x81, 0x42, 0x71, 0x9f, 0xf5, 0x07, + 0x8c, 0x62, 0x2a, 0xd4, 0xad, 0xe4, 0xe8, 0xf1, 0x84, 0x2f, 0x23, 0xe3, 0xae, 0x06, 0xf7, 0xfa, + 0xc3, 0x09, 0x33, 0x32, 0xee, 0xce, 0x02, 0x7a, 0xa1, 0x24, 0xb1, 0x1c, 0x12, 0x2e, 0x48, 0x87, + 0xef, 0x9f, 0xf9, 0x94, 0xe2, 0x1e, 0xda, 0x99, 0xbc, 0xe6, 0x98, 0x73, 0xb4, 0xea, 0xfd, 0xf4, + 0x1c, 0x33, 0x68, 0x89, 0x90, 0xd0, 0xd3, 0x28, 0xf4, 0xce, 0x02, 0x7a, 0x0e, 0xe5, 0x44, 0x4b, + 0x8c, 0x1e, 0xe6, 0x21, 0x35, 0xde, 0x33, 0xd7, 0x2f, 0xcb, 0x11, 0x67, 0x01, 0x75, 0xa1, 0x9a, + 0xfa, 0x66, 0x83, 0x1a, 0x97, 0x29, 0xf1, 0xe4, 0x87, 0x92, 0xfa, 0x77, 0x67, 0xf0, 0x8c, 0x77, + 0xff, 0x1b, 0x0d, 0xd8, 0xd8, 0x47, 0x8f, 0xed, 0x09, 0x2f, 0x99, 0xf4, 0x79, 0xa6, 0xfe, 0x78, + 0xf6, 0x09, 0xf1, 0xe2, 0xc1, 0xe8, 0x90, 0x2a, 0x7f, 0xd0, 0xa3, 0xe9, 0xed, 0x86, 0x5e, 0xad, + 0x31, 0x6b, 0x5f, 0xe2, 0x2c, 0xa0, 0x13, 0xb0, 0xe3, 0xd6, 0x00, 0x7d, 0x27, 0x6f, 0x62, 0xb6, + 0x73, 0x98, 0x21, 0x38, 0x29, 0x7d, 0x9e, 0x1f, 0x9c, 0xbc, 0xf6, 0x20, 0x3f, 0x38, 0xb9, 0x62, + 0xdf, 0x59, 0x40, 0xbf, 0x1d, 0x7d, 0xb8, 0x4b, 0xa9, 0x62, 0xf4, 0xf8, 0xb2, 0xe3, 0xe7, 0x89, + 0xf4, 0xfa, 0xf7, 0xdf, 0x62, 0x46, 0x22, 0x39, 0x50, 0xeb, 0x8c, 0xbd, 0xd2, 0xea, 0x64, 0x18, + 0xfa, 0x52, 0xcc, 0x4f, 0xbe, 0xbf, 0xe3, 0xae, 0x13, 0x17, 0xbf, 0x64, 0x46, 0xbc, 0xb8, 0x07, + 0xf0, 0x14, 0x8b, 0x23, 0x2c, 0x42, 0xd2, 0xe1, 0xd9, 0x6b, 0x65, 0x06, 0x23, 0x87, 0x68, 0xa9, + 0x47, 0x53, 0xfd, 0xa2, 0x05, 0x76, 0xfe, 0xb9, 0x6c, 0xfe, 0x41, 0x3a, 0x66, 0x01, 0x7e, 0x37, + 0x6a, 0xd5, 0x09, 0xd8, 0xb1, 0x3e, 0xcf, 0xbf, 0x0a, 0x59, 0xf9, 0x3e, 0xed, 0x2a, 0x7c, 0x0e, + 0x76, 0x2c, 0x78, 0xf2, 0xdf, 0x98, 0x15, 0x8b, 0xf5, 0x07, 0x53, 0xbc, 0xe2, 0xdd, 0x1e, 0x43, + 0x29, 0x12, 0x28, 0xe8, 0xfe, 0xa4, 0x7b, 0x9b, 0x7c, 0xf3, 0x94, 0xbd, 0xfe, 0x1a, 0xca, 0x09, + 0xf6, 0xce, 0xaf, 0xd4, 0xe3, 0xac, 0x5f, 0x7f, 0x34, 0xd5, 0xef, 0xdd, 0xb8, 0x30, 0x7b, 0x3f, + 0xf8, 0x7c, 0xe7, 0x94, 0x88, 0xb3, 0x61, 0x5b, 0x22, 0xbb, 0xad, 0x3d, 0x3f, 0x20, 0xcc, 0xfc, + 0xda, 0x8e, 0x76, 0xb9, 0xad, 0xde, 0xb4, 0xad, 0x70, 0x1a, 0xb4, 0xdb, 0xcb, 0x6a, 0xf8, 0xe1, + 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0x28, 0x07, 0xcf, 0x00, 0x04, 0x1e, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/internal/querycoordv2/observers/handoff_observer.go b/internal/querycoordv2/observers/handoff_observer.go index af3f00e754..98034c3789 100644 --- a/internal/querycoordv2/observers/handoff_observer.go +++ b/internal/querycoordv2/observers/handoff_observer.go @@ -241,6 +241,7 @@ func (ob *HandoffObserver) handoff(segment *querypb.SegmentInfo) { ID: segment.SegmentID, CollectionID: segment.CollectionID, PartitionID: segment.PartitionID, + NumOfRows: segment.NumRows, InsertChannel: segment.GetDmChannel(), State: segment.GetSegmentState(), CreatedByCompaction: segment.GetCreatedByCompaction(),