From bcf3b7426afdd296091941f446793aba32ce80b9 Mon Sep 17 00:00:00 2001 From: "cai.zhang" Date: Tue, 31 May 2022 16:36:03 +0800 Subject: [PATCH] Add distributed lock for segment refgerence (#16782) Signed-off-by: Cai.Zhang --- internal/datacoord/const.go | 27 + internal/datacoord/garbage_collector.go | 25 +- internal/datacoord/garbage_collector_test.go | 69 +- internal/datacoord/meta.go | 12 + internal/datacoord/meta_test.go | 23 + .../datacoord/segment_reference_manager.go | 244 ++++++ .../segment_reference_manager_test.go | 299 ++++++++ internal/datacoord/server.go | 89 ++- internal/datacoord/server_test.go | 241 +++++- internal/datacoord/services.go | 72 +- internal/datacoord/util.go | 8 + .../distributed/datacoord/client/client.go | 28 + .../datacoord/client/client_test.go | 6 + internal/distributed/datacoord/service.go | 10 + .../distributed/datacoord/service_test.go | 32 + .../indexcoord/client/client_test.go | 3 + internal/distributed/indexcoord/service.go | 39 +- .../distributed/indexcoord/service_test.go | 4 + internal/distributed/proxy/service_test.go | 8 + .../distributed/rootcoord/service_test.go | 21 +- internal/indexcoord/const.go | 27 + internal/indexcoord/index_coord.go | 110 ++- internal/indexcoord/index_coord_mock.go | 128 ++++ internal/indexcoord/index_coord_mock_test.go | 107 +++ internal/indexcoord/index_coord_test.go | 82 ++ internal/indexcoord/meta_table.go | 15 +- internal/indexcoord/task.go | 5 - internal/proto/data_coord.proto | 14 + internal/proto/datapb/data_coord.pb.go | 714 +++++++++++------- internal/proto/index_coord.proto | 1 + internal/proto/indexpb/index_coord.pb.go | 146 ++-- internal/proxy/data_coord_mock_test.go | 14 + internal/querycoord/global_meta_broker.go | 54 ++ .../querycoord/mock_3rd_component_test.go | 37 + internal/querycoord/task.go | 39 +- internal/querycoord/task_scheduler.go | 7 +- internal/querycoord/task_test.go | 88 ++- internal/rootcoord/root_coord.go | 180 ++--- internal/rootcoord/root_coord_test.go | 67 +- internal/rootcoord/task.go | 4 +- internal/types/types.go | 5 + internal/util/mock/datacoord_client.go | 140 ++++ internal/util/mock/grpc_datacoord_client.go | 8 + 43 files changed, 2701 insertions(+), 551 deletions(-) create mode 100644 internal/datacoord/const.go create mode 100644 internal/datacoord/segment_reference_manager.go create mode 100644 internal/datacoord/segment_reference_manager_test.go create mode 100644 internal/indexcoord/const.go create mode 100644 internal/util/mock/datacoord_client.go diff --git a/internal/datacoord/const.go b/internal/datacoord/const.go new file mode 100644 index 0000000000..876b114981 --- /dev/null +++ b/internal/datacoord/const.go @@ -0,0 +1,27 @@ +// 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 datacoord + +// segment reference lock +const ( + // segmentReferPrefix is the prefix of the segment reference lock path + segmentReferPrefix = "segmentRefer" +) + +const ( + moduleName = "DataCoord" +) diff --git a/internal/datacoord/garbage_collector.go b/internal/datacoord/garbage_collector.go index d8621ef1d4..f3665beb4a 100644 --- a/internal/datacoord/garbage_collector.go +++ b/internal/datacoord/garbage_collector.go @@ -50,8 +50,9 @@ type GcOption struct { // garbageCollector handles garbage files in object storage // which could be dropped collection remanent or data node failure traces type garbageCollector struct { - option GcOption - meta *meta + option GcOption + meta *meta + segRefer *SegmentReferenceManager startOnce sync.Once stopOnce sync.Once @@ -60,13 +61,14 @@ type garbageCollector struct { } // newGarbageCollector create garbage collector with meta and option -func newGarbageCollector(meta *meta, opt GcOption) *garbageCollector { +func newGarbageCollector(meta *meta, segRefer *SegmentReferenceManager, opt GcOption) *garbageCollector { log.Info("GC with option", zap.Bool("enabled", opt.enabled), zap.Duration("interval", opt.checkInterval), zap.Duration("missingTolerance", opt.missingTolerance), zap.Duration("dropTolerance", opt.dropTolerance)) return &garbageCollector{ - meta: meta, - option: opt, - closeCh: make(chan struct{}), + meta: meta, + segRefer: segRefer, + option: opt, + closeCh: make(chan struct{}), } } @@ -134,6 +136,15 @@ func (gc *garbageCollector) scan() { v++ continue } + + // binlog path should consist of "/files/insertLog/collID/partID/segID/fieldID/fileName" + segmentID, err := parseSegmentIDByBinlog(info.Key) + if err == nil { + if gc.segRefer.HasSegmentLock(segmentID) { + v++ + continue + } + } m++ // not found in meta, check last modified time exceeds tolerance duration if time.Since(info.LastModified) > gc.option.missingTolerance { @@ -148,7 +159,7 @@ func (gc *garbageCollector) scan() { func (gc *garbageCollector) clearEtcd() { drops := gc.meta.SelectSegments(func(segment *SegmentInfo) bool { - return segment.GetState() == commonpb.SegmentState_Dropped + return segment.GetState() == commonpb.SegmentState_Dropped && !gc.segRefer.HasSegmentLock(segment.ID) }) for _, sinfo := range drops { diff --git a/internal/datacoord/garbage_collector_test.go b/internal/datacoord/garbage_collector_test.go index 7fb6239cae..d6b995a6c4 100644 --- a/internal/datacoord/garbage_collector_test.go +++ b/internal/datacoord/garbage_collector_test.go @@ -20,12 +20,15 @@ import ( "bytes" "context" "path" + "strconv" "strings" "testing" "time" + etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/util/etcd" "github.com/milvus-io/milvus/internal/util/funcutil" "github.com/minio/minio-go/v7" "github.com/minio/minio-go/v7/pkg/credentials" @@ -44,8 +47,15 @@ func Test_garbageCollector_basic(t *testing.T) { meta, err := newMemoryMeta(mockAllocator) assert.Nil(t, err) + etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) + assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) + segRefer, err := NewSegmentReferenceManager(etcdKV, nil) + assert.NoError(t, err) + assert.NotNil(t, segRefer) + t.Run("normal gc", func(t *testing.T) { - gc := newGarbageCollector(meta, GcOption{ + gc := newGarbageCollector(meta, segRefer, GcOption{ cli: cli, enabled: true, checkInterval: time.Millisecond * 10, @@ -63,7 +73,7 @@ func Test_garbageCollector_basic(t *testing.T) { }) t.Run("with nil cli", func(t *testing.T) { - gc := newGarbageCollector(meta, GcOption{ + gc := newGarbageCollector(meta, segRefer, GcOption{ cli: nil, enabled: true, checkInterval: time.Millisecond * 10, @@ -102,8 +112,50 @@ func Test_garbageCollector_scan(t *testing.T) { meta, err := newMemoryMeta(mockAllocator) assert.Nil(t, err) + etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) + assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) + segRefer, err := NewSegmentReferenceManager(etcdKV, nil) + assert.NoError(t, err) + assert.NotNil(t, segRefer) + + t.Run("key is reference", func(t *testing.T) { + segReferManager := &SegmentReferenceManager{ + etcdKV: etcdKV, + segmentsLock: map[UniqueID][]*SegmentLock{ + 2: { + { + segmentID: 2, + nodeID: 1, + locKey: "path", + }, + }, + }, + } + gc := newGarbageCollector(meta, segRefer, GcOption{ + cli: cli, + enabled: true, + checkInterval: time.Minute * 30, + missingTolerance: time.Hour * 24, + dropTolerance: time.Hour * 24, + bucketName: bucketName, + rootPath: rootPath, + }) + gc.segRefer = segReferManager + gc.scan() + + validateMinioPrefixElements(t, cli, bucketName, path.Join(rootPath, insertLogPrefix), inserts) + validateMinioPrefixElements(t, cli, bucketName, path.Join(rootPath, statsLogPrefix), stats) + validateMinioPrefixElements(t, cli, bucketName, path.Join(rootPath, deltaLogPrefix), delta) + validateMinioPrefixElements(t, cli, bucketName, path.Join(rootPath, `indexes`), others) + + err = gc.segRefer.ReleaseSegmentsLock([]UniqueID{2}, 1) + assert.NoError(t, err) + gc.close() + }) + t.Run("missing all but save tolerance", func(t *testing.T) { - gc := newGarbageCollector(meta, GcOption{ + gc := newGarbageCollector(meta, segRefer, GcOption{ cli: cli, enabled: true, checkInterval: time.Minute * 30, @@ -130,7 +182,7 @@ func Test_garbageCollector_scan(t *testing.T) { err = meta.AddSegment(segment) require.NoError(t, err) - gc := newGarbageCollector(meta, GcOption{ + gc := newGarbageCollector(meta, segRefer, GcOption{ cli: cli, enabled: true, checkInterval: time.Minute * 30, @@ -160,7 +212,7 @@ func Test_garbageCollector_scan(t *testing.T) { err = meta.AddSegment(segment) require.NoError(t, err) - gc := newGarbageCollector(meta, GcOption{ + gc := newGarbageCollector(meta, segRefer, GcOption{ cli: cli, enabled: true, checkInterval: time.Minute * 30, @@ -178,7 +230,7 @@ func Test_garbageCollector_scan(t *testing.T) { gc.close() }) t.Run("missing gc all", func(t *testing.T) { - gc := newGarbageCollector(meta, GcOption{ + gc := newGarbageCollector(meta, segRefer, GcOption{ cli: cli, enabled: true, checkInterval: time.Minute * 30, @@ -229,7 +281,10 @@ func initUtOSSEnv(bucket, root string, n int) (cli *minio.Client, inserts []stri content := []byte("test") for i := 0; i < n; i++ { reader := bytes.NewReader(content) - token := funcutil.RandomString(8) + token := path.Join(funcutil.RandomString(8), funcutil.RandomString(8), strconv.Itoa(i), funcutil.RandomString(8), funcutil.RandomString(8)) + if i == 1 { + token = path.Join(funcutil.RandomString(8), funcutil.RandomString(8), strconv.Itoa(i), funcutil.RandomString(8)) + } // insert filePath := path.Join(root, insertLogPrefix, token) info, err := cli.PutObject(context.TODO(), bucket, filePath, reader, int64(len(content)), minio.PutObjectOptions{}) diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 22f98246c3..3225722395 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -1052,3 +1052,15 @@ func isSegmentHealthy(segment *SegmentInfo) bool { segment.GetState() != commonpb.SegmentState_NotExist && segment.GetState() != commonpb.SegmentState_Dropped } + +func (m *meta) HasSegments(segIDs []UniqueID) (bool, error) { + m.RLock() + defer m.RUnlock() + + for _, segID := range segIDs { + if _, ok := m.segments.segments[segID]; !ok { + return false, fmt.Errorf("segment is not exist with ID = %d", segID) + } + } + return true, nil +} diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index a9ff83f755..0be912cb4f 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -698,3 +698,26 @@ func Test_meta_GetSegmentsOfCollection(t *testing.T) { }) } } + +func TestMeta_HasSegments(t *testing.T) { + m := &meta{ + segments: &SegmentsInfo{ + segments: map[UniqueID]*SegmentInfo{ + 1: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + }, + currRows: 100, + }, + }, + }, + } + + has, err := m.HasSegments([]UniqueID{1}) + assert.Equal(t, true, has) + assert.NoError(t, err) + + has, err = m.HasSegments([]UniqueID{2}) + assert.Equal(t, false, has) + assert.Error(t, err) +} diff --git a/internal/datacoord/segment_reference_manager.go b/internal/datacoord/segment_reference_manager.go new file mode 100644 index 0000000000..cbe6de1566 --- /dev/null +++ b/internal/datacoord/segment_reference_manager.go @@ -0,0 +1,244 @@ +// 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 datacoord + +import ( + "fmt" + "path" + "strconv" + "strings" + "sync" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/kv" + "github.com/milvus-io/milvus/internal/log" +) + +type SegmentLock struct { + segmentID UniqueID + nodeID UniqueID + locKey string +} + +type SegmentReferenceManager struct { + etcdKV kv.BaseKV + + segmentsLock map[UniqueID][]*SegmentLock + lock sync.RWMutex +} + +func parseLockKey(key string) (segID UniqueID, nodeID UniqueID, err error) { + ss := strings.Split(key, "/") + // segment lock key consists of at least "meta/segmentRefer/nodeID/segID" + if len(ss) < 4 { + return 0, 0, fmt.Errorf("segment lock key is invalid with %s", key) + } + segID, err = strconv.ParseInt(ss[len(ss)-1], 10, 64) + if err != nil { + return 0, 0, err + } + nodeID, err = strconv.ParseInt(ss[len(ss)-2], 10, 64) + return segID, nodeID, err +} + +func NewSegmentReferenceManager(etcdKV kv.BaseKV, onlineIDs []UniqueID) (*SegmentReferenceManager, error) { + log.Info("New segment reference manager") + segReferManager := &SegmentReferenceManager{ + etcdKV: etcdKV, + segmentsLock: make(map[UniqueID][]*SegmentLock), + } + keys, _, err := segReferManager.etcdKV.LoadWithPrefix(segmentReferPrefix) + if err != nil { + log.Error("load segments lock from etcd failed", zap.Error(err)) + return nil, err + } + + for _, key := range keys { + segID, nodeID, err := parseLockKey(key) + if err != nil { + log.Error("parse segment lock key failed", zap.String("lock key", key), zap.Error(err)) + return nil, err + } + segLock := &SegmentLock{ + segmentID: segID, + nodeID: nodeID, + locKey: key, + } + segReferManager.segmentsLock[segID] = append(segReferManager.segmentsLock[segID], segLock) + } + + err = segReferManager.recoverySegReferManager(onlineIDs) + if err != nil { + log.Error("Recovery segment reference manager failed", zap.Error(err)) + return nil, err + } + + log.Info("New segment reference manager successfully") + return segReferManager, nil +} + +func (srm *SegmentReferenceManager) AddSegmentsLock(segIDs []UniqueID, nodeID UniqueID) error { + srm.lock.Lock() + defer srm.lock.Unlock() + log.Info("Add reference lock on segments", zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID)) + locKVs := make(map[string]string) + segID2SegmentLock := make(map[UniqueID][]*SegmentLock) + for _, segID := range segIDs { + locKey := path.Join(segmentReferPrefix, strconv.FormatInt(nodeID, 10), strconv.FormatInt(segID, 10)) + locKVs[locKey] = strconv.FormatInt(nodeID, 10) + segLock := &SegmentLock{ + segmentID: segID, + nodeID: nodeID, + locKey: locKey, + } + segID2SegmentLock[segID] = append(segID2SegmentLock[segID], segLock) + } + if err := srm.etcdKV.MultiSave(locKVs); err != nil { + log.Error("AddSegmentsLock save segment lock to etcd failed", zap.Int64s("segIDs", segIDs), zap.Error(err)) + return err + } + for segID, segLocks := range segID2SegmentLock { + srm.segmentsLock[segID] = append(srm.segmentsLock[segID], segLocks...) + } + log.Info("Add reference lock on segments successfully", zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID)) + return nil +} + +func (srm *SegmentReferenceManager) ReleaseSegmentsLock(segIDs []UniqueID, nodeID UniqueID) error { + srm.lock.Lock() + defer srm.lock.Unlock() + + log.Info("Release reference lock on segments", zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID)) + locKeys := make([]string, 0) + for _, segID := range segIDs { + if _, ok := srm.segmentsLock[segID]; !ok { + continue + } + for _, segLock := range srm.segmentsLock[segID] { + if segLock.nodeID == nodeID { + locKeys = append(locKeys, segLock.locKey) + } + } + } + if err := srm.etcdKV.MultiRemove(locKeys); err != nil { + log.Error("Remove reference lock paths on segments failed", zap.Int64s("segIDs", segIDs), + zap.Int64("nodeID", nodeID), zap.Error(err)) + return err + } + + for _, segID := range segIDs { + if _, ok := srm.segmentsLock[segID]; !ok { + continue + } + for i := 0; i < len(srm.segmentsLock[segID]); i++ { + segLock := srm.segmentsLock[segID][i] + if segLock.nodeID == nodeID { + srm.segmentsLock[segID] = append(srm.segmentsLock[segID][:i], srm.segmentsLock[segID][i+1:]...) + i-- + } + } + if len(srm.segmentsLock[segID]) == 0 { + delete(srm.segmentsLock, segID) + } + } + log.Info("Release reference lock on segments successfully", zap.Int64s("segIDs", segIDs), zap.Int64("nodeID", nodeID)) + return nil +} + +func (srm *SegmentReferenceManager) ReleaseSegmentsLockByNodeID(nodeID UniqueID) error { + srm.lock.Lock() + defer srm.lock.Unlock() + + log.Info("Release reference lock on segments by node", zap.Int64("nodeID", nodeID)) + locKeys := make([]string, 0) + for segID := range srm.segmentsLock { + for _, segLock := range srm.segmentsLock[segID] { + if segLock.nodeID == nodeID { + locKeys = append(locKeys, segLock.locKey) + } + } + } + if err := srm.etcdKV.MultiRemove(locKeys); err != nil { + log.Error("Remove reference lock paths on segments by node failed", + zap.Int64("nodeID", nodeID), zap.Error(err)) + return err + } + + for segID := range srm.segmentsLock { + for i := 0; i < len(srm.segmentsLock[segID]); i++ { + segLock := srm.segmentsLock[segID][i] + if segLock.nodeID == nodeID { + srm.segmentsLock[segID] = append(srm.segmentsLock[segID][:i], srm.segmentsLock[segID][i+1:]...) + i-- + } + } + if len(srm.segmentsLock[segID]) == 0 { + delete(srm.segmentsLock, segID) + } + } + log.Info("Release reference lock on segments by node successfully", zap.Int64("nodeID", nodeID)) + return nil +} + +func (srm *SegmentReferenceManager) recoverySegReferManager(nodeIDs []UniqueID) error { + log.Info("Recovery reference lock on segments by online nodes", zap.Int64s("online nodeIDs", nodeIDs)) + offlineIDs := make(map[UniqueID]struct{}) + for segID := range srm.segmentsLock { + for _, segLock := range srm.segmentsLock[segID] { + alive := false + for _, nodeID := range nodeIDs { + if segLock.nodeID == nodeID { + alive = true + break + } + } + if !alive { + offlineIDs[segLock.nodeID] = struct{}{} + } + } + } + for nodeID := range offlineIDs { + if err := srm.ReleaseSegmentsLockByNodeID(nodeID); err != nil { + log.Error("Remove reference lock on segments by offline node failed", + zap.Int64("offline nodeID", nodeID), zap.Error(err)) + return err + } + } + log.Info("Recovery reference lock on segments by online nodes successfully", zap.Int64s("online nodeIDs", nodeIDs), + zap.Any("offline nodeIDs", offlineIDs)) + return nil +} + +func (srm *SegmentReferenceManager) HasSegmentLock(segID UniqueID) bool { + srm.lock.RLock() + defer srm.lock.RUnlock() + + _, ok := srm.segmentsLock[segID] + return ok +} + +func (srm *SegmentReferenceManager) GetHasReferLockSegmentIDs() []UniqueID { + srm.lock.RLock() + defer srm.lock.RUnlock() + + segIDs := make([]UniqueID, 0) + for segID := range srm.segmentsLock { + segIDs = append(segIDs, segID) + } + return segIDs +} diff --git a/internal/datacoord/segment_reference_manager_test.go b/internal/datacoord/segment_reference_manager_test.go new file mode 100644 index 0000000000..ed8e9caec0 --- /dev/null +++ b/internal/datacoord/segment_reference_manager_test.go @@ -0,0 +1,299 @@ +// 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 datacoord + +import ( + "errors" + "path" + "strconv" + "testing" + + "github.com/milvus-io/milvus/internal/kv" + + etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" + "github.com/milvus-io/milvus/internal/util/etcd" + "github.com/stretchr/testify/assert" +) + +func Test_SegmentReferenceManager(t *testing.T) { + var segRefer *SegmentReferenceManager + var err error + Params.Init() + etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) + assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, "unittest") + + t.Run("NewSegmentReferenceManager", func(t *testing.T) { + var segRefer *SegmentReferenceManager + var err error + var locKey string + nodeID := int64(1) + locKey = path.Join(segmentReferPrefix, strconv.FormatInt(nodeID, 10), strconv.FormatInt(2, 10)) + err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10)) + assert.NoError(t, err) + + segRefer, err = NewSegmentReferenceManager(etcdKV, []UniqueID{nodeID}) + assert.NoError(t, err) + assert.NotNil(t, segRefer) + err = etcdKV.Remove(locKey) + assert.NoError(t, err) + + locKey = path.Join(segmentReferPrefix, strconv.FormatInt(nodeID, 10), "segID") + err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10)) + assert.NoError(t, err) + segRefer, err = NewSegmentReferenceManager(etcdKV, []UniqueID{nodeID}) + assert.Error(t, err) + assert.Nil(t, segRefer) + err = etcdKV.Remove(locKey) + assert.NoError(t, err) + + locKey = path.Join(segmentReferPrefix, "nodeID", strconv.FormatInt(3, 10)) + err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10)) + assert.NoError(t, err) + segRefer, err = NewSegmentReferenceManager(etcdKV, []UniqueID{nodeID}) + assert.Error(t, err) + assert.Nil(t, segRefer) + err = etcdKV.Remove(locKey) + assert.NoError(t, err) + + locKey = path.Join(segmentReferPrefix, "nodeID") + err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10)) + assert.NoError(t, err) + segRefer, err = NewSegmentReferenceManager(etcdKV, nil) + assert.Error(t, err) + assert.Nil(t, segRefer) + err = etcdKV.Remove(locKey) + assert.NoError(t, err) + + locKey = path.Join(segmentReferPrefix, strconv.FormatInt(nodeID, 10), strconv.FormatInt(2, 10)) + err = etcdKV.Save(locKey, strconv.FormatInt(nodeID, 10)) + assert.NoError(t, err) + segRefer, err = NewSegmentReferenceManager(etcdKV, nil) + assert.NoError(t, err) + assert.NotNil(t, segRefer) + has := segRefer.HasSegmentLock(2) + assert.False(t, has) + }) + + segIDs := []UniqueID{1, 2, 3, 4, 5} + nodeID := UniqueID(1) + segRefer, err = NewSegmentReferenceManager(etcdKV, nil) + assert.NoError(t, err) + assert.NotNil(t, segRefer) + var has bool + + t.Run("AddSegmentsLock", func(t *testing.T) { + err = segRefer.AddSegmentsLock(segIDs, nodeID) + assert.NoError(t, err) + + for _, segID := range segIDs { + has = segRefer.HasSegmentLock(segID) + assert.True(t, has) + } + }) + + t.Run("ReleaseSegmentsLock", func(t *testing.T) { + err = segRefer.ReleaseSegmentsLock(segIDs, nodeID) + assert.NoError(t, err) + + for _, segID := range segIDs { + has = segRefer.HasSegmentLock(segID) + assert.False(t, has) + } + + err = segRefer.ReleaseSegmentsLock([]UniqueID{6}, nodeID) + assert.NoError(t, err) + + has = segRefer.HasSegmentLock(6) + assert.False(t, has) + }) + + t.Run("ReleaseSegmentsLockByNodeID", func(t *testing.T) { + segIDs = []UniqueID{10, 11, 12, 13, 14, 15} + nodeID = 2 + err = segRefer.AddSegmentsLock(segIDs, nodeID) + assert.NoError(t, err) + + for _, segID := range segIDs { + has = segRefer.HasSegmentLock(segID) + assert.True(t, has) + } + + err = segRefer.ReleaseSegmentsLockByNodeID(UniqueID(2)) + assert.NoError(t, err) + + for _, segID := range segIDs { + has = segRefer.HasSegmentLock(segID) + assert.False(t, has) + } + + err = segRefer.ReleaseSegmentsLockByNodeID(UniqueID(11)) + assert.NoError(t, err) + }) + + t.Run("RecoverySegReferManager", func(t *testing.T) { + segIDs = []UniqueID{16, 17, 18, 19, 20} + err = segRefer.AddSegmentsLock(segIDs, UniqueID(3)) + assert.NoError(t, err) + + for _, segID := range segIDs { + has = segRefer.HasSegmentLock(segID) + assert.True(t, has) + } + + segIDs2 := []UniqueID{21, 22, 23, 24, 25} + err = segRefer.AddSegmentsLock(segIDs2, UniqueID(4)) + assert.NoError(t, err) + + for _, segID := range segIDs2 { + has = segRefer.HasSegmentLock(segID) + assert.True(t, has) + } + + err = segRefer.recoverySegReferManager([]int64{4, 5}) + assert.NoError(t, err) + + for _, segID := range segIDs { + has = segRefer.HasSegmentLock(segID) + assert.False(t, has) + } + + err = segRefer.ReleaseSegmentsLockByNodeID(4) + assert.NoError(t, err) + + for _, segID := range segIDs2 { + has = segRefer.HasSegmentLock(segID) + assert.False(t, has) + } + }) + + t.Run("HasSegmentLock", func(t *testing.T) { + exist := segRefer.HasSegmentLock(UniqueID(1)) + assert.False(t, exist) + }) + + t.Run("GetHasReferLockSegmentIDs", func(t *testing.T) { + segIDs = []UniqueID{26, 27, 28, 29, 30} + err = segRefer.AddSegmentsLock(segIDs, UniqueID(5)) + assert.NoError(t, err) + + for _, segID := range segIDs { + has = segRefer.HasSegmentLock(segID) + assert.True(t, has) + } + + segmentIDs := segRefer.GetHasReferLockSegmentIDs() + assert.Equal(t, 5, len(segmentIDs)) + assert.ElementsMatch(t, segIDs, segmentIDs) + + err = segRefer.ReleaseSegmentsLockByNodeID(UniqueID(5)) + assert.NoError(t, err) + + for _, segID := range segIDs { + has = segRefer.HasSegmentLock(segID) + assert.False(t, has) + } + + segIDs = segRefer.GetHasReferLockSegmentIDs() + assert.Equal(t, 0, len(segIDs)) + }) +} + +type etcdKVMock struct { + kv.BaseKV + + Fail int +} + +func (em *etcdKVMock) MultiSave(data map[string]string) error { + if em.Fail > 0 { + return errors.New("error occurred") + } + return nil +} + +func (em *etcdKVMock) MultiRemove(keys []string) error { + if em.Fail > 0 { + return errors.New("error occurred") + } + return nil +} + +func (em *etcdKVMock) LoadWithPrefix(prefix string) ([]string, []string, error) { + if em.Fail > 2 { + return nil, nil, errors.New("error occurs") + } + if em.Fail > 1 { + return []string{"key"}, []string{"value"}, nil + } + return []string{"meta/segmentRefer/1/2"}, []string{"1"}, nil +} + +func TestSegmentReferenceManager_Error(t *testing.T) { + emKV := &etcdKVMock{ + Fail: 3, + } + + t.Run("NewSegmentReferenceManager", func(t *testing.T) { + segRefer, err := NewSegmentReferenceManager(emKV, nil) + assert.Error(t, err) + assert.Nil(t, segRefer) + + emKV2 := &etcdKVMock{Fail: 2} + segRefer, err = NewSegmentReferenceManager(emKV2, nil) + assert.Error(t, err) + assert.Nil(t, segRefer) + + emKV3 := &etcdKVMock{Fail: 1} + segRefer, err = NewSegmentReferenceManager(emKV3, nil) + assert.Error(t, err) + assert.Nil(t, segRefer) + }) + + segRefer := &SegmentReferenceManager{ + etcdKV: emKV, + } + + t.Run("AddSegmentsLock", func(t *testing.T) { + err := segRefer.AddSegmentsLock([]UniqueID{1}, 1) + assert.Error(t, err) + }) + + t.Run("ReleaseSegmentsLock", func(t *testing.T) { + err := segRefer.ReleaseSegmentsLock([]UniqueID{1}, 1) + assert.Error(t, err) + }) + + t.Run("ReleaseSegmentsLockByNodeID", func(t *testing.T) { + err := segRefer.ReleaseSegmentsLockByNodeID(1) + assert.Error(t, err) + }) + + t.Run("recoverySegReferManager", func(t *testing.T) { + segRefer.segmentsLock = map[UniqueID][]*SegmentLock{ + 2: { + { + segmentID: 2, + nodeID: 2, + locKey: "1/2/3", + }, + }, + } + err := segRefer.recoverySegReferManager([]UniqueID{1}) + assert.Error(t, err) + }) +} diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 93310aadae..8cc106ab4b 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -127,11 +127,15 @@ type Server struct { flushCh chan UniqueID factory dependency.Factory - session *sessionutil.Session - eventCh <-chan *sessionutil.SessionEvent + session *sessionutil.Session + dnEventCh <-chan *sessionutil.SessionEvent + icEventCh <-chan *sessionutil.SessionEvent + qcEventCh <-chan *sessionutil.SessionEvent dataNodeCreator dataNodeCreatorFunc rootCoordClientCreator rootCoordCreatorFunc + + segReferManager *SegmentReferenceManager } // ServerHelper datacoord server injection helper @@ -380,7 +384,7 @@ func (s *Server) initGarbageCollection() error { } } - s.garbageCollector = newGarbageCollector(s.meta, GcOption{ + s.garbageCollector = newGarbageCollector(s.meta, s.segReferManager, GcOption{ cli: cli, enabled: Params.DataCoordCfg.EnableGarbageCollection, bucketName: Params.MinioCfg.BucketName, @@ -413,8 +417,31 @@ func (s *Server) initServiceDiscovery() error { s.cluster.Startup(s.ctx, datanodes) // TODO implement rewatch logic - s.eventCh = s.session.WatchServices(typeutil.DataNodeRole, rev+1, nil) - return nil + s.dnEventCh = s.session.WatchServices(typeutil.DataNodeRole, rev+1, nil) + + icSessions, icRevision, err := s.session.GetSessions(typeutil.IndexCoordRole) + if err != nil { + log.Error("DataCoord get IndexCoord session failed", zap.Error(err)) + return err + } + serverIDs := make([]UniqueID, 0, len(icSessions)) + for _, session := range icSessions { + serverIDs = append(serverIDs, session.ServerID) + } + s.icEventCh = s.session.WatchServices(typeutil.IndexCoordRole, icRevision+1, nil) + + qcSessions, qcRevision, err := s.session.GetSessions(typeutil.QueryCoordRole) + if err != nil { + log.Error("DataCoord get QueryCoord session failed", zap.Error(err)) + return err + } + for _, session := range qcSessions { + serverIDs = append(serverIDs, session.ServerID) + } + s.qcEventCh = s.session.WatchServices(typeutil.QueryCoordRole, qcRevision+1, nil) + + s.segReferManager, err = NewSegmentReferenceManager(s.kvClient, serverIDs) + return err } func (s *Server) startSegmentManager() { @@ -628,7 +655,7 @@ func (s *Server) watchService(ctx context.Context) { case <-ctx.Done(): log.Info("watch service shutdown") return - case event, ok := <-s.eventCh: + case event, ok := <-s.dnEventCh: if !ok { // ErrCompacted in handled inside SessionWatcher // So there is some other error occurred, closing DataCoord server @@ -649,6 +676,56 @@ func (s *Server) watchService(ctx context.Context) { }() return } + case event, ok := <-s.icEventCh: + if !ok { + // ErrCompacted in handled inside SessionWatcher + // So there is some other error occurred, closing DataCoord server + logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", s.session.ServerID)) + go s.Stop() + if s.session.TriggerKill { + if p, err := os.FindProcess(os.Getpid()); err == nil { + p.Signal(syscall.SIGINT) + } + } + return + } + switch event.EventType { + case sessionutil.SessionAddEvent: + log.Info("there is a new IndexCoord online", zap.Int64("serverID", event.Session.ServerID)) + + case sessionutil.SessionDelEvent: + log.Warn("there is IndexCoord offline", zap.Int64("serverID", event.Session.ServerID)) + if err := retry.Do(ctx, func() error { + return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID) + }, retry.Attempts(100)); err != nil { + panic(err) + } + } + case event, ok := <-s.qcEventCh: + if !ok { + // ErrCompacted in handled inside SessionWatcher + // So there is some other error occurred, closing DataCoord server + logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", s.session.ServerID)) + go s.Stop() + if s.session.TriggerKill { + if p, err := os.FindProcess(os.Getpid()); err == nil { + p.Signal(syscall.SIGINT) + } + } + return + } + switch event.EventType { + case sessionutil.SessionAddEvent: + log.Info("there is a new QueryCoord online", zap.Int64("serverID", event.Session.ServerID)) + + case sessionutil.SessionDelEvent: + log.Warn("there is QueryCoord offline", zap.Int64("serverID", event.Session.ServerID)) + if err := retry.Do(ctx, func() error { + return s.segReferManager.ReleaseSegmentsLockByNodeID(event.Session.ServerID) + }, retry.Attempts(100)); err != nil { + panic(err) + } + } } } } diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 91b607e780..ff904ddd9a 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -30,6 +30,8 @@ import ( "testing" "time" + "github.com/milvus-io/milvus/internal/util/typeutil" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -38,6 +40,7 @@ import ( "github.com/milvus-io/milvus/internal/common" "github.com/milvus-io/milvus/internal/kv" + etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/mq/msgstream" "github.com/milvus-io/milvus/internal/proto/commonpb" @@ -627,7 +630,7 @@ func TestService_WatchServices(t *testing.T) { svr.serverLoopWg.Add(1) ech := make(chan *sessionutil.SessionEvent) - svr.eventCh = ech + svr.dnEventCh = ech flag := false closed := false @@ -654,7 +657,7 @@ func TestService_WatchServices(t *testing.T) { ech = make(chan *sessionutil.SessionEvent) flag = false - svr.eventCh = ech + svr.dnEventCh = ech ctx, cancel := context.WithCancel(context.Background()) svr.serverLoopWg.Add(1) @@ -670,6 +673,128 @@ func TestService_WatchServices(t *testing.T) { assert.True(t, flag) } +func TestServer_watchCoord(t *testing.T) { + Params.Init() + etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) + assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) + assert.NotNil(t, etcdKV) + factory := dependency.NewDefaultFactory(true) + svr := CreateServer(context.TODO(), factory) + svr.session = &sessionutil.Session{ + TriggerKill: true, + } + svr.kvClient = etcdKV + + dnCh := make(chan *sessionutil.SessionEvent) + icCh := make(chan *sessionutil.SessionEvent) + qcCh := make(chan *sessionutil.SessionEvent) + + svr.dnEventCh = dnCh + svr.icEventCh = icCh + svr.qcEventCh = qcCh + + segRefer, err := NewSegmentReferenceManager(etcdKV, nil) + assert.NoError(t, err) + assert.NotNil(t, segRefer) + svr.segReferManager = segRefer + + sc := make(chan os.Signal, 1) + signal.Notify(sc, syscall.SIGINT) + defer signal.Reset(syscall.SIGINT) + closed := false + sigQuit := make(chan struct{}, 1) + + svr.serverLoopWg.Add(1) + go func() { + svr.watchService(context.Background()) + }() + + go func() { + <-sc + closed = true + sigQuit <- struct{}{} + }() + + icCh <- &sessionutil.SessionEvent{ + EventType: sessionutil.SessionAddEvent, + Session: &sessionutil.Session{ + ServerID: 1, + }, + } + icCh <- &sessionutil.SessionEvent{ + EventType: sessionutil.SessionDelEvent, + Session: &sessionutil.Session{ + ServerID: 1, + }, + } + close(icCh) + <-sigQuit + svr.serverLoopWg.Wait() + assert.True(t, closed) +} + +func TestServer_watchQueryCoord(t *testing.T) { + Params.Init() + etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) + assert.Nil(t, err) + etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath) + assert.NotNil(t, etcdKV) + factory := dependency.NewDefaultFactory(true) + svr := CreateServer(context.TODO(), factory) + svr.session = &sessionutil.Session{ + TriggerKill: true, + } + svr.kvClient = etcdKV + + dnCh := make(chan *sessionutil.SessionEvent) + icCh := make(chan *sessionutil.SessionEvent) + qcCh := make(chan *sessionutil.SessionEvent) + + svr.dnEventCh = dnCh + svr.icEventCh = icCh + svr.qcEventCh = qcCh + + segRefer, err := NewSegmentReferenceManager(etcdKV, nil) + assert.NoError(t, err) + assert.NotNil(t, segRefer) + svr.segReferManager = segRefer + + sc := make(chan os.Signal, 1) + signal.Notify(sc, syscall.SIGINT) + defer signal.Reset(syscall.SIGINT) + closed := false + sigQuit := make(chan struct{}, 1) + + svr.serverLoopWg.Add(1) + go func() { + svr.watchService(context.Background()) + }() + + go func() { + <-sc + closed = true + sigQuit <- struct{}{} + }() + + qcCh <- &sessionutil.SessionEvent{ + EventType: sessionutil.SessionAddEvent, + Session: &sessionutil.Session{ + ServerID: 2, + }, + } + qcCh <- &sessionutil.SessionEvent{ + EventType: sessionutil.SessionDelEvent, + Session: &sessionutil.Session{ + ServerID: 2, + }, + } + close(qcCh) + <-sigQuit + svr.serverLoopWg.Wait() + assert.True(t, closed) +} + func TestServer_GetMetrics(t *testing.T) { svr := newTestServer(t, nil) defer closeTestServer(t, svr) @@ -2494,6 +2619,30 @@ func TestImport(t *testing.T) { assert.NoError(t, err) assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode()) }) + + t.Run("test acquire segment reference lock with closed server", func(t *testing.T) { + svr := newTestServer(t, nil) + closeTestServer(t, svr) + + status, err := svr.AcquireSegmentLock(context.TODO(), &datapb.AcquireSegmentLockRequest{ + SegmentIDs: []UniqueID{1, 2}, + NodeID: UniqueID(1), + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode()) + }) + + t.Run("test release segment reference lock with closed server", func(t *testing.T) { + svr := newTestServer(t, nil) + closeTestServer(t, svr) + + status, err := svr.ReleaseSegmentLock(context.TODO(), &datapb.ReleaseSegmentLockRequest{ + SegmentIDs: []UniqueID{1, 2}, + NodeID: UniqueID(1), + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, status.GetErrorCode()) + }) } // https://github.com/milvus-io/milvus/issues/15659 @@ -2580,3 +2729,91 @@ func closeTestServer(t *testing.T, svr *Server) { err = svr.CleanMeta() assert.Nil(t, err) } + +func newTestServer2(t *testing.T, receiveCh chan interface{}, opts ...Option) *Server { + var err error + Params.Init() + Params.CommonCfg.DataCoordTimeTick = Params.CommonCfg.DataCoordTimeTick + strconv.Itoa(rand.Int()) + factory := dependency.NewDefaultFactory(true) + + etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg) + assert.Nil(t, err) + sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot) + _, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) + assert.Nil(t, err) + + icSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli) + icSession.Init(typeutil.IndexCoordRole, "localhost:31000", true, true) + icSession.Register() + + qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, etcdCli) + qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true) + qcSession.Register() + + svr := CreateServer(context.TODO(), factory, opts...) + svr.SetEtcdClient(etcdCli) + svr.dataNodeCreator = func(ctx context.Context, addr string) (types.DataNode, error) { + return newMockDataNodeClient(0, receiveCh) + } + svr.rootCoordClientCreator = func(ctx context.Context, metaRootPath string, etcdCli *clientv3.Client) (types.RootCoord, error) { + return newMockRootCoordService(), nil + } + + err = svr.Init() + assert.Nil(t, err) + err = svr.Start() + assert.Nil(t, err) + + _, err = etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) + assert.Nil(t, err) + + err = svr.Register() + assert.Nil(t, err) + + // Stop channal watch state watcher in tests + if svr.channelManager != nil && svr.channelManager.stopChecker != nil { + svr.channelManager.stopChecker() + } + + return svr +} + +func Test_initServiceDiscovery(t *testing.T) { + server := newTestServer2(t, nil) + assert.NotNil(t, server) + + segmentID := rand.Int63() + err := server.meta.AddSegment(&SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: segmentID, + CollectionID: rand.Int63(), + PartitionID: rand.Int63(), + NumOfRows: 100, + }, + currRows: 100, + }) + assert.Nil(t, err) + + qcSession := sessionutil.NewSession(context.Background(), Params.EtcdCfg.MetaRootPath, server.etcdCli) + qcSession.Init(typeutil.QueryCoordRole, "localhost:19532", true, true) + qcSession.Register() + req := &datapb.AcquireSegmentLockRequest{ + NodeID: qcSession.ServerID, + SegmentIDs: []UniqueID{segmentID}, + } + resp, err := server.AcquireSegmentLock(context.TODO(), req) + assert.Nil(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, resp.GetErrorCode()) + + sessKey := path.Join(Params.EtcdCfg.MetaRootPath, sessionutil.DefaultServiceRoot, typeutil.QueryCoordRole) + _, err = server.etcdCli.Delete(context.Background(), sessKey, clientv3.WithPrefix()) + assert.Nil(t, err) + + for { + if !server.segReferManager.HasSegmentLock(segmentID) { + break + } + } + + closeTestServer(t, server) +} diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index bce9927793..ec44004823 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -34,11 +34,10 @@ import ( "github.com/milvus-io/milvus/internal/proto/milvuspb" "github.com/milvus-io/milvus/internal/util/logutil" "github.com/milvus-io/milvus/internal/util/metricsinfo" + "github.com/milvus-io/milvus/internal/util/retry" "github.com/milvus-io/milvus/internal/util/trace" ) -const moduleName = "DataCoord" - // checks whether server in Healthy State func (s *Server) isClosed() bool { return atomic.LoadInt64(&s.isServing) != ServerStateHealthy @@ -666,13 +665,15 @@ func (s *Server) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedS } ret := make([]UniqueID, 0, len(segmentIDs)) for _, id := range segmentIDs { - s := s.meta.GetSegment(id) - if s != nil && s.GetState() != commonpb.SegmentState_Flushed { + segment := s.meta.GetSegment(id) + if segment != nil && segment.GetState() != commonpb.SegmentState_Flushed { continue } + // if this segment == nil, we assume this segment has been compacted and flushed ret = append(ret, id) } + resp.Segments = ret resp.Status.ErrorCode = commonpb.ErrorCode_Success return resp, nil @@ -1073,3 +1074,66 @@ func getDiff(base, remove []int64) []int64 { } return diff } + +// AcquireSegmentLock acquire the reference lock of the segments. +func (s *Server) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { + resp := &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + } + + if s.isClosed() { + log.Warn("failed to acquire segments reference lock for closed server") + resp.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.GetNodeID()) + return resp, nil + } + + hasSegments, err := s.meta.HasSegments(req.SegmentIDs) + if !hasSegments || err != nil { + log.Error("AcquireSegmentLock failed", zap.Error(err)) + resp.Reason = err.Error() + return resp, nil + } + + err = s.segReferManager.AddSegmentsLock(req.SegmentIDs, req.NodeID) + if err != nil { + log.Warn("Add reference lock on segments failed", zap.Int64s("segIDs", req.SegmentIDs), zap.Error(err)) + resp.Reason = err.Error() + return resp, nil + } + hasSegments, err = s.meta.HasSegments(req.SegmentIDs) + if !hasSegments || err != nil { + log.Error("AcquireSegmentLock failed, try to release reference lock", zap.Error(err)) + if err2 := retry.Do(ctx, func() error { + return s.segReferManager.ReleaseSegmentsLock(req.SegmentIDs, req.NodeID) + }, retry.Attempts(100)); err2 != nil { + panic(err) + } + resp.Reason = err.Error() + return resp, nil + } + resp.ErrorCode = commonpb.ErrorCode_Success + return resp, nil +} + +// ReleaseSegmentLock release the reference lock of the segments. +func (s *Server) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { + resp := &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + } + + if s.isClosed() { + log.Warn("failed to release segments reference lock for closed server") + resp.Reason = msgDataCoordIsUnhealthy(Params.DataCoordCfg.GetNodeID()) + return resp, nil + } + + err := s.segReferManager.ReleaseSegmentsLock(req.SegmentIDs, req.NodeID) + if err != nil { + log.Error("DataCoord ReleaseSegmentLock failed", zap.Int64s("segmentIDs", req.SegmentIDs), zap.Int64("nodeID", req.NodeID), + zap.Error(err)) + resp.Reason = err.Error() + return resp, nil + } + resp.ErrorCode = commonpb.ErrorCode_Success + return resp, nil +} diff --git a/internal/datacoord/util.go b/internal/datacoord/util.go index 0b4c49cdb5..c980a28735 100644 --- a/internal/datacoord/util.go +++ b/internal/datacoord/util.go @@ -19,6 +19,8 @@ package datacoord import ( "context" "errors" + "strconv" + "strings" "time" "github.com/milvus-io/milvus/internal/proto/commonpb" @@ -77,3 +79,9 @@ func getTimetravelReverseTime(ctx context.Context, allocator allocator) (*timetr tt := tsoutil.ComposeTS(ttpts.UnixNano()/int64(time.Millisecond), 0) return &timetravel{tt}, nil } + +func parseSegmentIDByBinlog(path string) (UniqueID, error) { + // binlog path should consist of "files/insertLog/collID/partID/segID/fieldID/fileName" + keyStr := strings.Split(path, "/") + return strconv.ParseInt(keyStr[len(keyStr)-3], 10, 64) +} diff --git a/internal/distributed/datacoord/client/client.go b/internal/distributed/datacoord/client/client.go index 1daf4c3733..4a36c56b05 100644 --- a/internal/distributed/datacoord/client/client.go +++ b/internal/distributed/datacoord/client/client.go @@ -531,3 +531,31 @@ func (c *Client) UpdateSegmentStatistics(ctx context.Context, req *datapb.Update } return ret.(*commonpb.Status), err } + +// AcquireSegmentLock acquire the reference lock of the segments. +func (c *Client) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { + ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) { + if !funcutil.CheckCtxValid(ctx) { + return nil, ctx.Err() + } + return client.(datapb.DataCoordClient).AcquireSegmentLock(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*commonpb.Status), err +} + +// ReleaseSegmentLock release the reference lock of the segments. +func (c *Client) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { + ret, err := c.grpcClient.ReCall(ctx, func(client interface{}) (interface{}, error) { + if !funcutil.CheckCtxValid(ctx) { + return nil, ctx.Err() + } + return client.(datapb.DataCoordClient).ReleaseSegmentLock(ctx, req) + }) + if err != nil || ret == nil { + return nil, err + } + return ret.(*commonpb.Status), err +} diff --git a/internal/distributed/datacoord/client/client_test.go b/internal/distributed/datacoord/client/client_test.go index 5c30ea300c..01a32a770c 100644 --- a/internal/distributed/datacoord/client/client_test.go +++ b/internal/distributed/datacoord/client/client_test.go @@ -129,6 +129,12 @@ func Test_NewClient(t *testing.T) { r24, err := client.UpdateSegmentStatistics(ctx, nil) retCheck(retNotNil, r24, err) + + r25, err := client.AcquireSegmentLock(ctx, nil) + retCheck(retNotNil, r25, err) + + r26, err := client.ReleaseSegmentLock(ctx, nil) + retCheck(retNotNil, r26, err) } client.grpcClient = &mock.GRPCClientBase{ diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index e20b560baf..8cebd0b0cb 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -340,3 +340,13 @@ func (s *Server) Import(ctx context.Context, req *datapb.ImportTaskRequest) (*da func (s *Server) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) { return s.dataCoord.UpdateSegmentStatistics(ctx, req) } + +// AcquireSegmentLock acquire the reference lock of the segments. +func (s *Server) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { + return s.dataCoord.AcquireSegmentLock(ctx, req) +} + +// ReleaseSegmentLock release the reference lock of the segments. +func (s *Server) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { + return s.dataCoord.ReleaseSegmentLock(ctx, req) +} diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index 8510926f5f..21b847eaf8 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -58,6 +58,8 @@ type MockDataCoord struct { setSegmentStateResp *datapb.SetSegmentStateResponse importResp *datapb.ImportTaskResponse updateSegStatResp *commonpb.Status + acquireSegLockResp *commonpb.Status + releaseSegLockResp *commonpb.Status } func (m *MockDataCoord) Init() error { @@ -179,6 +181,14 @@ func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb return m.updateSegStatResp, m.err } +func (m *MockDataCoord) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { + return m.acquireSegLockResp, m.err +} + +func (m *MockDataCoord) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { + return m.releaseSegLockResp, m.err +} + /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// func Test_NewServer(t *testing.T) { ctx := context.Background() @@ -420,6 +430,28 @@ func Test_NewServer(t *testing.T) { assert.NotNil(t, resp) }) + t.Run("acquire segment reference lock", func(t *testing.T) { + server.dataCoord = &MockDataCoord{ + acquireSegLockResp: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + } + resp, err := server.AcquireSegmentLock(ctx, nil) + assert.Nil(t, err) + assert.NotNil(t, resp) + }) + + t.Run("release segment reference lock", func(t *testing.T) { + server.dataCoord = &MockDataCoord{ + releaseSegLockResp: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + } + resp, err := server.ReleaseSegmentLock(ctx, nil) + assert.Nil(t, err) + assert.NotNil(t, resp) + }) + err := server.Stop() assert.Nil(t, err) } diff --git a/internal/distributed/indexcoord/client/client_test.go b/internal/distributed/indexcoord/client/client_test.go index 6e6d68d92a..b859dab439 100644 --- a/internal/distributed/indexcoord/client/client_test.go +++ b/internal/distributed/indexcoord/client/client_test.go @@ -42,6 +42,9 @@ func TestIndexCoordClient(t *testing.T) { icm := &indexcoord.Mock{} err = server.SetClient(icm) assert.Nil(t, err) + dcm := &indexcoord.DataCoordMock{} + err = server.SetDataCoord(dcm) + assert.Nil(t, err) err = server.Run() assert.Nil(t, err) diff --git a/internal/distributed/indexcoord/service.go b/internal/distributed/indexcoord/service.go index c77f6367df..3269a51f2a 100644 --- a/internal/distributed/indexcoord/service.go +++ b/internal/distributed/indexcoord/service.go @@ -30,8 +30,9 @@ import ( "google.golang.org/grpc/keepalive" ot "github.com/grpc-ecosystem/go-grpc-middleware/tracing/opentracing" - + dcc "github.com/milvus-io/milvus/internal/distributed/datacoord/client" "github.com/milvus-io/milvus/internal/indexcoord" + ic "github.com/milvus-io/milvus/internal/indexcoord" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/indexpb" @@ -65,6 +66,8 @@ type Server struct { etcdCli *clientv3.Client + dataCoord types.DataCoord + closer io.Closer } @@ -113,6 +116,34 @@ func (s *Server) init() error { return err } + // --- DataCoord --- + if s.dataCoord == nil { + s.dataCoord, err = dcc.NewClient(s.loopCtx, ic.Params.EtcdCfg.MetaRootPath, s.etcdCli) + if err != nil { + log.Debug("IndexCoord try to new DataCoord client failed", zap.Error(err)) + panic(err) + } + } + + if err = s.dataCoord.Init(); err != nil { + log.Debug("IndexCoord DataCoordClient Init failed", zap.Error(err)) + panic(err) + } + if err = s.dataCoord.Start(); err != nil { + log.Debug("IndexCoord DataCoordClient Start failed", zap.Error(err)) + panic(err) + } + log.Debug("IndexCoord try to wait for DataCoord ready") + err = funcutil.WaitForComponentHealthy(s.loopCtx, s.dataCoord, "DataCoord", 1000000, time.Millisecond*200) + if err != nil { + log.Debug("IndexCoord wait for DataCoord ready failed", zap.Error(err)) + panic(err) + } + + if err := s.SetDataCoord(s.dataCoord); err != nil { + panic(err) + } + return nil } @@ -160,6 +191,12 @@ func (s *Server) SetClient(indexCoordClient types.IndexCoordComponent) error { return nil } +// SetDataCoord sets the DataCoord's client for IndexCoord component. +func (s *Server) SetDataCoord(d types.DataCoord) error { + s.dataCoord = d + return s.indexcoord.SetDataCoord(d) +} + // GetComponentStates gets the component states of IndexCoord. func (s *Server) GetComponentStates(ctx context.Context, req *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) { return s.indexcoord.GetComponentStates(ctx) diff --git a/internal/distributed/indexcoord/service_test.go b/internal/distributed/indexcoord/service_test.go index e6a3763c14..3282ec777f 100644 --- a/internal/distributed/indexcoord/service_test.go +++ b/internal/distributed/indexcoord/service_test.go @@ -40,10 +40,14 @@ func TestIndexCoordinateServer(t *testing.T) { Params.Init() etcd, err := etcd.GetEtcdClient(&Params.EtcdCfg) assert.NoError(t, err) + assert.NotNil(t, etcd) indexCoordClient := &indexcoord.Mock{} indexCoordClient.SetEtcdClient(etcd) err = server.SetClient(indexCoordClient) assert.Nil(t, err) + + dcm := &indexcoord.DataCoordMock{} + server.dataCoord = dcm err = server.Run() assert.Nil(t, err) diff --git a/internal/distributed/proxy/service_test.go b/internal/distributed/proxy/service_test.go index 1d680fc932..dfbb044fc5 100644 --- a/internal/distributed/proxy/service_test.go +++ b/internal/distributed/proxy/service_test.go @@ -518,6 +518,14 @@ func (m *MockDataCoord) UpdateSegmentStatistics(ctx context.Context, req *datapb return nil, nil } +func (m *MockDataCoord) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { + return nil, nil +} + +func (m *MockDataCoord) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { + return nil, nil +} + /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// type MockProxy struct { MockBase diff --git a/internal/distributed/rootcoord/service_test.go b/internal/distributed/rootcoord/service_test.go index 32e5031341..0ed0094747 100644 --- a/internal/distributed/rootcoord/service_test.go +++ b/internal/distributed/rootcoord/service_test.go @@ -161,12 +161,21 @@ func TestGrpcService(t *testing.T) { return nil } - core.CallGetBinlogFilePathsService = func(ctx context.Context, segID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]string, error) { - return []string{"file1", "file2", "file3"}, nil - } - core.CallGetNumRowsService = func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (int64, error) { - return rootcoord.Params.RootCoordCfg.MinSegmentSizeToEnableIndex, nil + core.CallGetRecoveryInfoService = func(ctx context.Context, collID, partID rootcoord.UniqueID) ([]*datapb.SegmentBinlogs, error) { + return []*datapb.SegmentBinlogs{ + { + SegmentID: segID, + NumOfRows: rootcoord.Params.RootCoordCfg.MinSegmentSizeToEnableIndex, + FieldBinlogs: []*datapb.FieldBinlog{ + { + FieldID: fieldID, + Binlogs: []*datapb.Binlog{{LogPath: "file1"}, {LogPath: "file2"}, {LogPath: "file3"}}, + }, + }, + }, + }, nil } + core.CallWatchChannels = func(ctx context.Context, collectionID int64, channelNames []string) error { return nil } @@ -183,7 +192,7 @@ func TestGrpcService(t *testing.T) { var binlogLock sync.Mutex binlogPathArray := make([]string, 0, 16) - core.CallBuildIndexService = func(ctx context.Context, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) { + core.CallBuildIndexService = func(ctx context.Context, segID typeutil.UniqueID, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) { binlogLock.Lock() defer binlogLock.Unlock() binlogPathArray = append(binlogPathArray, binlog...) diff --git a/internal/indexcoord/const.go b/internal/indexcoord/const.go new file mode 100644 index 0000000000..efeb1f7736 --- /dev/null +++ b/internal/indexcoord/const.go @@ -0,0 +1,27 @@ +// 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 + +const ( + indexSizeFactor = 6 + indexFilePrefix = "indexes" +) + +const ( + // IndexAddTaskName is the name of the operation to add index task. + IndexAddTaskName = "IndexAddTask" +) diff --git a/internal/indexcoord/index_coord.go b/internal/indexcoord/index_coord.go index 184184492b..f5ea454a16 100644 --- a/internal/indexcoord/index_coord.go +++ b/internal/indexcoord/index_coord.go @@ -28,26 +28,25 @@ import ( "syscall" "time" - "github.com/milvus-io/milvus/internal/metrics" - "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/internal/util/dependency" - - "go.etcd.io/etcd/api/v3/mvccpb" - "go.uber.org/zap" - "github.com/golang/protobuf/proto" + "go.etcd.io/etcd/api/v3/mvccpb" clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/zap" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/common" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/log" + "github.com/milvus-io/milvus/internal/metrics" "github.com/milvus-io/milvus/internal/proto/commonpb" + "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" + "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/metricsinfo" "github.com/milvus-io/milvus/internal/util/paramtable" "github.com/milvus-io/milvus/internal/util/retry" @@ -60,10 +59,6 @@ import ( // make sure IndexCoord implements types.IndexCoord var _ types.IndexCoord = (*IndexCoord)(nil) -const ( - indexSizeFactor = 6 -) - var Params paramtable.ComponentParam // IndexCoord is a component responsible for scheduling index construction tasks and maintaining index status. @@ -103,6 +98,8 @@ type IndexCoord struct { assignTaskInterval time.Duration taskLimit int + dataCoordClient types.DataCoord + // Add callback functions at different stages startCallbacks []func() closeCallbacks []func() @@ -319,6 +316,16 @@ func (i *IndexCoord) SetEtcdClient(etcdClient *clientv3.Client) { i.etcdCli = etcdClient } +// SetDataCoord sets data coordinator's client +func (i *IndexCoord) SetDataCoord(dataCoord types.DataCoord) error { + if dataCoord == nil { + return errors.New("null DataCoord interface") + } + + i.dataCoordClient = dataCoord + return nil +} + // UpdateStateCode updates the component state of IndexCoord. func (i *IndexCoord) UpdateStateCode(code internalpb.StateCode) { i.stateCode.Store(code) @@ -395,21 +402,26 @@ func (i *IndexCoord) BuildIndex(ctx context.Context, req *indexpb.BuildIndexRequ }, }, err } - metrics.IndexCoordIndexRequestCounter.WithLabelValues(metrics.TotalLabel).Inc() - log.Debug("IndexCoord building index ...", - zap.Int64("IndexBuildID", req.IndexBuildID), - zap.String("IndexName = ", req.IndexName), - zap.Int64("IndexID = ", req.IndexID), - zap.Strings("DataPath = ", req.DataPaths), - zap.Any("TypeParams", req.TypeParams), - zap.Any("IndexParams", req.IndexParams), - zap.Int64("numRow", req.NumRows), + log.Debug("IndexCoord building index ...", zap.Int64("segmentID", req.SegmentID), + zap.String("IndexName", req.IndexName), zap.Int64("IndexID", req.IndexID), + zap.Strings("DataPath", req.DataPaths), zap.Any("TypeParams", req.TypeParams), + zap.Any("IndexParams", req.IndexParams), zap.Int64("numRows", req.NumRows), zap.Any("field type", req.FieldSchema.DataType)) + + ret := &indexpb.BuildIndexResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + }, + IndexBuildID: 0, + } + + metrics.IndexCoordIndexRequestCounter.WithLabelValues(metrics.TotalLabel).Inc() + sp, ctx := trace.StartSpanFromContextWithOperationName(ctx, "IndexCoord-BuildIndex") defer sp.Finish() hasIndex, indexBuildID := i.metaTable.HasSameReq(req) if hasIndex { - log.Debug("IndexCoord", zap.Int64("hasIndex true", indexBuildID), zap.Strings("data paths", req.DataPaths)) + log.Debug("IndexCoord has same index", zap.Int64("buildID", indexBuildID), zap.Int64("segmentID", req.SegmentID)) return &indexpb.BuildIndexResponse{ Status: &commonpb.Status{ ErrorCode: commonpb.ErrorCode_Success, @@ -418,11 +430,7 @@ func (i *IndexCoord) BuildIndex(ctx context.Context, req *indexpb.BuildIndexRequ IndexBuildID: indexBuildID, }, nil } - ret := &indexpb.BuildIndexResponse{ - Status: &commonpb.Status{ - ErrorCode: commonpb.ErrorCode_UnexpectedError, - }, - } + t := &IndexAddTask{ BaseTask: BaseTask{ ctx: ctx, @@ -837,6 +845,9 @@ func (i *IndexCoord) watchMetaLoop() { log.Debug("This task has finished", zap.Int64("indexBuildID", indexBuildID), zap.Int64("Finish by IndexNode", indexMeta.NodeID), zap.Int64("The version of the task", indexMeta.Version)) + if err = i.tryReleaseSegmentReferLock(ctx, []UniqueID{indexMeta.Req.SegmentID}); err != nil { + panic(err) + } i.nodeManager.pq.IncPriority(indexMeta.NodeID, -1) metrics.IndexCoordIndexTaskCounter.WithLabelValues(metrics.InProgressIndexTaskLabel).Dec() if indexMeta.State == commonpb.IndexState_Finished { @@ -854,6 +865,47 @@ func (i *IndexCoord) watchMetaLoop() { } } +func (i *IndexCoord) tryAcquireSegmentReferLock(ctx context.Context, segIDs []UniqueID) error { + status, err := i.dataCoordClient.AcquireSegmentLock(ctx, &datapb.AcquireSegmentLockRequest{ + NodeID: i.session.ServerID, + SegmentIDs: segIDs, + }) + if err != nil { + log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64s("segIDs", segIDs), + zap.Error(err)) + return err + } + if status.ErrorCode != commonpb.ErrorCode_Success { + log.Error("IndexCoord try to acquire segment reference lock failed", zap.Int64s("segIDs", segIDs), + zap.Error(errors.New(status.Reason))) + return errors.New(status.Reason) + } + return nil +} + +func (i *IndexCoord) tryReleaseSegmentReferLock(ctx context.Context, segIDs []UniqueID) error { + releaseLock := func() error { + status, err := i.dataCoordClient.ReleaseSegmentLock(ctx, &datapb.ReleaseSegmentLockRequest{ + NodeID: i.session.ServerID, + SegmentIDs: segIDs, + }) + if err != nil { + return err + } + if status.ErrorCode != commonpb.ErrorCode_Success { + return errors.New(status.Reason) + } + return nil + } + err := retry.Do(ctx, releaseLock, retry.Attempts(100)) + if err != nil { + log.Error("IndexCoord try to release segment reference lock failed", zap.Int64s("segIDs", segIDs), + zap.Error(err)) + return err + } + return nil +} + // assignTask sends the index task to the IndexNode, it has a timeout interval, if the IndexNode doesn't respond within // the interval, it is considered that the task sending failed. func (i *IndexCoord) assignTask(builderClient types.IndexNode, req *indexpb.CreateIndexRequest) bool { @@ -903,6 +955,12 @@ func (i *IndexCoord) assignTaskLoop() { } for index, meta := range metas { indexBuildID := meta.indexMeta.IndexBuildID + segID := meta.indexMeta.Req.SegmentID + if err := i.tryAcquireSegmentReferLock(ctx, []UniqueID{segID}); err != nil { + log.Warn("IndexCoord try to acquire segment reference lock failed, maybe this segment has been compacted", + zap.Int64("segID", segID), zap.Int64("buildID", indexBuildID), zap.Error(err)) + continue + } if err := i.metaTable.UpdateVersion(indexBuildID); err != nil { log.Warn("IndexCoord assignmentTasksLoop metaTable.UpdateVersion failed", zap.Error(err)) continue diff --git a/internal/indexcoord/index_coord_mock.go b/internal/indexcoord/index_coord_mock.go index b6197bcb87..7cd1911968 100644 --- a/internal/indexcoord/index_coord_mock.go +++ b/internal/indexcoord/index_coord_mock.go @@ -21,11 +21,15 @@ import ( "errors" "strconv" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/storage" + etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/milvuspb" + "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/typeutil" clientv3 "go.etcd.io/etcd/client/v3" @@ -83,6 +87,10 @@ func (icm *Mock) SetEtcdClient(client *clientv3.Client) { icm.etcdCli = client } +func (icm *Mock) SetDataCoord(dataCoord types.DataCoord) error { + return nil +} + func (icm *Mock) UpdateStateCode(stateCode internalpb.StateCode) { } @@ -243,3 +251,123 @@ func (icm *Mock) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsReq ComponentName: "IndexCoord", }, nil } + +type DataCoordMock struct { + types.DataCoord + + Fail bool + Err bool +} + +func (dcm *DataCoordMock) Init() error { + if dcm.Err || dcm.Fail { + return errors.New("DataCoord mock init failed") + } + return nil +} + +func (dcm *DataCoordMock) Start() error { + if dcm.Err || dcm.Fail { + return errors.New("DataCoord mock start failed") + } + return nil +} + +func (dcm *DataCoordMock) GetComponentStates(ctx context.Context) (*internalpb.ComponentStates, error) { + if dcm.Err { + return &internalpb.ComponentStates{ + State: &internalpb.ComponentInfo{ + StateCode: internalpb.StateCode_Abnormal, + }, + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "", + }, + }, errors.New("DataCoord component state is not healthy") + } + if dcm.Fail { + return &internalpb.ComponentStates{ + State: &internalpb.ComponentInfo{ + StateCode: internalpb.StateCode_Abnormal, + }, + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "", + }, + }, nil + } + return &internalpb.ComponentStates{ + State: &internalpb.ComponentInfo{ + StateCode: internalpb.StateCode_Healthy, + }, + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, + }, nil +} + +func (dcm *DataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { + if dcm.Err { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "", + }, errors.New("an error occurred") + } + if dcm.Fail { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "failure reason", + }, nil + } + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, nil +} + +func (dcm *DataCoordMock) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { + if dcm.Err { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "", + }, errors.New("an error occurred") + } + if dcm.Fail { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "failure reason", + }, nil + } + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, nil +} + +type ChunkManagerMock struct { + storage.ChunkManager + + Fail bool + Err bool +} + +func (cmm *ChunkManagerMock) Exist(path string) (bool, error) { + if cmm.Err { + return false, errors.New("path not exist") + } + if cmm.Fail { + return false, nil + } + return true, nil +} + +func (cmm *ChunkManagerMock) RemoveWithPrefix(prefix string) error { + if cmm.Err { + return errors.New("error occurred") + } + if cmm.Fail { + return nil + } + return nil +} diff --git a/internal/indexcoord/index_coord_mock_test.go b/internal/indexcoord/index_coord_mock_test.go index 7a9953e65b..f20917ede1 100644 --- a/internal/indexcoord/index_coord_mock_test.go +++ b/internal/indexcoord/index_coord_mock_test.go @@ -20,6 +20,8 @@ import ( "context" "testing" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/commonpb" "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/proto/internalpb" @@ -201,3 +203,108 @@ func TestIndexCoordMockError(t *testing.T) { err = icm.Stop() assert.NotNil(t, err) } + +func TestDataCoordMock_Error(t *testing.T) { + t.Run("Init", func(t *testing.T) { + dcm := &DataCoordMock{ + Fail: true, + Err: true, + } + err := dcm.Init() + assert.Error(t, err) + }) + + t.Run("Start", func(t *testing.T) { + dcm := &DataCoordMock{ + Fail: true, + Err: true, + } + err := dcm.Start() + assert.Error(t, err) + }) + + t.Run("GetComponentStates", func(t *testing.T) { + dcm := &DataCoordMock{ + Fail: true, + Err: true, + } + resp, err := dcm.GetComponentStates(context.TODO()) + assert.Error(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode()) + + dcm.Err = false + resp, err = dcm.GetComponentStates(context.TODO()) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.GetErrorCode()) + }) + + t.Run("AcquireSegmentLock", func(t *testing.T) { + dcm := &DataCoordMock{ + Fail: true, + Err: true, + } + resp, err := dcm.AcquireSegmentLock(context.TODO(), &datapb.AcquireSegmentLockRequest{}) + assert.Error(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode()) + + dcm.Err = false + resp, err = dcm.AcquireSegmentLock(context.TODO(), &datapb.AcquireSegmentLockRequest{}) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode()) + }) + + t.Run("ReleaseSegmentLock", func(t *testing.T) { + dcm := &DataCoordMock{ + Fail: true, + Err: true, + } + + resp, err := dcm.ReleaseSegmentLock(context.TODO(), &datapb.ReleaseSegmentLockRequest{}) + assert.Error(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode()) + + dcm.Err = false + resp, err = dcm.ReleaseSegmentLock(context.TODO(), &datapb.ReleaseSegmentLockRequest{}) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.GetErrorCode()) + }) +} + +func TestChunkManageMock_Error(t *testing.T) { + t.Run("Exist", func(t *testing.T) { + cmm := &ChunkManagerMock{ + Fail: true, + Err: true, + } + ok, err := cmm.Exist("") + assert.Error(t, err) + assert.False(t, ok) + + cmm.Err = false + ok, err = cmm.Exist("") + assert.NoError(t, err) + assert.False(t, ok) + + cmm.Fail = false + ok, err = cmm.Exist("") + assert.NoError(t, err) + assert.True(t, ok) + }) + t.Run("RemoveWithPrefix", func(t *testing.T) { + cmm := &ChunkManagerMock{ + Fail: true, + Err: true, + } + + err := cmm.RemoveWithPrefix("") + assert.Error(t, err) + + cmm.Err = false + err = cmm.RemoveWithPrefix("") + assert.NoError(t, err) + + cmm.Fail = false + err = cmm.RemoveWithPrefix("") + assert.NoError(t, err) + }) +} diff --git a/internal/indexcoord/index_coord_test.go b/internal/indexcoord/index_coord_test.go index 14bbaa06f1..ab4dfdd0da 100644 --- a/internal/indexcoord/index_coord_test.go +++ b/internal/indexcoord/index_coord_test.go @@ -65,9 +65,23 @@ func TestIndexCoord(t *testing.T) { ic.assignTaskInterval = 200 * time.Millisecond ic.taskLimit = 20 + dcm := &DataCoordMock{ + Err: false, + Fail: false, + } + err = ic.SetDataCoord(dcm) + assert.Nil(t, err) + ic.SetEtcdClient(etcdCli) err = ic.Init() assert.Nil(t, err) + + ccm := &ChunkManagerMock{ + Err: false, + Fail: false, + } + ic.chunkManager = ccm + err = ic.Register() assert.Nil(t, err) err = ic.Start() @@ -387,4 +401,72 @@ func TestIndexCoord_GetIndexFilePaths(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_UnexpectedError, resp.Status.ErrorCode) assert.NotEqual(t, "", resp.Status.Reason) }) + + t.Run("set DataCoord with nil", func(t *testing.T) { + err := ic.SetDataCoord(nil) + assert.Error(t, err) + }) +} + +func Test_tryAcquireSegmentReferLock(t *testing.T) { + ic := &IndexCoord{ + session: &sessionutil.Session{ + ServerID: 1, + }, + } + dcm := &DataCoordMock{ + Err: false, + Fail: false, + } + cmm := &ChunkManagerMock{ + Err: false, + Fail: false, + } + + ic.dataCoordClient = dcm + ic.chunkManager = cmm + + t.Run("success", func(t *testing.T) { + err := ic.tryAcquireSegmentReferLock(context.Background(), []UniqueID{1}) + assert.Nil(t, err) + }) + + t.Run("error", func(t *testing.T) { + dcmE := &DataCoordMock{ + Err: true, + Fail: false, + } + ic.dataCoordClient = dcmE + err := ic.tryAcquireSegmentReferLock(context.Background(), []UniqueID{1}) + assert.Error(t, err) + }) + + t.Run("Fail", func(t *testing.T) { + dcmF := &DataCoordMock{ + Err: false, + Fail: true, + } + ic.dataCoordClient = dcmF + err := ic.tryAcquireSegmentReferLock(context.Background(), []UniqueID{1}) + assert.Error(t, err) + }) +} + +func Test_tryReleaseSegmentReferLock(t *testing.T) { + ic := &IndexCoord{ + session: &sessionutil.Session{ + ServerID: 1, + }, + } + dcm := &DataCoordMock{ + Err: false, + Fail: false, + } + + ic.dataCoordClient = dcm + + t.Run("success", func(t *testing.T) { + err := ic.tryReleaseSegmentReferLock(context.Background(), []UniqueID{1}) + assert.NoError(t, err) + }) } diff --git a/internal/indexcoord/meta_table.go b/internal/indexcoord/meta_table.go index 68e84366c3..ee5db54cea 100644 --- a/internal/indexcoord/meta_table.go +++ b/internal/indexcoord/meta_table.go @@ -20,6 +20,7 @@ import ( "context" "errors" "fmt" + "path" "strconv" "sync" @@ -99,7 +100,7 @@ func (mt *metaTable) saveIndexMeta(meta *Meta) error { if err != nil { return err } - key := "indexes/" + strconv.FormatInt(meta.indexMeta.IndexBuildID, 10) + key := path.Join(indexFilePrefix, strconv.FormatInt(meta.indexMeta.IndexBuildID, 10)) err = mt.client.CompareVersionAndSwap(key, meta.revision, string(value)) log.Debug("IndexCoord metaTable saveIndexMeta ", zap.String("key", key), zap.Error(err)) if err != nil { @@ -434,13 +435,13 @@ func (mt *metaTable) GetUnassignedTasks(onlineNodeIDs []int64) []Meta { // HasSameReq determine whether there are same indexing tasks. func (mt *metaTable) HasSameReq(req *indexpb.BuildIndexRequest) (bool, UniqueID) { - mt.lock.Lock() - defer mt.lock.Unlock() + mt.lock.RLock() + defer mt.lock.RUnlock() - log.Debug("IndexCoord judges whether the same task exists in meta table", zap.Int64("indexBuildID", req.IndexBuildID), - zap.Int64("indexID", req.IndexID), zap.Any("index params", req.IndexParams), - zap.Any("type params", req.TypeParams)) for _, meta := range mt.indexBuildID2Meta { + if req.GetSegmentID() != meta.indexMeta.Req.GetSegmentID() { + continue + } if meta.indexMeta.Req.IndexID != req.IndexID { continue } @@ -497,7 +498,7 @@ func (mt *metaTable) HasSameReq(req *indexpb.BuildIndexRequest) (bool, UniqueID) return true, meta.indexMeta.IndexBuildID } - return false, -1 + return false, 0 } // LoadMetaFromETCD load the meta of specified indexBuildID from ETCD. diff --git a/internal/indexcoord/task.go b/internal/indexcoord/task.go index 7e8bb49629..d481321688 100644 --- a/internal/indexcoord/task.go +++ b/internal/indexcoord/task.go @@ -27,11 +27,6 @@ import ( "github.com/milvus-io/milvus/internal/proto/indexpb" ) -const ( - // IndexAddTaskName is the name of the operation to add index task. - IndexAddTaskName = "IndexAddTask" -) - type task interface { Ctx() context.Context ID() UniqueID // return ReqID diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index 9d38ef5d50..b9076a174d 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -49,6 +49,9 @@ service DataCoord { // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load rpc Import(ImportTaskRequest) returns (ImportTaskResponse) {} rpc UpdateSegmentStatistics(UpdateSegmentStatisticsRequest) returns (common.Status) {} + + rpc AcquireSegmentLock(AcquireSegmentLockRequest) returns (common.Status) {} + rpc ReleaseSegmentLock(ReleaseSegmentLockRequest) returns (common.Status) {} } service DataNode { @@ -175,6 +178,17 @@ message GetPartitionStatisticsResponse { message GetSegmentInfoChannelRequest { } +message AcquireSegmentLockRequest { + common.MsgBase base = 1; + int64 nodeID = 2; + repeated int64 segmentIDs = 3; +} + +message ReleaseSegmentLockRequest { + common.MsgBase base = 1; + int64 nodeID = 2; + repeated int64 segmentIDs = 3; +} message VchannelInfo { int64 collectionID = 1; diff --git a/internal/proto/datapb/data_coord.pb.go b/internal/proto/datapb/data_coord.pb.go index 5cee9c1a6b..d9be729e8a 100644 --- a/internal/proto/datapb/data_coord.pb.go +++ b/internal/proto/datapb/data_coord.pb.go @@ -1116,6 +1116,116 @@ func (m *GetSegmentInfoChannelRequest) XXX_DiscardUnknown() { var xxx_messageInfo_GetSegmentInfoChannelRequest proto.InternalMessageInfo +type AcquireSegmentLockRequest struct { + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *AcquireSegmentLockRequest) Reset() { *m = AcquireSegmentLockRequest{} } +func (m *AcquireSegmentLockRequest) String() string { return proto.CompactTextString(m) } +func (*AcquireSegmentLockRequest) ProtoMessage() {} +func (*AcquireSegmentLockRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{19} +} + +func (m *AcquireSegmentLockRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_AcquireSegmentLockRequest.Unmarshal(m, b) +} +func (m *AcquireSegmentLockRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_AcquireSegmentLockRequest.Marshal(b, m, deterministic) +} +func (m *AcquireSegmentLockRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_AcquireSegmentLockRequest.Merge(m, src) +} +func (m *AcquireSegmentLockRequest) XXX_Size() int { + return xxx_messageInfo_AcquireSegmentLockRequest.Size(m) +} +func (m *AcquireSegmentLockRequest) XXX_DiscardUnknown() { + xxx_messageInfo_AcquireSegmentLockRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_AcquireSegmentLockRequest proto.InternalMessageInfo + +func (m *AcquireSegmentLockRequest) GetBase() *commonpb.MsgBase { + if m != nil { + return m.Base + } + return nil +} + +func (m *AcquireSegmentLockRequest) GetNodeID() int64 { + if m != nil { + return m.NodeID + } + return 0 +} + +func (m *AcquireSegmentLockRequest) GetSegmentIDs() []int64 { + if m != nil { + return m.SegmentIDs + } + return nil +} + +type ReleaseSegmentLockRequest struct { + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + NodeID int64 `protobuf:"varint,2,opt,name=nodeID,proto3" json:"nodeID,omitempty"` + SegmentIDs []int64 `protobuf:"varint,3,rep,packed,name=segmentIDs,proto3" json:"segmentIDs,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ReleaseSegmentLockRequest) Reset() { *m = ReleaseSegmentLockRequest{} } +func (m *ReleaseSegmentLockRequest) String() string { return proto.CompactTextString(m) } +func (*ReleaseSegmentLockRequest) ProtoMessage() {} +func (*ReleaseSegmentLockRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_82cd95f524594f49, []int{20} +} + +func (m *ReleaseSegmentLockRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ReleaseSegmentLockRequest.Unmarshal(m, b) +} +func (m *ReleaseSegmentLockRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ReleaseSegmentLockRequest.Marshal(b, m, deterministic) +} +func (m *ReleaseSegmentLockRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ReleaseSegmentLockRequest.Merge(m, src) +} +func (m *ReleaseSegmentLockRequest) XXX_Size() int { + return xxx_messageInfo_ReleaseSegmentLockRequest.Size(m) +} +func (m *ReleaseSegmentLockRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ReleaseSegmentLockRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ReleaseSegmentLockRequest proto.InternalMessageInfo + +func (m *ReleaseSegmentLockRequest) GetBase() *commonpb.MsgBase { + if m != nil { + return m.Base + } + return nil +} + +func (m *ReleaseSegmentLockRequest) GetNodeID() int64 { + if m != nil { + return m.NodeID + } + return 0 +} + +func (m *ReleaseSegmentLockRequest) GetSegmentIDs() []int64 { + if m != nil { + return m.SegmentIDs + } + return nil +} + type VchannelInfo struct { CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` ChannelName string `protobuf:"bytes,2,opt,name=channelName,proto3" json:"channelName,omitempty"` @@ -1132,7 +1242,7 @@ func (m *VchannelInfo) Reset() { *m = VchannelInfo{} } func (m *VchannelInfo) String() string { return proto.CompactTextString(m) } func (*VchannelInfo) ProtoMessage() {} func (*VchannelInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{19} + return fileDescriptor_82cd95f524594f49, []int{21} } func (m *VchannelInfo) XXX_Unmarshal(b []byte) error { @@ -1207,7 +1317,7 @@ func (m *WatchDmChannelsRequest) Reset() { *m = WatchDmChannelsRequest{} func (m *WatchDmChannelsRequest) String() string { return proto.CompactTextString(m) } func (*WatchDmChannelsRequest) ProtoMessage() {} func (*WatchDmChannelsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{20} + return fileDescriptor_82cd95f524594f49, []int{22} } func (m *WatchDmChannelsRequest) XXX_Unmarshal(b []byte) error { @@ -1257,7 +1367,7 @@ func (m *FlushSegmentsRequest) Reset() { *m = FlushSegmentsRequest{} } func (m *FlushSegmentsRequest) String() string { return proto.CompactTextString(m) } func (*FlushSegmentsRequest) ProtoMessage() {} func (*FlushSegmentsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{21} + return fileDescriptor_82cd95f524594f49, []int{23} } func (m *FlushSegmentsRequest) XXX_Unmarshal(b []byte) error { @@ -1325,7 +1435,7 @@ func (m *SegmentMsg) Reset() { *m = SegmentMsg{} } func (m *SegmentMsg) String() string { return proto.CompactTextString(m) } func (*SegmentMsg) ProtoMessage() {} func (*SegmentMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{22} + return fileDescriptor_82cd95f524594f49, []int{24} } func (m *SegmentMsg) XXX_Unmarshal(b []byte) error { @@ -1374,7 +1484,7 @@ func (m *CollectionInfo) Reset() { *m = CollectionInfo{} } func (m *CollectionInfo) String() string { return proto.CompactTextString(m) } func (*CollectionInfo) ProtoMessage() {} func (*CollectionInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{23} + return fileDescriptor_82cd95f524594f49, []int{25} } func (m *CollectionInfo) XXX_Unmarshal(b []byte) error { @@ -1451,7 +1561,7 @@ func (m *SegmentInfo) Reset() { *m = SegmentInfo{} } func (m *SegmentInfo) String() string { return proto.CompactTextString(m) } func (*SegmentInfo) ProtoMessage() {} func (*SegmentInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{24} + return fileDescriptor_82cd95f524594f49, []int{26} } func (m *SegmentInfo) XXX_Unmarshal(b []byte) error { @@ -1596,7 +1706,7 @@ func (m *SegmentStartPosition) Reset() { *m = SegmentStartPosition{} } func (m *SegmentStartPosition) String() string { return proto.CompactTextString(m) } func (*SegmentStartPosition) ProtoMessage() {} func (*SegmentStartPosition) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{25} + return fileDescriptor_82cd95f524594f49, []int{27} } func (m *SegmentStartPosition) XXX_Unmarshal(b []byte) error { @@ -1652,7 +1762,7 @@ func (m *SaveBinlogPathsRequest) Reset() { *m = SaveBinlogPathsRequest{} func (m *SaveBinlogPathsRequest) String() string { return proto.CompactTextString(m) } func (*SaveBinlogPathsRequest) ProtoMessage() {} func (*SaveBinlogPathsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{26} + return fileDescriptor_82cd95f524594f49, []int{28} } func (m *SaveBinlogPathsRequest) XXX_Unmarshal(b []byte) error { @@ -1763,7 +1873,7 @@ func (m *CheckPoint) Reset() { *m = CheckPoint{} } func (m *CheckPoint) String() string { return proto.CompactTextString(m) } func (*CheckPoint) ProtoMessage() {} func (*CheckPoint) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{27} + return fileDescriptor_82cd95f524594f49, []int{29} } func (m *CheckPoint) XXX_Unmarshal(b []byte) error { @@ -1820,7 +1930,7 @@ func (m *DeltaLogInfo) Reset() { *m = DeltaLogInfo{} } func (m *DeltaLogInfo) String() string { return proto.CompactTextString(m) } func (*DeltaLogInfo) ProtoMessage() {} func (*DeltaLogInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{28} + return fileDescriptor_82cd95f524594f49, []int{30} } func (m *DeltaLogInfo) XXX_Unmarshal(b []byte) error { @@ -1890,7 +2000,7 @@ func (m *DataNodeTtMsg) Reset() { *m = DataNodeTtMsg{} } func (m *DataNodeTtMsg) String() string { return proto.CompactTextString(m) } func (*DataNodeTtMsg) ProtoMessage() {} func (*DataNodeTtMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{29} + return fileDescriptor_82cd95f524594f49, []int{31} } func (m *DataNodeTtMsg) XXX_Unmarshal(b []byte) error { @@ -1951,7 +2061,7 @@ func (m *SegmentStats) Reset() { *m = SegmentStats{} } func (m *SegmentStats) String() string { return proto.CompactTextString(m) } func (*SegmentStats) ProtoMessage() {} func (*SegmentStats) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{30} + return fileDescriptor_82cd95f524594f49, []int{32} } func (m *SegmentStats) XXX_Unmarshal(b []byte) error { @@ -1999,7 +2109,7 @@ func (m *ChannelStatus) Reset() { *m = ChannelStatus{} } func (m *ChannelStatus) String() string { return proto.CompactTextString(m) } func (*ChannelStatus) ProtoMessage() {} func (*ChannelStatus) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{31} + return fileDescriptor_82cd95f524594f49, []int{33} } func (m *ChannelStatus) XXX_Unmarshal(b []byte) error { @@ -2054,7 +2164,7 @@ func (m *DataNodeInfo) Reset() { *m = DataNodeInfo{} } func (m *DataNodeInfo) String() string { return proto.CompactTextString(m) } func (*DataNodeInfo) ProtoMessage() {} func (*DataNodeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{32} + return fileDescriptor_82cd95f524594f49, []int{34} } func (m *DataNodeInfo) XXX_Unmarshal(b []byte) error { @@ -2112,7 +2222,7 @@ func (m *SegmentBinlogs) Reset() { *m = SegmentBinlogs{} } func (m *SegmentBinlogs) String() string { return proto.CompactTextString(m) } func (*SegmentBinlogs) ProtoMessage() {} func (*SegmentBinlogs) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{33} + return fileDescriptor_82cd95f524594f49, []int{35} } func (m *SegmentBinlogs) XXX_Unmarshal(b []byte) error { @@ -2187,7 +2297,7 @@ func (m *FieldBinlog) Reset() { *m = FieldBinlog{} } func (m *FieldBinlog) String() string { return proto.CompactTextString(m) } func (*FieldBinlog) ProtoMessage() {} func (*FieldBinlog) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{34} + return fileDescriptor_82cd95f524594f49, []int{36} } func (m *FieldBinlog) XXX_Unmarshal(b []byte) error { @@ -2237,7 +2347,7 @@ func (m *Binlog) Reset() { *m = Binlog{} } func (m *Binlog) String() string { return proto.CompactTextString(m) } func (*Binlog) ProtoMessage() {} func (*Binlog) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{35} + return fileDescriptor_82cd95f524594f49, []int{37} } func (m *Binlog) XXX_Unmarshal(b []byte) error { @@ -2306,7 +2416,7 @@ func (m *GetRecoveryInfoResponse) Reset() { *m = GetRecoveryInfoResponse func (m *GetRecoveryInfoResponse) String() string { return proto.CompactTextString(m) } func (*GetRecoveryInfoResponse) ProtoMessage() {} func (*GetRecoveryInfoResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{36} + return fileDescriptor_82cd95f524594f49, []int{38} } func (m *GetRecoveryInfoResponse) XXX_Unmarshal(b []byte) error { @@ -2361,7 +2471,7 @@ func (m *GetRecoveryInfoRequest) Reset() { *m = GetRecoveryInfoRequest{} func (m *GetRecoveryInfoRequest) String() string { return proto.CompactTextString(m) } func (*GetRecoveryInfoRequest) ProtoMessage() {} func (*GetRecoveryInfoRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{37} + return fileDescriptor_82cd95f524594f49, []int{39} } func (m *GetRecoveryInfoRequest) XXX_Unmarshal(b []byte) error { @@ -2416,7 +2526,7 @@ func (m *GetFlushedSegmentsRequest) Reset() { *m = GetFlushedSegmentsReq func (m *GetFlushedSegmentsRequest) String() string { return proto.CompactTextString(m) } func (*GetFlushedSegmentsRequest) ProtoMessage() {} func (*GetFlushedSegmentsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{38} + return fileDescriptor_82cd95f524594f49, []int{40} } func (m *GetFlushedSegmentsRequest) XXX_Unmarshal(b []byte) error { @@ -2470,7 +2580,7 @@ func (m *GetFlushedSegmentsResponse) Reset() { *m = GetFlushedSegmentsRe func (m *GetFlushedSegmentsResponse) String() string { return proto.CompactTextString(m) } func (*GetFlushedSegmentsResponse) ProtoMessage() {} func (*GetFlushedSegmentsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{39} + return fileDescriptor_82cd95f524594f49, []int{41} } func (m *GetFlushedSegmentsResponse) XXX_Unmarshal(b []byte) error { @@ -2517,7 +2627,7 @@ func (m *SegmentFlushCompletedMsg) Reset() { *m = SegmentFlushCompletedM func (m *SegmentFlushCompletedMsg) String() string { return proto.CompactTextString(m) } func (*SegmentFlushCompletedMsg) ProtoMessage() {} func (*SegmentFlushCompletedMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{40} + return fileDescriptor_82cd95f524594f49, []int{42} } func (m *SegmentFlushCompletedMsg) XXX_Unmarshal(b []byte) error { @@ -2568,7 +2678,7 @@ func (m *ChannelWatchInfo) Reset() { *m = ChannelWatchInfo{} } func (m *ChannelWatchInfo) String() string { return proto.CompactTextString(m) } func (*ChannelWatchInfo) ProtoMessage() {} func (*ChannelWatchInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{41} + return fileDescriptor_82cd95f524594f49, []int{43} } func (m *ChannelWatchInfo) XXX_Unmarshal(b []byte) error { @@ -2632,7 +2742,7 @@ func (m *CompactionSegmentBinlogs) Reset() { *m = CompactionSegmentBinlo func (m *CompactionSegmentBinlogs) String() string { return proto.CompactTextString(m) } func (*CompactionSegmentBinlogs) ProtoMessage() {} func (*CompactionSegmentBinlogs) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{42} + return fileDescriptor_82cd95f524594f49, []int{44} } func (m *CompactionSegmentBinlogs) XXX_Unmarshal(b []byte) error { @@ -2705,7 +2815,7 @@ func (m *CompactionPlan) Reset() { *m = CompactionPlan{} } func (m *CompactionPlan) String() string { return proto.CompactTextString(m) } func (*CompactionPlan) ProtoMessage() {} func (*CompactionPlan) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{43} + return fileDescriptor_82cd95f524594f49, []int{45} } func (m *CompactionPlan) XXX_Unmarshal(b []byte) error { @@ -2791,7 +2901,7 @@ func (m *CompactionResult) Reset() { *m = CompactionResult{} } func (m *CompactionResult) String() string { return proto.CompactTextString(m) } func (*CompactionResult) ProtoMessage() {} func (*CompactionResult) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{44} + return fileDescriptor_82cd95f524594f49, []int{46} } func (m *CompactionResult) XXX_Unmarshal(b []byte) error { @@ -2867,7 +2977,7 @@ func (m *SegmentFieldBinlogMeta) Reset() { *m = SegmentFieldBinlogMeta{} func (m *SegmentFieldBinlogMeta) String() string { return proto.CompactTextString(m) } func (*SegmentFieldBinlogMeta) ProtoMessage() {} func (*SegmentFieldBinlogMeta) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{45} + return fileDescriptor_82cd95f524594f49, []int{47} } func (m *SegmentFieldBinlogMeta) XXX_Unmarshal(b []byte) error { @@ -2914,7 +3024,7 @@ func (m *WatchChannelsRequest) Reset() { *m = WatchChannelsRequest{} } func (m *WatchChannelsRequest) String() string { return proto.CompactTextString(m) } func (*WatchChannelsRequest) ProtoMessage() {} func (*WatchChannelsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{46} + return fileDescriptor_82cd95f524594f49, []int{48} } func (m *WatchChannelsRequest) XXX_Unmarshal(b []byte) error { @@ -2960,7 +3070,7 @@ func (m *WatchChannelsResponse) Reset() { *m = WatchChannelsResponse{} } func (m *WatchChannelsResponse) String() string { return proto.CompactTextString(m) } func (*WatchChannelsResponse) ProtoMessage() {} func (*WatchChannelsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{47} + return fileDescriptor_82cd95f524594f49, []int{49} } func (m *WatchChannelsResponse) XXX_Unmarshal(b []byte) error { @@ -3001,7 +3111,7 @@ func (m *SetSegmentStateRequest) Reset() { *m = SetSegmentStateRequest{} func (m *SetSegmentStateRequest) String() string { return proto.CompactTextString(m) } func (*SetSegmentStateRequest) ProtoMessage() {} func (*SetSegmentStateRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{48} + return fileDescriptor_82cd95f524594f49, []int{50} } func (m *SetSegmentStateRequest) XXX_Unmarshal(b []byte) error { @@ -3054,7 +3164,7 @@ func (m *SetSegmentStateResponse) Reset() { *m = SetSegmentStateResponse func (m *SetSegmentStateResponse) String() string { return proto.CompactTextString(m) } func (*SetSegmentStateResponse) ProtoMessage() {} func (*SetSegmentStateResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{49} + return fileDescriptor_82cd95f524594f49, []int{51} } func (m *SetSegmentStateResponse) XXX_Unmarshal(b []byte) error { @@ -3095,7 +3205,7 @@ func (m *DropVirtualChannelRequest) Reset() { *m = DropVirtualChannelReq func (m *DropVirtualChannelRequest) String() string { return proto.CompactTextString(m) } func (*DropVirtualChannelRequest) ProtoMessage() {} func (*DropVirtualChannelRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{50} + return fileDescriptor_82cd95f524594f49, []int{52} } func (m *DropVirtualChannelRequest) XXX_Unmarshal(b []byte) error { @@ -3155,7 +3265,7 @@ func (m *DropVirtualChannelSegment) Reset() { *m = DropVirtualChannelSeg func (m *DropVirtualChannelSegment) String() string { return proto.CompactTextString(m) } func (*DropVirtualChannelSegment) ProtoMessage() {} func (*DropVirtualChannelSegment) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{51} + return fileDescriptor_82cd95f524594f49, []int{53} } func (m *DropVirtualChannelSegment) XXX_Unmarshal(b []byte) error { @@ -3243,7 +3353,7 @@ func (m *DropVirtualChannelResponse) Reset() { *m = DropVirtualChannelRe func (m *DropVirtualChannelResponse) String() string { return proto.CompactTextString(m) } func (*DropVirtualChannelResponse) ProtoMessage() {} func (*DropVirtualChannelResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{52} + return fileDescriptor_82cd95f524594f49, []int{54} } func (m *DropVirtualChannelResponse) XXX_Unmarshal(b []byte) error { @@ -3289,7 +3399,7 @@ func (m *ImportTask) Reset() { *m = ImportTask{} } func (m *ImportTask) String() string { return proto.CompactTextString(m) } func (*ImportTask) ProtoMessage() {} func (*ImportTask) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{53} + return fileDescriptor_82cd95f524594f49, []int{55} } func (m *ImportTask) XXX_Unmarshal(b []byte) error { @@ -3381,7 +3491,7 @@ func (m *ImportTaskState) Reset() { *m = ImportTaskState{} } func (m *ImportTaskState) String() string { return proto.CompactTextString(m) } func (*ImportTaskState) ProtoMessage() {} func (*ImportTaskState) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{54} + return fileDescriptor_82cd95f524594f49, []int{56} } func (m *ImportTaskState) XXX_Unmarshal(b []byte) error { @@ -3460,7 +3570,7 @@ func (m *ImportTaskInfo) Reset() { *m = ImportTaskInfo{} } func (m *ImportTaskInfo) String() string { return proto.CompactTextString(m) } func (*ImportTaskInfo) ProtoMessage() {} func (*ImportTaskInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{55} + return fileDescriptor_82cd95f524594f49, []int{57} } func (m *ImportTaskInfo) XXX_Unmarshal(b []byte) error { @@ -3584,7 +3694,7 @@ func (m *ImportTaskResponse) Reset() { *m = ImportTaskResponse{} } func (m *ImportTaskResponse) String() string { return proto.CompactTextString(m) } func (*ImportTaskResponse) ProtoMessage() {} func (*ImportTaskResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{56} + return fileDescriptor_82cd95f524594f49, []int{58} } func (m *ImportTaskResponse) XXX_Unmarshal(b []byte) error { @@ -3632,7 +3742,7 @@ func (m *ImportTaskRequest) Reset() { *m = ImportTaskRequest{} } func (m *ImportTaskRequest) String() string { return proto.CompactTextString(m) } func (*ImportTaskRequest) ProtoMessage() {} func (*ImportTaskRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{57} + return fileDescriptor_82cd95f524594f49, []int{59} } func (m *ImportTaskRequest) XXX_Unmarshal(b []byte) error { @@ -3686,7 +3796,7 @@ func (m *UpdateSegmentStatisticsRequest) Reset() { *m = UpdateSegmentSta func (m *UpdateSegmentStatisticsRequest) String() string { return proto.CompactTextString(m) } func (*UpdateSegmentStatisticsRequest) ProtoMessage() {} func (*UpdateSegmentStatisticsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{58} + return fileDescriptor_82cd95f524594f49, []int{60} } func (m *UpdateSegmentStatisticsRequest) XXX_Unmarshal(b []byte) error { @@ -3732,7 +3842,7 @@ func (m *ResendSegmentStatsRequest) Reset() { *m = ResendSegmentStatsReq func (m *ResendSegmentStatsRequest) String() string { return proto.CompactTextString(m) } func (*ResendSegmentStatsRequest) ProtoMessage() {} func (*ResendSegmentStatsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{59} + return fileDescriptor_82cd95f524594f49, []int{61} } func (m *ResendSegmentStatsRequest) XXX_Unmarshal(b []byte) error { @@ -3772,7 +3882,7 @@ func (m *ResendSegmentStatsResponse) Reset() { *m = ResendSegmentStatsRe func (m *ResendSegmentStatsResponse) String() string { return proto.CompactTextString(m) } func (*ResendSegmentStatsResponse) ProtoMessage() {} func (*ResendSegmentStatsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_82cd95f524594f49, []int{60} + return fileDescriptor_82cd95f524594f49, []int{62} } func (m *ResendSegmentStatsResponse) XXX_Unmarshal(b []byte) error { @@ -3829,6 +3939,8 @@ func init() { proto.RegisterType((*GetPartitionStatisticsRequest)(nil), "milvus.proto.data.GetPartitionStatisticsRequest") proto.RegisterType((*GetPartitionStatisticsResponse)(nil), "milvus.proto.data.GetPartitionStatisticsResponse") proto.RegisterType((*GetSegmentInfoChannelRequest)(nil), "milvus.proto.data.GetSegmentInfoChannelRequest") + proto.RegisterType((*AcquireSegmentLockRequest)(nil), "milvus.proto.data.AcquireSegmentLockRequest") + proto.RegisterType((*ReleaseSegmentLockRequest)(nil), "milvus.proto.data.ReleaseSegmentLockRequest") proto.RegisterType((*VchannelInfo)(nil), "milvus.proto.data.VchannelInfo") proto.RegisterType((*WatchDmChannelsRequest)(nil), "milvus.proto.data.WatchDmChannelsRequest") proto.RegisterType((*FlushSegmentsRequest)(nil), "milvus.proto.data.FlushSegmentsRequest") @@ -3876,227 +3988,231 @@ func init() { func init() { proto.RegisterFile("data_coord.proto", fileDescriptor_82cd95f524594f49) } var fileDescriptor_82cd95f524594f49 = []byte{ - // 3518 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x1b, 0xcb, 0x6e, 0x1b, 0xd7, - 0xd5, 0xc3, 0x97, 0xc8, 0xc3, 0x87, 0xe8, 0x6b, 0x47, 0xa6, 0x69, 0x5b, 0x96, 0xc7, 0xb1, 0xa3, - 0x38, 0x8e, 0x9d, 0xc8, 0x09, 0x62, 0x34, 0x2f, 0xc4, 0x96, 0xad, 0x10, 0xb5, 0x54, 0x65, 0xa4, - 0xc4, 0x45, 0x53, 0x74, 0x30, 0xe2, 0x5c, 0x51, 0x13, 0x71, 0x66, 0xe8, 0x99, 0xa1, 0x65, 0x65, - 0x13, 0xa3, 0x01, 0x0a, 0xa4, 0x28, 0xfa, 0x40, 0x37, 0x6d, 0xd1, 0x45, 0xd1, 0x55, 0x1f, 0x9b, - 0x02, 0x41, 0x17, 0x6d, 0xd1, 0x4d, 0x57, 0x41, 0xbb, 0xe8, 0x27, 0x74, 0xd9, 0x5d, 0xbf, 0xa1, - 0xb8, 0x8f, 0xb9, 0xf3, 0xe0, 0x0c, 0x39, 0x22, 0xed, 0x78, 0xa7, 0x7b, 0xe6, 0x9c, 0x73, 0xcf, - 0x3d, 0xf7, 0xbc, 0x79, 0x05, 0x4d, 0x5d, 0xf3, 0x34, 0xb5, 0x6b, 0xdb, 0x8e, 0x7e, 0x6d, 0xe0, - 0xd8, 0x9e, 0x8d, 0x8e, 0x9b, 0x46, 0xff, 0xe1, 0xd0, 0x65, 0xab, 0x6b, 0xe4, 0x73, 0xbb, 0xd6, - 0xb5, 0x4d, 0xd3, 0xb6, 0x18, 0xa8, 0xdd, 0x30, 0x2c, 0x0f, 0x3b, 0x96, 0xd6, 0xe7, 0xeb, 0x5a, - 0x98, 0xa0, 0x5d, 0x73, 0xbb, 0x7b, 0xd8, 0xd4, 0xd8, 0x4a, 0x9e, 0x83, 0xe2, 0x1d, 0x73, 0xe0, - 0x1d, 0xca, 0xbf, 0x90, 0xa0, 0x76, 0xb7, 0x3f, 0x74, 0xf7, 0x14, 0xfc, 0x60, 0x88, 0x5d, 0x0f, - 0xbd, 0x02, 0x85, 0x1d, 0xcd, 0xc5, 0x2d, 0x69, 0x49, 0x5a, 0xae, 0xae, 0x9c, 0xbd, 0x16, 0xd9, - 0x95, 0xef, 0xb7, 0xee, 0xf6, 0x6e, 0x69, 0x2e, 0x56, 0x28, 0x26, 0x42, 0x50, 0xd0, 0x77, 0x3a, - 0xab, 0xad, 0xdc, 0x92, 0xb4, 0x9c, 0x57, 0xe8, 0xdf, 0x68, 0x11, 0xc0, 0xc5, 0x3d, 0x13, 0x5b, - 0x5e, 0x67, 0xd5, 0x6d, 0xe5, 0x97, 0xf2, 0xcb, 0x79, 0x25, 0x04, 0x41, 0x32, 0xd4, 0xba, 0x76, - 0xbf, 0x8f, 0xbb, 0x9e, 0x61, 0x5b, 0x9d, 0xd5, 0x56, 0x81, 0xd2, 0x46, 0x60, 0xf2, 0xaf, 0x25, - 0xa8, 0x73, 0xd1, 0xdc, 0x81, 0x6d, 0xb9, 0x18, 0xdd, 0x80, 0x92, 0xeb, 0x69, 0xde, 0xd0, 0xe5, - 0xd2, 0x9d, 0x49, 0x94, 0x6e, 0x8b, 0xa2, 0x28, 0x1c, 0x35, 0x51, 0xbc, 0xf8, 0xf6, 0xf9, 0xd1, - 0xed, 0x63, 0x47, 0x28, 0xc4, 0x8f, 0x20, 0xff, 0x4c, 0x82, 0xe6, 0x96, 0xbf, 0xf4, 0xb5, 0x77, - 0x12, 0x8a, 0x5d, 0x7b, 0x68, 0x79, 0x54, 0xc0, 0xba, 0xc2, 0x16, 0xe8, 0x02, 0xd4, 0xba, 0x7b, - 0x9a, 0x65, 0xe1, 0xbe, 0x6a, 0x69, 0x26, 0xa6, 0xa2, 0x54, 0x94, 0x2a, 0x87, 0x6d, 0x68, 0x26, - 0xce, 0x24, 0xd1, 0x12, 0x54, 0x07, 0x9a, 0xe3, 0x19, 0x11, 0x9d, 0x85, 0x41, 0xf2, 0x6f, 0x24, - 0x58, 0x78, 0xcf, 0x75, 0x8d, 0x9e, 0x35, 0x22, 0xd9, 0x02, 0x94, 0x2c, 0x5b, 0xc7, 0x9d, 0x55, - 0x2a, 0x5a, 0x5e, 0xe1, 0x2b, 0x74, 0x06, 0x2a, 0x03, 0x8c, 0x1d, 0xd5, 0xb1, 0xfb, 0xbe, 0x60, - 0x65, 0x02, 0x50, 0xec, 0x3e, 0x46, 0x1f, 0xc0, 0x71, 0x37, 0xc6, 0x88, 0xdd, 0x66, 0x75, 0xe5, - 0xe2, 0xb5, 0x11, 0x7b, 0xbc, 0x16, 0xdf, 0x54, 0x19, 0xa5, 0x96, 0x1f, 0xe7, 0xe0, 0x84, 0xc0, - 0x63, 0xb2, 0x92, 0xbf, 0x89, 0xe6, 0x5c, 0xdc, 0x13, 0xe2, 0xb1, 0x45, 0x16, 0xcd, 0x09, 0x95, - 0xe7, 0xc3, 0x2a, 0xcf, 0x60, 0x60, 0x71, 0x7d, 0x16, 0x47, 0xf4, 0x89, 0xce, 0x43, 0x15, 0x3f, - 0x1a, 0x18, 0x0e, 0x56, 0x3d, 0xc3, 0xc4, 0xad, 0xd2, 0x92, 0xb4, 0x5c, 0x50, 0x80, 0x81, 0xb6, - 0x0d, 0x33, 0x6c, 0x91, 0x73, 0x99, 0x2d, 0x52, 0xfe, 0xad, 0x04, 0xa7, 0x46, 0x6e, 0x89, 0x9b, - 0xb8, 0x02, 0x4d, 0x7a, 0xf2, 0x40, 0x33, 0xc4, 0xd8, 0x89, 0xc2, 0x2f, 0x8f, 0x53, 0x78, 0x80, - 0xae, 0x8c, 0xd0, 0x87, 0x84, 0xcc, 0x65, 0x17, 0x72, 0x1f, 0x4e, 0xad, 0x61, 0x8f, 0x6f, 0x40, - 0xbe, 0x61, 0x77, 0xfa, 0x10, 0x11, 0xf5, 0xa5, 0xdc, 0x88, 0x2f, 0xfd, 0x29, 0x27, 0x7c, 0x89, - 0x6e, 0xd5, 0xb1, 0x76, 0x6d, 0x74, 0x16, 0x2a, 0x02, 0x85, 0x5b, 0x45, 0x00, 0x40, 0x6f, 0x40, - 0x91, 0x48, 0xca, 0x4c, 0xa2, 0xb1, 0x72, 0x21, 0xf9, 0x4c, 0x21, 0x9e, 0x0a, 0xc3, 0x47, 0x1d, - 0x68, 0xb8, 0x9e, 0xe6, 0x78, 0xea, 0xc0, 0x76, 0xe9, 0x3d, 0x53, 0xc3, 0xa9, 0xae, 0xc8, 0x51, - 0x0e, 0x22, 0x98, 0xae, 0xbb, 0xbd, 0x4d, 0x8e, 0xa9, 0xd4, 0x29, 0xa5, 0xbf, 0x44, 0x77, 0xa0, - 0x86, 0x2d, 0x3d, 0x60, 0x54, 0xc8, 0xcc, 0xa8, 0x8a, 0x2d, 0x5d, 0xb0, 0x09, 0xee, 0xa7, 0x98, - 0xfd, 0x7e, 0x7e, 0x24, 0x41, 0x6b, 0xf4, 0x82, 0x66, 0x09, 0x94, 0x6f, 0x32, 0x22, 0xcc, 0x2e, - 0x68, 0xac, 0x87, 0x8b, 0x4b, 0x52, 0x38, 0x89, 0x6c, 0xc0, 0x73, 0x81, 0x34, 0xf4, 0xcb, 0x53, - 0x33, 0x96, 0xcf, 0x25, 0x58, 0x88, 0xef, 0x35, 0xcb, 0xb9, 0x5f, 0x83, 0xa2, 0x61, 0xed, 0xda, - 0xfe, 0xb1, 0x17, 0xc7, 0xf8, 0x19, 0xd9, 0x8b, 0x21, 0xcb, 0x26, 0x9c, 0x59, 0xc3, 0x5e, 0xc7, - 0x72, 0xb1, 0xe3, 0xdd, 0x32, 0xac, 0xbe, 0xdd, 0xdb, 0xd4, 0xbc, 0xbd, 0x19, 0x7c, 0x24, 0x62, - 0xee, 0xb9, 0x98, 0xb9, 0xcb, 0xbf, 0x93, 0xe0, 0x6c, 0xf2, 0x7e, 0xfc, 0xe8, 0x6d, 0x28, 0xef, - 0x1a, 0xb8, 0xaf, 0x13, 0x9d, 0x49, 0x54, 0x67, 0x62, 0x4d, 0x7c, 0x65, 0x40, 0x90, 0xf9, 0x09, - 0x2f, 0xa4, 0x18, 0xe8, 0x96, 0xe7, 0x18, 0x56, 0xef, 0x9e, 0xe1, 0x7a, 0x0a, 0xc3, 0x0f, 0xe9, - 0x33, 0x9f, 0xdd, 0x32, 0x7f, 0x28, 0xc1, 0xe2, 0x1a, 0xf6, 0x6e, 0x8b, 0x50, 0x4b, 0xbe, 0x1b, - 0xae, 0x67, 0x74, 0xdd, 0x27, 0x5b, 0x64, 0x64, 0xc8, 0x99, 0xf2, 0x4f, 0x24, 0x38, 0x9f, 0x2a, - 0x0c, 0x57, 0x1d, 0x0f, 0x25, 0x7e, 0xa0, 0x4d, 0x0e, 0x25, 0xdf, 0xc4, 0x87, 0x1f, 0x69, 0xfd, - 0x21, 0xde, 0xd4, 0x0c, 0x87, 0x85, 0x92, 0x29, 0x03, 0xeb, 0x1f, 0x25, 0x38, 0xb7, 0x86, 0xbd, - 0x4d, 0x3f, 0xcd, 0x3c, 0x43, 0xed, 0x64, 0xa8, 0x28, 0x7e, 0xcc, 0x2e, 0x33, 0x51, 0xda, 0x67, - 0xa2, 0xbe, 0x45, 0xea, 0x07, 0x21, 0x87, 0xbc, 0xcd, 0x6a, 0x01, 0xae, 0x3c, 0xf9, 0x71, 0x1e, - 0x6a, 0x1f, 0xf1, 0xfa, 0x80, 0xa6, 0x91, 0xb8, 0x1e, 0xa4, 0x64, 0x3d, 0x84, 0x4a, 0x8a, 0xa4, - 0x2a, 0x63, 0x0d, 0xea, 0x2e, 0xc6, 0xfb, 0xd3, 0x24, 0x8d, 0x1a, 0x21, 0x14, 0xc1, 0xfe, 0x1e, - 0x1c, 0x1f, 0x5a, 0xbb, 0xa4, 0xac, 0xc5, 0x3a, 0x3f, 0x05, 0xab, 0x2e, 0x27, 0x47, 0x9e, 0x51, - 0x42, 0xf4, 0x3e, 0xcc, 0xc7, 0x79, 0x15, 0x33, 0xf1, 0x8a, 0x93, 0xa1, 0x0e, 0x34, 0x75, 0xc7, - 0x1e, 0x0c, 0xb0, 0xae, 0xba, 0x3e, 0xab, 0x52, 0x36, 0x56, 0x9c, 0xce, 0x67, 0x25, 0x7f, 0x21, - 0xc1, 0xc2, 0x7d, 0xcd, 0xeb, 0xee, 0xad, 0x9a, 0xfc, 0x72, 0x66, 0x30, 0xed, 0xb7, 0xa1, 0xf2, - 0x90, 0x5f, 0x84, 0x1f, 0xbf, 0xce, 0x27, 0x08, 0x14, 0xbe, 0x72, 0x25, 0xa0, 0x90, 0xbf, 0x92, - 0xe0, 0x24, 0x6d, 0x22, 0x7c, 0xe9, 0xbe, 0x7e, 0x27, 0x9b, 0xd0, 0x48, 0xa0, 0xcb, 0xd0, 0x30, - 0x35, 0x67, 0x7f, 0x2b, 0xc0, 0x29, 0x52, 0x9c, 0x18, 0x54, 0x7e, 0x04, 0xc0, 0x57, 0xeb, 0x6e, - 0x6f, 0x0a, 0xf9, 0x6f, 0xc2, 0x1c, 0xdf, 0x95, 0xfb, 0xdb, 0xa4, 0x8b, 0xf5, 0xd1, 0xe5, 0x7f, - 0x4a, 0xd0, 0x08, 0x22, 0x28, 0xf5, 0xaa, 0x06, 0xe4, 0x84, 0x2f, 0xe5, 0x3a, 0xab, 0xe8, 0x6d, - 0x28, 0xb1, 0x06, 0x93, 0xf3, 0xbe, 0x14, 0xe5, 0xcd, 0x9b, 0xcf, 0x50, 0x18, 0xa6, 0x00, 0x85, - 0x13, 0x11, 0x1d, 0x89, 0xa8, 0x23, 0xfa, 0xc5, 0x00, 0x82, 0x3a, 0x30, 0x1f, 0x2d, 0xda, 0x7c, - 0x9f, 0x59, 0x4a, 0x8b, 0x36, 0xab, 0x9a, 0xa7, 0xd1, 0x60, 0xd3, 0x88, 0xd4, 0x6c, 0xae, 0xfc, - 0xbf, 0x22, 0x54, 0x43, 0xa7, 0x1c, 0x39, 0x49, 0xfc, 0x4a, 0x73, 0x93, 0xe3, 0x66, 0x7e, 0xb4, - 0x73, 0xb8, 0x04, 0x0d, 0x83, 0xe6, 0x6a, 0x95, 0x9b, 0x22, 0x0d, 0xae, 0x15, 0xa5, 0xce, 0xa0, - 0xdc, 0x2f, 0xd0, 0x22, 0x54, 0xad, 0xa1, 0xa9, 0xda, 0xbb, 0xaa, 0x63, 0x1f, 0xb8, 0xbc, 0x05, - 0xa9, 0x58, 0x43, 0xf3, 0x5b, 0xbb, 0x8a, 0x7d, 0xe0, 0x06, 0x55, 0x6e, 0xe9, 0x88, 0x55, 0xee, - 0x22, 0x54, 0x4d, 0xed, 0x11, 0xe1, 0xaa, 0x5a, 0x43, 0x93, 0x76, 0x27, 0x79, 0xa5, 0x62, 0x6a, - 0x8f, 0x14, 0xfb, 0x60, 0x63, 0x68, 0xa2, 0x65, 0x68, 0xf6, 0x35, 0xd7, 0x53, 0xc3, 0xed, 0x4d, - 0x99, 0xb6, 0x37, 0x0d, 0x02, 0xbf, 0x13, 0xb4, 0x38, 0xa3, 0xf5, 0x72, 0x65, 0x86, 0x7a, 0x59, - 0x37, 0xfb, 0x01, 0x23, 0xc8, 0x5e, 0x2f, 0xeb, 0x66, 0x5f, 0xb0, 0xb9, 0x09, 0x73, 0x3b, 0xb4, - 0x02, 0x72, 0x5b, 0xd5, 0xd4, 0x08, 0x75, 0x97, 0x14, 0x3f, 0xac, 0x50, 0x52, 0x7c, 0x74, 0xf4, - 0x16, 0x54, 0x68, 0xea, 0xa1, 0xb4, 0xb5, 0x4c, 0xb4, 0x01, 0x01, 0xa1, 0xd6, 0x71, 0xdf, 0xd3, - 0x28, 0x75, 0x3d, 0x1b, 0xb5, 0x20, 0x40, 0xaf, 0xc0, 0x89, 0xae, 0x83, 0x35, 0x0f, 0xeb, 0xb7, - 0x0e, 0x6f, 0xdb, 0xe6, 0x40, 0xa3, 0xc6, 0xd4, 0x6a, 0x2c, 0x49, 0xcb, 0x65, 0x25, 0xe9, 0x13, - 0x09, 0x0c, 0x5d, 0xb1, 0xba, 0xeb, 0xd8, 0x66, 0x6b, 0x9e, 0x05, 0x86, 0x28, 0x14, 0x9d, 0x03, - 0xf0, 0x43, 0xb7, 0xe6, 0xb5, 0x9a, 0xf4, 0x16, 0x2b, 0x1c, 0xf2, 0x9e, 0x27, 0x7f, 0x06, 0x27, - 0x03, 0x0b, 0x09, 0xdd, 0xc6, 0xe8, 0xc5, 0x4a, 0xd3, 0x5e, 0xec, 0xf8, 0xda, 0xf5, 0xdf, 0x05, - 0x58, 0xd8, 0xd2, 0x1e, 0xe2, 0xa7, 0x5f, 0x26, 0x67, 0x8a, 0xc7, 0xf7, 0xe0, 0x38, 0xad, 0x8c, - 0x57, 0x42, 0xf2, 0x8c, 0xc9, 0xc0, 0xe1, 0xeb, 0x1c, 0x25, 0x44, 0xef, 0x92, 0xd2, 0x01, 0x77, - 0xf7, 0x37, 0x6d, 0x23, 0xc8, 0xbe, 0xe7, 0x12, 0xf8, 0xdc, 0x16, 0x58, 0x4a, 0x98, 0x02, 0x6d, - 0x8e, 0x86, 0x36, 0x96, 0x77, 0x5f, 0x18, 0xdb, 0x7f, 0x05, 0xda, 0x8f, 0x47, 0x38, 0xd4, 0x82, - 0x39, 0x9e, 0xdd, 0xa9, 0xdf, 0x97, 0x15, 0x7f, 0x89, 0x36, 0xe1, 0x04, 0x3b, 0xc1, 0x16, 0x37, - 0x6a, 0x76, 0xf8, 0x72, 0xa6, 0xc3, 0x27, 0x91, 0x46, 0x7d, 0xa2, 0x72, 0x54, 0x9f, 0x68, 0xc1, - 0x1c, 0xb7, 0x53, 0x1a, 0x0b, 0xca, 0x8a, 0xbf, 0x24, 0xd7, 0x6c, 0x98, 0x03, 0xdb, 0xf1, 0x0c, - 0xab, 0xd7, 0xaa, 0xd2, 0x6f, 0x01, 0x80, 0xb4, 0x18, 0x10, 0xe8, 0x73, 0xc2, 0xa4, 0xe0, 0x1d, - 0x28, 0x0b, 0x0b, 0xcf, 0x65, 0xb6, 0x70, 0x41, 0x13, 0x8f, 0xd1, 0xf9, 0x58, 0x8c, 0x96, 0xff, - 0x25, 0x41, 0x6d, 0x95, 0x1c, 0xe9, 0x9e, 0xdd, 0xa3, 0x19, 0xe5, 0x12, 0x34, 0x1c, 0xdc, 0xb5, - 0x1d, 0x5d, 0xc5, 0x96, 0xe7, 0x18, 0x98, 0x75, 0xa3, 0x05, 0xa5, 0xce, 0xa0, 0x77, 0x18, 0x90, - 0xa0, 0x91, 0xb0, 0xeb, 0x7a, 0x9a, 0x39, 0x50, 0x77, 0x89, 0x7b, 0xe7, 0x18, 0x9a, 0x80, 0x52, - 0xef, 0xbe, 0x00, 0xb5, 0x00, 0xcd, 0xb3, 0xe9, 0xfe, 0x05, 0xa5, 0x2a, 0x60, 0xdb, 0x36, 0x7a, - 0x1e, 0x1a, 0x54, 0xa7, 0x6a, 0xdf, 0xee, 0xa9, 0xa4, 0x73, 0xe3, 0xc9, 0xa6, 0xa6, 0x73, 0xb1, - 0xc8, 0x5d, 0x45, 0xb1, 0x5c, 0xe3, 0x53, 0xcc, 0xd3, 0x8d, 0xc0, 0xda, 0x32, 0x3e, 0xc5, 0x24, - 0xd7, 0xd7, 0x49, 0xee, 0xdc, 0xb0, 0x75, 0xbc, 0x3d, 0x65, 0xa5, 0x91, 0x61, 0x6a, 0x77, 0x16, - 0x2a, 0xe2, 0x04, 0xfc, 0x48, 0x01, 0x00, 0xdd, 0x85, 0x86, 0x5f, 0x84, 0xaa, 0xac, 0xb7, 0x28, - 0xa4, 0x56, 0x7e, 0xa1, 0xec, 0xe7, 0x2a, 0x75, 0x9f, 0x8c, 0x2e, 0xe5, 0xbb, 0x50, 0x0b, 0x7f, - 0x26, 0xbb, 0x6e, 0xc5, 0x0d, 0x45, 0x00, 0x88, 0x35, 0x6e, 0x0c, 0x4d, 0x72, 0xa7, 0x3c, 0xb0, - 0xf8, 0x4b, 0xf9, 0x73, 0x09, 0xea, 0x3c, 0x65, 0x6f, 0x89, 0xa9, 0x32, 0x3d, 0x9a, 0x44, 0x8f, - 0x46, 0xff, 0x46, 0xdf, 0x88, 0x8e, 0xa4, 0x9e, 0x4f, 0x0c, 0x02, 0x94, 0x09, 0xad, 0x8e, 0x23, - 0xf9, 0x3a, 0x4b, 0x2f, 0xfb, 0x98, 0x18, 0x1a, 0xbf, 0x1a, 0x6a, 0x68, 0x2d, 0x98, 0xd3, 0x74, - 0xdd, 0xc1, 0xae, 0xcb, 0xe5, 0xf0, 0x97, 0xe4, 0xcb, 0x43, 0xec, 0xb8, 0xbe, 0xc9, 0xe7, 0x15, - 0x7f, 0x89, 0xde, 0x82, 0xb2, 0x28, 0xa7, 0xf3, 0x49, 0x25, 0x54, 0x58, 0x4e, 0xde, 0x7b, 0x09, - 0x0a, 0xf9, 0xcf, 0x39, 0x68, 0x70, 0x85, 0xdd, 0xe2, 0x39, 0x75, 0xbc, 0xf3, 0xdd, 0x82, 0xda, - 0x6e, 0xe0, 0xfb, 0xe3, 0x66, 0x2c, 0xe1, 0x10, 0x11, 0xa1, 0x99, 0xe4, 0x80, 0xd1, 0xac, 0x5e, - 0x98, 0x29, 0xab, 0x17, 0x8f, 0x1a, 0xc1, 0x46, 0xeb, 0xbc, 0x52, 0x42, 0x9d, 0x27, 0x7f, 0x17, - 0xaa, 0x21, 0x06, 0x34, 0x42, 0xb3, 0xe1, 0x0c, 0xd7, 0x98, 0xbf, 0x44, 0x37, 0x82, 0xda, 0x86, - 0xa9, 0xea, 0x74, 0x82, 0x2c, 0xb1, 0xb2, 0x46, 0xfe, 0xbd, 0x04, 0x25, 0xce, 0xf9, 0x3c, 0x54, - 0x79, 0xd0, 0xa1, 0x75, 0x1f, 0xe3, 0x0e, 0x1c, 0x44, 0x0a, 0xbf, 0x27, 0x17, 0x75, 0x4e, 0x43, - 0x39, 0x16, 0x6f, 0xe6, 0x78, 0x5a, 0xf0, 0x3f, 0x85, 0x82, 0x0c, 0xf9, 0x44, 0xe3, 0xcb, 0x57, - 0x12, 0x1d, 0x2c, 0x2b, 0xb8, 0x6b, 0x3f, 0xc4, 0xce, 0xe1, 0xec, 0xe3, 0xbb, 0x37, 0x43, 0x06, - 0x9d, 0xb1, 0x3f, 0x14, 0x04, 0xe8, 0xcd, 0x40, 0xdd, 0xf9, 0xa4, 0xe9, 0x45, 0x38, 0xc2, 0x70, - 0x73, 0x0c, 0xd4, 0xfe, 0x53, 0x36, 0x88, 0x8c, 0x1e, 0x65, 0xda, 0xba, 0xe6, 0x89, 0xb4, 0x1d, - 0xf2, 0xcf, 0x25, 0x38, 0xbd, 0x86, 0xbd, 0xbb, 0xd1, 0xe6, 0xfe, 0x59, 0x4b, 0x65, 0x42, 0x3b, - 0x49, 0xa8, 0x59, 0x6e, 0xbd, 0x0d, 0x65, 0x31, 0xa6, 0x60, 0x23, 0x62, 0xb1, 0x96, 0x7f, 0x20, - 0x41, 0x8b, 0xef, 0x42, 0xf7, 0x24, 0x25, 0x75, 0x1f, 0x7b, 0x58, 0xff, 0xba, 0xfb, 0xe6, 0xbf, - 0x4b, 0xd0, 0x0c, 0x47, 0x7c, 0x1a, 0xb4, 0x5f, 0x87, 0x22, 0x1d, 0x4f, 0x70, 0x09, 0x26, 0x1a, - 0x2b, 0xc3, 0x26, 0x21, 0x83, 0x96, 0x79, 0xdb, 0x22, 0x39, 0xf1, 0x65, 0x90, 0x76, 0xf2, 0x47, - 0x4f, 0x3b, 0x3c, 0x0d, 0xdb, 0x43, 0xc2, 0x97, 0x8d, 0xff, 0x02, 0x80, 0xfc, 0x65, 0x0e, 0x5a, - 0x41, 0x3f, 0xf2, 0xb5, 0xc7, 0xfd, 0x94, 0x6a, 0x35, 0xff, 0x84, 0xaa, 0xd5, 0xc2, 0xec, 0xb1, - 0xbe, 0x98, 0x14, 0xeb, 0xff, 0x96, 0x83, 0x46, 0xa0, 0xb5, 0xcd, 0xbe, 0x66, 0xa1, 0x05, 0x28, - 0x0d, 0xfa, 0x5a, 0x30, 0x7d, 0xe4, 0x2b, 0xb4, 0x25, 0xea, 0x9c, 0xa8, 0x9e, 0x5e, 0x4a, 0xba, - 0xc3, 0x94, 0x8b, 0x50, 0x62, 0x2c, 0x48, 0x3b, 0xc8, 0x1a, 0x0a, 0xda, 0xd4, 0xf3, 0xda, 0x8a, - 0x19, 0x0b, 0xe9, 0xe7, 0xaf, 0x02, 0xe2, 0x37, 0xac, 0x1a, 0x96, 0xea, 0xe2, 0xae, 0x6d, 0xe9, - 0xec, 0xee, 0x8b, 0x4a, 0x93, 0x7f, 0xe9, 0x58, 0x5b, 0x0c, 0x8e, 0x5e, 0x87, 0x82, 0x77, 0x38, - 0x60, 0x51, 0xbc, 0x91, 0x18, 0x1d, 0x03, 0xb9, 0xb6, 0x0f, 0x07, 0x58, 0xa1, 0xe8, 0x68, 0x11, - 0x80, 0xb0, 0xf2, 0x1c, 0xed, 0x21, 0x4f, 0x89, 0x05, 0x25, 0x04, 0x21, 0xd6, 0xec, 0xeb, 0x70, - 0x8e, 0xa5, 0x0e, 0xbe, 0x94, 0xff, 0x92, 0x83, 0x66, 0xc0, 0x52, 0xc1, 0xee, 0xb0, 0xef, 0xa5, - 0xea, 0x6f, 0x7c, 0x33, 0x38, 0xa9, 0x6e, 0x78, 0x17, 0xaa, 0xfc, 0x3e, 0x8f, 0x60, 0x0f, 0xc0, - 0x48, 0xee, 0x8d, 0x31, 0xd0, 0xe2, 0x13, 0x32, 0xd0, 0xd2, 0x11, 0x0d, 0x54, 0xde, 0x82, 0x05, - 0x3f, 0xee, 0x05, 0x08, 0xeb, 0xd8, 0xd3, 0xc6, 0x14, 0x1c, 0xe7, 0xa1, 0xca, 0xf2, 0x19, 0x4b, - 0xe4, 0xac, 0x54, 0x87, 0x1d, 0xd1, 0xe1, 0xca, 0xdf, 0x83, 0x93, 0x34, 0x6e, 0xc4, 0x47, 0xb9, - 0x59, 0xe6, 0xea, 0xb2, 0x68, 0x04, 0x48, 0xd1, 0xcf, 0xac, 0xbb, 0xa2, 0x44, 0x60, 0xf2, 0x3d, - 0x78, 0x2e, 0xc6, 0x7f, 0x86, 0xbc, 0x40, 0x4a, 0xa1, 0x85, 0xad, 0xe8, 0xcf, 0xa2, 0xd3, 0x67, - 0xbf, 0x73, 0x62, 0x72, 0xab, 0x1a, 0x7a, 0xdc, 0xbe, 0x74, 0xf4, 0x0e, 0x54, 0x2c, 0x7c, 0xa0, - 0x86, 0x83, 0x6f, 0x86, 0x01, 0x5d, 0xd9, 0xc2, 0x07, 0xf4, 0x2f, 0x79, 0x03, 0x4e, 0x8d, 0x88, - 0x3a, 0xcb, 0xd9, 0xff, 0x2a, 0xc1, 0xe9, 0x55, 0xc7, 0x1e, 0x7c, 0x64, 0x38, 0xde, 0x50, 0xeb, - 0x47, 0x7f, 0x18, 0x79, 0x3a, 0x6d, 0xdc, 0xfb, 0xa1, 0x34, 0xcc, 0xe2, 0xf2, 0xd5, 0x04, 0x73, - 0x1d, 0x15, 0x8a, 0x1f, 0x3a, 0x94, 0xb4, 0xff, 0x9b, 0x4f, 0x12, 0x9e, 0xe3, 0x4d, 0x48, 0x36, - 0x59, 0xaa, 0x94, 0xc4, 0xa9, 0x4f, 0x7e, 0xda, 0xa9, 0x4f, 0x8a, 0xe7, 0x17, 0x9e, 0x90, 0xe7, - 0x1f, 0xb9, 0x0d, 0x79, 0x1f, 0xa2, 0x13, 0x39, 0x1a, 0x72, 0xa7, 0x1a, 0xe5, 0xdd, 0x02, 0x08, - 0xa6, 0x53, 0xfc, 0x55, 0x4b, 0x16, 0x36, 0x21, 0x2a, 0x72, 0x5b, 0x22, 0xca, 0xd2, 0xa9, 0x72, - 0x64, 0x5e, 0xf2, 0x01, 0xb4, 0x93, 0xac, 0x74, 0x16, 0xcb, 0xff, 0x32, 0x07, 0xd0, 0xa1, 0xd3, - 0xa1, 0x6d, 0xcd, 0xdd, 0x9f, 0xae, 0xa2, 0xbc, 0x08, 0xf5, 0xc0, 0x60, 0x02, 0x7f, 0x0f, 0x5b, - 0x91, 0x4e, 0x5c, 0x42, 0x14, 0xb6, 0x04, 0x67, 0xa4, 0xd8, 0xd5, 0x29, 0x9f, 0x90, 0xd7, 0x30, - 0xa3, 0x88, 0x05, 0x3d, 0x74, 0x06, 0x2a, 0x8e, 0x7d, 0xa0, 0x12, 0x37, 0xd3, 0x69, 0x6e, 0x2d, - 0x2b, 0x65, 0xc7, 0x3e, 0x20, 0xce, 0xa7, 0xa3, 0x53, 0x30, 0xe7, 0x69, 0xee, 0x3e, 0xe1, 0x5f, - 0x62, 0xe9, 0x8e, 0x2c, 0x3b, 0x3a, 0x3a, 0x09, 0xc5, 0x5d, 0xa3, 0x8f, 0xdd, 0xd6, 0x1c, 0x65, - 0xc9, 0x16, 0xe8, 0x0d, 0xff, 0xfd, 0x42, 0x39, 0xf3, 0xef, 0xaf, 0xec, 0x09, 0xc3, 0x57, 0x12, - 0xcc, 0x07, 0x5a, 0xa3, 0x01, 0x88, 0xc4, 0x34, 0x1a, 0xcf, 0x6e, 0xdb, 0x3a, 0x0b, 0x15, 0x8d, - 0x94, 0x9f, 0x58, 0x18, 0x21, 0x8b, 0x5a, 0x01, 0xc9, 0xb8, 0xba, 0x9c, 0x9c, 0x8b, 0x1c, 0xda, - 0xd0, 0xfd, 0x5f, 0x78, 0x4a, 0x8e, 0x7d, 0xd0, 0xd1, 0x85, 0x36, 0xd8, 0x33, 0x2e, 0x56, 0x85, - 0x12, 0x6d, 0xdc, 0xa6, 0x2f, 0xb9, 0x2e, 0x42, 0x1d, 0x3b, 0x8e, 0xed, 0xa8, 0x26, 0x76, 0x5d, - 0xad, 0x87, 0x79, 0xd1, 0x55, 0xa3, 0xc0, 0x75, 0x06, 0x93, 0xff, 0x93, 0x87, 0x46, 0x70, 0x14, - 0xff, 0x77, 0x1d, 0x43, 0xf7, 0x7f, 0xd7, 0x31, 0x74, 0x12, 0xcc, 0x1d, 0x16, 0x0a, 0x43, 0xc1, - 0x9c, 0x43, 0x3a, 0x3a, 0xc9, 0x83, 0xc4, 0xc1, 0x2c, 0x5b, 0xc7, 0xc1, 0xc5, 0x82, 0x0f, 0xe2, - 0xf7, 0x1a, 0xb1, 0x8f, 0x42, 0x06, 0xfb, 0x28, 0x66, 0xb0, 0x8f, 0x52, 0x82, 0x7d, 0x2c, 0x40, - 0x69, 0x67, 0xd8, 0xdd, 0xc7, 0x1e, 0x2f, 0x8f, 0xf8, 0x2a, 0x6a, 0x37, 0xe5, 0x98, 0xdd, 0x08, - 0xf3, 0xa8, 0x84, 0xcd, 0xe3, 0x0c, 0x54, 0xd8, 0x8f, 0x0b, 0xaa, 0xe7, 0xd2, 0x29, 0x6b, 0x5e, - 0x29, 0x33, 0xc0, 0xb6, 0x8b, 0x6e, 0xfa, 0xbd, 0x43, 0x35, 0xc9, 0xd1, 0x69, 0xc4, 0x89, 0x59, - 0x88, 0xdf, 0x39, 0xdc, 0x84, 0xd6, 0x1e, 0x1e, 0x3a, 0xf4, 0x2d, 0x80, 0x4a, 0x1f, 0xab, 0x3e, - 0x18, 0x62, 0xe7, 0x50, 0xdb, 0xe9, 0xe3, 0x56, 0x8d, 0x0a, 0xb6, 0x20, 0xbe, 0xaf, 0x6a, 0x9e, - 0xf6, 0x81, 0xff, 0x15, 0xbd, 0x06, 0x0b, 0x31, 0x4a, 0xc3, 0xd2, 0xf1, 0x23, 0xac, 0xb7, 0xea, - 0x94, 0xee, 0x64, 0x84, 0xae, 0xc3, 0xbe, 0xc9, 0x9f, 0x00, 0x0a, 0x24, 0x99, 0xad, 0x77, 0x8c, - 0x5d, 0x75, 0x2e, 0x7e, 0xd5, 0xf2, 0x1f, 0x24, 0x38, 0x1e, 0xde, 0x6c, 0xda, 0x04, 0xfa, 0x0e, - 0x54, 0xd9, 0xcc, 0x5a, 0x25, 0x0e, 0xcc, 0xbb, 0xc7, 0x73, 0x63, 0x75, 0xac, 0x80, 0x11, 0xc4, - 0xb1, 0x8b, 0x50, 0x3f, 0xb0, 0x9d, 0x7d, 0xc3, 0xea, 0xa9, 0x44, 0x32, 0xdf, 0x6d, 0x6a, 0x1c, - 0xb8, 0x41, 0x60, 0xf2, 0x17, 0x12, 0x2c, 0x7e, 0x38, 0xd0, 0x35, 0x0f, 0x87, 0x2a, 0x89, 0x59, - 0x1f, 0x94, 0xbc, 0xee, 0xbf, 0xe9, 0xc8, 0x65, 0x9b, 0xbb, 0x32, 0x6c, 0x79, 0x1d, 0x4e, 0x2b, - 0xd8, 0xc5, 0x96, 0x1e, 0xf9, 0x38, 0xad, 0x14, 0xf2, 0x00, 0xda, 0x49, 0xec, 0x66, 0xb9, 0x7b, - 0x56, 0xd2, 0xa9, 0x0e, 0x61, 0xeb, 0xf1, 0x08, 0x45, 0x2a, 0x09, 0xba, 0x8f, 0x77, 0xe5, 0x97, - 0x12, 0x1c, 0x1f, 0x69, 0x96, 0x51, 0x03, 0xe0, 0x43, 0xab, 0xcb, 0xa7, 0x08, 0xcd, 0x63, 0xa8, - 0x06, 0x65, 0x7f, 0xa6, 0xd0, 0x94, 0x50, 0x15, 0xe6, 0xb6, 0x6d, 0x8a, 0xdd, 0xcc, 0xa1, 0x26, - 0xd4, 0x18, 0xe1, 0xb0, 0xdb, 0xc5, 0xae, 0xdb, 0xcc, 0x0b, 0xc8, 0x5d, 0xcd, 0xe8, 0x0f, 0x1d, - 0xdc, 0x2c, 0xa0, 0x3a, 0x54, 0xb6, 0x6d, 0x05, 0xf7, 0xb1, 0xe6, 0xe2, 0x66, 0x11, 0x21, 0x68, - 0xf0, 0x85, 0x4f, 0x54, 0x0a, 0xc1, 0x7c, 0xb2, 0xb9, 0x2b, 0xbb, 0xe1, 0xb6, 0x92, 0xf4, 0x5a, - 0xe8, 0x14, 0x9c, 0xf8, 0xd0, 0xd2, 0xf1, 0xae, 0x61, 0x61, 0x3d, 0xf8, 0xd4, 0x3c, 0x86, 0x4e, - 0xc0, 0x7c, 0xc7, 0xb2, 0xb0, 0x13, 0x02, 0x4a, 0x04, 0xb8, 0x8e, 0x9d, 0x1e, 0x0e, 0x01, 0x73, - 0xe8, 0x38, 0xd4, 0xd7, 0x8d, 0x47, 0x21, 0x50, 0x7e, 0xe5, 0x1f, 0xcf, 0x41, 0x85, 0x78, 0xde, - 0x6d, 0xdb, 0x76, 0x74, 0x34, 0x00, 0x44, 0xdf, 0x4f, 0x99, 0x03, 0xdb, 0x12, 0x0f, 0x0d, 0xd1, - 0x2b, 0x29, 0x15, 0xc1, 0x28, 0x2a, 0xbf, 0xfd, 0xf6, 0xe5, 0x14, 0x8a, 0x18, 0xba, 0x7c, 0x0c, - 0x99, 0x74, 0x47, 0xd2, 0xab, 0x6e, 0x1b, 0xdd, 0x7d, 0xff, 0x97, 0xf2, 0x31, 0x3b, 0xc6, 0x50, - 0xfd, 0x1d, 0x63, 0xef, 0x17, 0xf9, 0x82, 0x3d, 0x72, 0xf3, 0x8d, 0x48, 0x3e, 0x86, 0x1e, 0xc0, - 0xc9, 0x35, 0x1c, 0x72, 0x1a, 0x7f, 0xc3, 0x95, 0xf4, 0x0d, 0x47, 0x90, 0x8f, 0xb8, 0xe5, 0x3d, - 0x28, 0xd2, 0xc1, 0x14, 0x4a, 0xf2, 0xab, 0xf0, 0x6b, 0xfc, 0xf6, 0x52, 0x3a, 0x82, 0xe0, 0xf6, - 0x09, 0xcc, 0xc7, 0x5e, 0x13, 0xa3, 0x17, 0x13, 0xc8, 0x92, 0xdf, 0x85, 0xb7, 0xaf, 0x64, 0x41, - 0x15, 0x7b, 0xf5, 0xa0, 0x11, 0x7d, 0x7d, 0x85, 0x96, 0x13, 0xe8, 0x13, 0x5f, 0x82, 0xb6, 0x5f, - 0xcc, 0x80, 0x29, 0x36, 0x32, 0xa1, 0x19, 0x7f, 0xdd, 0x8a, 0xae, 0x8c, 0x65, 0x10, 0x35, 0xb7, - 0x97, 0x32, 0xe1, 0x8a, 0xed, 0x0e, 0xa9, 0x11, 0x8c, 0xbc, 0xae, 0x44, 0xd7, 0x92, 0xd9, 0xa4, - 0x3d, 0xfb, 0x6c, 0x5f, 0xcf, 0x8c, 0x2f, 0xb6, 0xfe, 0x3e, 0x1b, 0x88, 0x27, 0xbd, 0x50, 0x44, - 0xaf, 0x26, 0xb3, 0x1b, 0xf3, 0xb4, 0xb2, 0xbd, 0x72, 0x14, 0x12, 0x21, 0xc4, 0x67, 0x74, 0x92, - 0x9d, 0xf0, 0xca, 0x2f, 0xee, 0x77, 0x3e, 0xbf, 0xf4, 0xe7, 0x8b, 0xed, 0x57, 0x8f, 0x40, 0x21, - 0x04, 0xb0, 0xe3, 0xef, 0x87, 0x7d, 0x37, 0xbc, 0x3e, 0xd1, 0x6a, 0xa6, 0xf3, 0xc1, 0x8f, 0x61, - 0x3e, 0xf6, 0x26, 0x21, 0xd1, 0x6b, 0x92, 0xdf, 0x2d, 0xb4, 0xc7, 0xe5, 0x1a, 0xe6, 0x92, 0xb1, - 0x1f, 0x06, 0x50, 0x8a, 0xf5, 0x27, 0xfc, 0x78, 0xd0, 0xbe, 0x92, 0x05, 0x55, 0x1c, 0xc4, 0xa5, - 0xe1, 0x32, 0x36, 0x5c, 0x47, 0x57, 0x93, 0x79, 0x24, 0xff, 0x30, 0xd0, 0x7e, 0x39, 0x23, 0xb6, - 0xd8, 0x54, 0x05, 0x58, 0xc3, 0xde, 0x3a, 0xf6, 0x1c, 0x62, 0x23, 0x97, 0x13, 0x55, 0x1e, 0x20, - 0xf8, 0xdb, 0xbc, 0x30, 0x11, 0x4f, 0x6c, 0xf0, 0x6d, 0x40, 0x7e, 0x8a, 0x0d, 0xbd, 0x88, 0xb9, - 0x38, 0x76, 0xfe, 0xc8, 0x86, 0x85, 0x93, 0xee, 0xe6, 0x01, 0x34, 0xd7, 0x35, 0x8b, 0x74, 0x9e, - 0x01, 0xdf, 0xab, 0x89, 0x82, 0xc5, 0xd1, 0x52, 0xb4, 0x95, 0x8a, 0x2d, 0x0e, 0x73, 0x20, 0x72, - 0xa8, 0x26, 0x5c, 0x10, 0xc7, 0x63, 0x4b, 0xa0, 0x8d, 0x18, 0x62, 0x4a, 0x6c, 0x19, 0x83, 0x2f, - 0x36, 0x7e, 0x2c, 0xd1, 0x57, 0xea, 0x31, 0x84, 0xfb, 0x86, 0xb7, 0xb7, 0xd9, 0xd7, 0x2c, 0x37, - 0x8b, 0x08, 0x14, 0xf1, 0x08, 0x22, 0x70, 0x7c, 0x21, 0x82, 0x0e, 0xf5, 0xc8, 0x78, 0x0f, 0x25, - 0x3d, 0x6b, 0x49, 0x1a, 0x30, 0xb6, 0x97, 0x27, 0x23, 0x8a, 0x5d, 0xf6, 0xa0, 0xee, 0xdb, 0x2b, - 0x53, 0xee, 0x8b, 0x69, 0x92, 0x06, 0x38, 0x29, 0xee, 0x96, 0x8c, 0x1a, 0x76, 0xb7, 0xd1, 0xe9, - 0x05, 0xca, 0x36, 0xf5, 0x1a, 0xe7, 0x6e, 0xe9, 0x23, 0x11, 0x16, 0x4f, 0x62, 0x93, 0xc2, 0xe4, - 0x60, 0x95, 0x38, 0xf8, 0x4c, 0x8c, 0x27, 0x29, 0x83, 0x47, 0xf9, 0x18, 0xba, 0x0f, 0x25, 0xd6, - 0x8e, 0xa0, 0xe7, 0xc7, 0x77, 0x2a, 0x9c, 0xfb, 0xa5, 0x09, 0x58, 0x82, 0xf1, 0x3e, 0x9c, 0x4a, - 0xe9, 0x53, 0x12, 0xf3, 0xdc, 0xf8, 0x9e, 0x66, 0x82, 0x97, 0xaf, 0xfc, 0xaa, 0x04, 0x65, 0xff, - 0xad, 0xc4, 0x33, 0xa8, 0x61, 0x9f, 0x41, 0x51, 0xf9, 0x31, 0xcc, 0xc7, 0x1e, 0x5d, 0x27, 0xda, - 0x48, 0xf2, 0xc3, 0xec, 0x49, 0x41, 0xf3, 0x3e, 0xff, 0x57, 0x4c, 0x91, 0x5f, 0x5e, 0x48, 0x2b, - 0x4c, 0xe3, 0xa9, 0x65, 0x02, 0xe3, 0xa7, 0x9e, 0x48, 0x36, 0x00, 0x42, 0x81, 0x7e, 0xfc, 0x0f, - 0x58, 0x24, 0x76, 0x4d, 0x12, 0x78, 0xfd, 0x88, 0xee, 0x31, 0x81, 0x9d, 0x0b, 0x68, 0xb4, 0xc5, - 0x4d, 0x0c, 0x27, 0xa9, 0x8d, 0x75, 0x62, 0x38, 0x49, 0xef, 0x9b, 0xe5, 0x63, 0xb7, 0x6e, 0x7c, - 0xe7, 0xd5, 0x9e, 0xe1, 0xed, 0x0d, 0x77, 0x88, 0x38, 0xd7, 0x19, 0xf1, 0xcb, 0x86, 0xcd, 0xff, - 0xba, 0xee, 0x5b, 0xe5, 0x75, 0xca, 0xef, 0x3a, 0xe1, 0x37, 0xd8, 0xd9, 0x29, 0xd1, 0xd5, 0x8d, - 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0x3b, 0x4f, 0x3d, 0x31, 0x9a, 0x3c, 0x00, 0x00, + // 3579 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x3b, 0x4b, 0x6f, 0x1b, 0xd7, + 0xd5, 0x1e, 0xbe, 0x44, 0x1e, 0x3e, 0x44, 0x5f, 0x3b, 0x32, 0x4d, 0xdb, 0xb2, 0x3c, 0x8e, 0x1d, + 0xc5, 0x71, 0xec, 0x44, 0x4e, 0x10, 0xe3, 0xcb, 0x0b, 0xb1, 0x65, 0x2b, 0xc4, 0x27, 0xb9, 0xca, + 0x48, 0x89, 0x8b, 0xa6, 0xe8, 0x60, 0xc4, 0xb9, 0xa2, 0x26, 0xe2, 0xcc, 0xd0, 0x33, 0x43, 0xcb, + 0xca, 0x26, 0x46, 0x83, 0x16, 0x4d, 0x51, 0xf4, 0x81, 0x6e, 0xda, 0xa2, 0x8b, 0xa2, 0xab, 0x3e, + 0x36, 0x05, 0x82, 0x2e, 0xda, 0xa2, 0xfb, 0xa0, 0x5d, 0xf4, 0x27, 0x74, 0xd9, 0x5d, 0x7f, 0x43, + 0x71, 0x1f, 0x73, 0xe7, 0xc1, 0x19, 0x72, 0x44, 0xda, 0xf1, 0x4e, 0xf7, 0xcc, 0x39, 0xe7, 0x9e, + 0x7b, 0xee, 0x79, 0xf3, 0x0a, 0x9a, 0xba, 0xe6, 0x69, 0x6a, 0xd7, 0xb6, 0x1d, 0xfd, 0xda, 0xc0, + 0xb1, 0x3d, 0x1b, 0x1d, 0x37, 0x8d, 0xfe, 0xc3, 0xa1, 0xcb, 0x56, 0xd7, 0xc8, 0xe7, 0x76, 0xad, + 0x6b, 0x9b, 0xa6, 0x6d, 0x31, 0x50, 0xbb, 0x61, 0x58, 0x1e, 0x76, 0x2c, 0xad, 0xcf, 0xd7, 0xb5, + 0x30, 0x41, 0xbb, 0xe6, 0x76, 0xf7, 0xb0, 0xa9, 0xb1, 0x95, 0x3c, 0x07, 0xc5, 0x3b, 0xe6, 0xc0, + 0x3b, 0x94, 0x7f, 0x21, 0x41, 0xed, 0x6e, 0x7f, 0xe8, 0xee, 0x29, 0xf8, 0xc1, 0x10, 0xbb, 0x1e, + 0x7a, 0x05, 0x0a, 0x3b, 0x9a, 0x8b, 0x5b, 0xd2, 0x92, 0xb4, 0x5c, 0x5d, 0x39, 0x7b, 0x2d, 0xb2, + 0x2b, 0xdf, 0x6f, 0xc3, 0xed, 0xdd, 0xd2, 0x5c, 0xac, 0x50, 0x4c, 0x84, 0xa0, 0xa0, 0xef, 0x74, + 0x56, 0x5b, 0xb9, 0x25, 0x69, 0x39, 0xaf, 0xd0, 0xbf, 0xd1, 0x22, 0x80, 0x8b, 0x7b, 0x26, 0xb6, + 0xbc, 0xce, 0xaa, 0xdb, 0xca, 0x2f, 0xe5, 0x97, 0xf3, 0x4a, 0x08, 0x82, 0x64, 0xa8, 0x75, 0xed, + 0x7e, 0x1f, 0x77, 0x3d, 0xc3, 0xb6, 0x3a, 0xab, 0xad, 0x02, 0xa5, 0x8d, 0xc0, 0xe4, 0x5f, 0x4b, + 0x50, 0xe7, 0xa2, 0xb9, 0x03, 0xdb, 0x72, 0x31, 0xba, 0x01, 0x25, 0xd7, 0xd3, 0xbc, 0xa1, 0xcb, + 0xa5, 0x3b, 0x93, 0x28, 0xdd, 0x16, 0x45, 0x51, 0x38, 0x6a, 0xa2, 0x78, 0xf1, 0xed, 0xf3, 0xa3, + 0xdb, 0xc7, 0x8e, 0x50, 0x88, 0x1f, 0x41, 0xfe, 0x99, 0x04, 0xcd, 0x2d, 0x7f, 0xe9, 0x6b, 0xef, + 0x24, 0x14, 0xbb, 0xf6, 0xd0, 0xf2, 0xa8, 0x80, 0x75, 0x85, 0x2d, 0xd0, 0x05, 0xa8, 0x75, 0xf7, + 0x34, 0xcb, 0xc2, 0x7d, 0xd5, 0xd2, 0x4c, 0x4c, 0x45, 0xa9, 0x28, 0x55, 0x0e, 0xbb, 0xa7, 0x99, + 0x38, 0x93, 0x44, 0x4b, 0x50, 0x1d, 0x68, 0x8e, 0x67, 0x44, 0x74, 0x16, 0x06, 0xc9, 0xbf, 0x91, + 0x60, 0xe1, 0x3d, 0xd7, 0x35, 0x7a, 0xd6, 0x88, 0x64, 0x0b, 0x50, 0xb2, 0x6c, 0x1d, 0x77, 0x56, + 0xa9, 0x68, 0x79, 0x85, 0xaf, 0xd0, 0x19, 0xa8, 0x0c, 0x30, 0x76, 0x54, 0xc7, 0xee, 0xfb, 0x82, + 0x95, 0x09, 0x40, 0xb1, 0xfb, 0x18, 0x7d, 0x00, 0xc7, 0xdd, 0x18, 0x23, 0x76, 0x9b, 0xd5, 0x95, + 0x8b, 0xd7, 0x46, 0xec, 0xf1, 0x5a, 0x7c, 0x53, 0x65, 0x94, 0x5a, 0x7e, 0x9c, 0x83, 0x13, 0x02, + 0x8f, 0xc9, 0x4a, 0xfe, 0x26, 0x9a, 0x73, 0x71, 0x4f, 0x88, 0xc7, 0x16, 0x59, 0x34, 0x27, 0x54, + 0x9e, 0x0f, 0xab, 0x3c, 0x83, 0x81, 0xc5, 0xf5, 0x59, 0x1c, 0xd1, 0x27, 0x3a, 0x0f, 0x55, 0xfc, + 0x68, 0x60, 0x38, 0x58, 0xf5, 0x0c, 0x13, 0xb7, 0x4a, 0x4b, 0xd2, 0x72, 0x41, 0x01, 0x06, 0xda, + 0x36, 0xcc, 0xb0, 0x45, 0xce, 0x65, 0xb6, 0x48, 0xf9, 0xb7, 0x12, 0x9c, 0x1a, 0xb9, 0x25, 0x6e, + 0xe2, 0x0a, 0x34, 0xe9, 0xc9, 0x03, 0xcd, 0x10, 0x63, 0x27, 0x0a, 0xbf, 0x3c, 0x4e, 0xe1, 0x01, + 0xba, 0x32, 0x42, 0x1f, 0x12, 0x32, 0x97, 0x5d, 0xc8, 0x7d, 0x38, 0xb5, 0x86, 0x3d, 0xbe, 0x01, + 0xf9, 0x86, 0xdd, 0xe9, 0x43, 0x44, 0xd4, 0x97, 0x72, 0x23, 0xbe, 0xf4, 0xa7, 0x9c, 0xf0, 0x25, + 0xba, 0x55, 0xc7, 0xda, 0xb5, 0xd1, 0x59, 0xa8, 0x08, 0x14, 0x6e, 0x15, 0x01, 0x00, 0xbd, 0x01, + 0x45, 0x22, 0x29, 0x33, 0x89, 0xc6, 0xca, 0x85, 0xe4, 0x33, 0x85, 0x78, 0x2a, 0x0c, 0x1f, 0x75, + 0xa0, 0xe1, 0x7a, 0x9a, 0xe3, 0xa9, 0x03, 0xdb, 0xa5, 0xf7, 0x4c, 0x0d, 0xa7, 0xba, 0x22, 0x47, + 0x39, 0x88, 0x60, 0xba, 0xe1, 0xf6, 0x36, 0x39, 0xa6, 0x52, 0xa7, 0x94, 0xfe, 0x12, 0xdd, 0x81, + 0x1a, 0xb6, 0xf4, 0x80, 0x51, 0x21, 0x33, 0xa3, 0x2a, 0xb6, 0x74, 0xc1, 0x26, 0xb8, 0x9f, 0x62, + 0xf6, 0xfb, 0xf9, 0x91, 0x04, 0xad, 0xd1, 0x0b, 0x9a, 0x25, 0x50, 0xbe, 0xc9, 0x88, 0x30, 0xbb, + 0xa0, 0xb1, 0x1e, 0x2e, 0x2e, 0x49, 0xe1, 0x24, 0xb2, 0x01, 0xcf, 0x05, 0xd2, 0xd0, 0x2f, 0x4f, + 0xcd, 0x58, 0x3e, 0x97, 0x60, 0x21, 0xbe, 0xd7, 0x2c, 0xe7, 0x7e, 0x0d, 0x8a, 0x86, 0xb5, 0x6b, + 0xfb, 0xc7, 0x5e, 0x1c, 0xe3, 0x67, 0x64, 0x2f, 0x86, 0x2c, 0x9b, 0x70, 0x66, 0x0d, 0x7b, 0x1d, + 0xcb, 0xc5, 0x8e, 0x77, 0xcb, 0xb0, 0xfa, 0x76, 0x6f, 0x53, 0xf3, 0xf6, 0x66, 0xf0, 0x91, 0x88, + 0xb9, 0xe7, 0x62, 0xe6, 0x2e, 0xff, 0x4e, 0x82, 0xb3, 0xc9, 0xfb, 0xf1, 0xa3, 0xb7, 0xa1, 0xbc, + 0x6b, 0xe0, 0xbe, 0x4e, 0x74, 0x26, 0x51, 0x9d, 0x89, 0x35, 0xf1, 0x95, 0x01, 0x41, 0xe6, 0x27, + 0xbc, 0x90, 0x62, 0xa0, 0x5b, 0x9e, 0x63, 0x58, 0xbd, 0x75, 0xc3, 0xf5, 0x14, 0x86, 0x1f, 0xd2, + 0x67, 0x3e, 0xbb, 0x65, 0xfe, 0x50, 0x82, 0xc5, 0x35, 0xec, 0xdd, 0x16, 0xa1, 0x96, 0x7c, 0x37, + 0x5c, 0xcf, 0xe8, 0xba, 0x4f, 0xb6, 0xc8, 0xc8, 0x90, 0x33, 0xe5, 0x9f, 0x48, 0x70, 0x3e, 0x55, + 0x18, 0xae, 0x3a, 0x1e, 0x4a, 0xfc, 0x40, 0x9b, 0x1c, 0x4a, 0xfe, 0x1f, 0x1f, 0x7e, 0xa4, 0xf5, + 0x87, 0x78, 0x53, 0x33, 0x1c, 0x16, 0x4a, 0xa6, 0x0c, 0xac, 0x7f, 0x94, 0xe0, 0xdc, 0x1a, 0xf6, + 0x36, 0xfd, 0x34, 0xf3, 0x0c, 0xb5, 0x93, 0xa1, 0xa2, 0xf8, 0x31, 0xbb, 0xcc, 0x44, 0x69, 0x9f, + 0x89, 0xfa, 0x16, 0xa9, 0x1f, 0x84, 0x1c, 0xf2, 0x36, 0xab, 0x05, 0xb8, 0xf2, 0xe4, 0xef, 0x49, + 0x70, 0xfa, 0xbd, 0xee, 0x83, 0xa1, 0xe1, 0x60, 0x8e, 0xb4, 0x6e, 0x77, 0xf7, 0xa7, 0x57, 0x6d, + 0x50, 0x37, 0xe5, 0x22, 0x75, 0xd3, 0x84, 0x0a, 0x97, 0xca, 0xa1, 0xe0, 0x3e, 0xd6, 0xdc, 0x67, + 0x2b, 0xc7, 0xe3, 0x3c, 0xd4, 0x3e, 0xe2, 0xf5, 0x12, 0x4d, 0xab, 0x71, 0xbb, 0x90, 0x92, 0xed, + 0x22, 0x54, 0x62, 0x25, 0x55, 0x5d, 0x6b, 0x50, 0x77, 0x31, 0xde, 0x9f, 0x26, 0x89, 0xd6, 0x08, + 0xa1, 0x48, 0x7e, 0xeb, 0x70, 0x7c, 0x68, 0xed, 0x92, 0x32, 0x1f, 0xeb, 0x5c, 0x51, 0xac, 0xda, + 0x9e, 0x1c, 0x89, 0x47, 0x09, 0xd1, 0xfb, 0x30, 0x1f, 0xe7, 0x55, 0xcc, 0xc4, 0x2b, 0x4e, 0x86, + 0x3a, 0xd0, 0xd4, 0x1d, 0x7b, 0x30, 0xc0, 0xba, 0xea, 0xfa, 0xac, 0x4a, 0xd9, 0x58, 0x71, 0x3a, + 0x9f, 0x95, 0xfc, 0x85, 0x04, 0x0b, 0xf7, 0x35, 0xaf, 0xbb, 0xb7, 0x6a, 0x72, 0x63, 0x9d, 0xc1, + 0xd5, 0xdf, 0x86, 0xca, 0x43, 0x7e, 0x11, 0x7e, 0x3c, 0x3f, 0x9f, 0x20, 0x50, 0xf8, 0xca, 0x95, + 0x80, 0x42, 0xfe, 0x4a, 0x82, 0x93, 0xb4, 0xa9, 0xf2, 0xa5, 0xfb, 0xfa, 0x83, 0xce, 0x84, 0xc6, + 0x0a, 0x5d, 0x86, 0x86, 0xa9, 0x39, 0xfb, 0x5b, 0x01, 0x4e, 0x91, 0xe2, 0xc4, 0xa0, 0xf2, 0x23, + 0x00, 0xbe, 0xda, 0x70, 0x7b, 0x53, 0xc8, 0x7f, 0x13, 0xe6, 0xf8, 0xae, 0x3c, 0xfe, 0x4c, 0xba, + 0x58, 0x1f, 0x5d, 0xfe, 0x87, 0x04, 0x8d, 0x20, 0xa3, 0x50, 0xaf, 0x6a, 0x40, 0x4e, 0xf8, 0x52, + 0xae, 0xb3, 0x8a, 0xde, 0x86, 0x12, 0x6b, 0xb8, 0x39, 0xef, 0x4b, 0x51, 0xde, 0xbc, 0x19, 0x0f, + 0xa5, 0x25, 0x0a, 0x50, 0x38, 0x11, 0xd1, 0x91, 0x88, 0xc2, 0xc2, 0xab, 0x03, 0x08, 0xea, 0xc0, + 0x7c, 0xb4, 0x88, 0xf5, 0x7d, 0x66, 0x29, 0x2d, 0xfa, 0xae, 0x6a, 0x9e, 0x46, 0x83, 0x6f, 0x23, + 0x52, 0xc3, 0xba, 0xf2, 0x7f, 0x8b, 0x50, 0x0d, 0x9d, 0x72, 0xe4, 0x24, 0xf1, 0x2b, 0xcd, 0x4d, + 0xce, 0x23, 0xf9, 0xd1, 0x4e, 0xea, 0x12, 0x34, 0x0c, 0x5a, 0xbb, 0xa8, 0xdc, 0x14, 0x69, 0xb2, + 0xa9, 0x28, 0x75, 0x06, 0xe5, 0x7e, 0x81, 0x16, 0xa1, 0x6a, 0x0d, 0x4d, 0xd5, 0xde, 0x55, 0x1d, + 0xfb, 0xc0, 0xe5, 0x2d, 0x59, 0xc5, 0x1a, 0x9a, 0xdf, 0xd8, 0x55, 0xec, 0x03, 0x37, 0xa8, 0xfa, + 0x4b, 0x47, 0xac, 0xfa, 0x17, 0xa1, 0x6a, 0x6a, 0x8f, 0x08, 0x57, 0xd5, 0x1a, 0x9a, 0xb4, 0x5b, + 0xcb, 0x2b, 0x15, 0x53, 0x7b, 0xa4, 0xd8, 0x07, 0xf7, 0x86, 0x26, 0x5a, 0x86, 0x66, 0x5f, 0x73, + 0x3d, 0x35, 0xdc, 0xee, 0x95, 0x69, 0xbb, 0xd7, 0x20, 0xf0, 0x3b, 0x41, 0xcb, 0x37, 0xda, 0x3f, + 0x54, 0x66, 0xe8, 0x1f, 0x74, 0xb3, 0x1f, 0x30, 0x82, 0xec, 0xfd, 0x83, 0x6e, 0xf6, 0x05, 0x9b, + 0x9b, 0x30, 0xb7, 0x43, 0x2b, 0x42, 0xb7, 0x55, 0x4d, 0x8d, 0x50, 0x77, 0x49, 0x31, 0xc8, 0x0a, + 0x47, 0xc5, 0x47, 0x47, 0x6f, 0x41, 0x85, 0xa6, 0x62, 0x4a, 0x5b, 0xcb, 0x44, 0x1b, 0x10, 0x10, + 0x6a, 0x1d, 0xf7, 0x3d, 0x8d, 0x52, 0xd7, 0xb3, 0x51, 0x0b, 0x02, 0xf4, 0x0a, 0x9c, 0xe8, 0x3a, + 0x58, 0xf3, 0xb0, 0x7e, 0xeb, 0xf0, 0xb6, 0x6d, 0x0e, 0x34, 0x6a, 0x4c, 0xad, 0xc6, 0x92, 0xb4, + 0x5c, 0x56, 0x92, 0x3e, 0x91, 0xc0, 0xd0, 0x15, 0xab, 0xbb, 0x8e, 0x6d, 0xb6, 0xe6, 0x59, 0x60, + 0x88, 0x42, 0xd1, 0x39, 0x00, 0x3f, 0x74, 0x6b, 0x5e, 0xab, 0x49, 0x6f, 0xb1, 0xc2, 0x21, 0xef, + 0x79, 0xf2, 0x67, 0x70, 0x32, 0xb0, 0x90, 0xd0, 0x6d, 0x8c, 0x5e, 0xac, 0x34, 0xed, 0xc5, 0x8e, + 0xaf, 0xe5, 0xff, 0x55, 0x80, 0x85, 0x2d, 0xed, 0x21, 0x7e, 0xfa, 0x6d, 0x43, 0xa6, 0x78, 0xbc, + 0x0e, 0xc7, 0x69, 0xa7, 0xb0, 0x12, 0x92, 0x67, 0x4c, 0x06, 0x0e, 0x5f, 0xe7, 0x28, 0x21, 0x7a, + 0x97, 0x94, 0x0e, 0xb8, 0xbb, 0xbf, 0x69, 0x1b, 0x41, 0xf6, 0x3d, 0x97, 0xc0, 0xe7, 0xb6, 0xc0, + 0x52, 0xc2, 0x14, 0x68, 0x73, 0x34, 0xb4, 0xb1, 0xbc, 0xfb, 0xc2, 0xd8, 0x7e, 0x34, 0xd0, 0x7e, + 0x3c, 0xc2, 0xa1, 0x16, 0xcc, 0xf1, 0xec, 0x4e, 0xfd, 0xbe, 0xac, 0xf8, 0x4b, 0xb4, 0x09, 0x27, + 0xd8, 0x09, 0xb6, 0xb8, 0x51, 0xb3, 0xc3, 0x97, 0x33, 0x1d, 0x3e, 0x89, 0x34, 0xea, 0x13, 0x95, + 0xa3, 0xfa, 0x44, 0x0b, 0xe6, 0xb8, 0x9d, 0xd2, 0x58, 0x50, 0x56, 0xfc, 0x25, 0xb9, 0x66, 0xc3, + 0x1c, 0xd8, 0x8e, 0x67, 0x58, 0xbd, 0x56, 0x95, 0x7e, 0x0b, 0x00, 0xa4, 0xe5, 0x82, 0x40, 0x9f, + 0x13, 0x26, 0x27, 0xef, 0x40, 0x59, 0x58, 0x78, 0x2e, 0xb3, 0x85, 0x0b, 0x9a, 0x78, 0x8c, 0xce, + 0xc7, 0x62, 0xb4, 0xfc, 0x4f, 0x09, 0x6a, 0xab, 0xe4, 0x48, 0xeb, 0x76, 0x8f, 0x66, 0x94, 0x4b, + 0xd0, 0x70, 0x70, 0xd7, 0x76, 0x74, 0x15, 0x5b, 0x9e, 0x63, 0x60, 0xd6, 0x9d, 0x17, 0x94, 0x3a, + 0x83, 0xde, 0x61, 0x40, 0x82, 0x46, 0xc2, 0xae, 0xeb, 0x69, 0xe6, 0x40, 0xdd, 0x25, 0xee, 0x9d, + 0x63, 0x68, 0x02, 0x4a, 0xbd, 0xfb, 0x02, 0xd4, 0x02, 0x34, 0xcf, 0xa6, 0xfb, 0x17, 0x94, 0xaa, + 0x80, 0x6d, 0xdb, 0xe8, 0x79, 0x68, 0x50, 0x9d, 0xaa, 0x7d, 0xbb, 0xa7, 0x92, 0x4e, 0x96, 0x27, + 0x9b, 0x9a, 0xce, 0xc5, 0x22, 0x77, 0x15, 0xc5, 0x72, 0x8d, 0x4f, 0x31, 0x4f, 0x37, 0x02, 0x6b, + 0xcb, 0xf8, 0x14, 0x93, 0x5c, 0x5f, 0x27, 0xb9, 0xf3, 0x9e, 0xad, 0xe3, 0xed, 0x29, 0x2b, 0x8d, + 0x0c, 0x53, 0xcc, 0xb3, 0x50, 0x11, 0x27, 0xe0, 0x47, 0x0a, 0x00, 0xe8, 0x2e, 0x34, 0xfc, 0x22, + 0x54, 0x65, 0xbd, 0x56, 0x21, 0xb5, 0xf2, 0x0b, 0x65, 0x3f, 0x57, 0xa9, 0xfb, 0x64, 0x74, 0x29, + 0xdf, 0x85, 0x5a, 0xf8, 0x33, 0xd9, 0x75, 0x2b, 0x6e, 0x28, 0x02, 0x40, 0xac, 0xf1, 0xde, 0xd0, + 0x24, 0x77, 0xca, 0x03, 0x8b, 0xbf, 0x94, 0x3f, 0x97, 0xa0, 0xce, 0x53, 0xf6, 0x96, 0x98, 0xb2, + 0xd3, 0xa3, 0x49, 0xf4, 0x68, 0xf4, 0x6f, 0xf4, 0x7f, 0xd1, 0x11, 0xdd, 0xf3, 0x89, 0x41, 0x80, + 0x32, 0xa1, 0xd5, 0x71, 0x24, 0x5f, 0x67, 0xe9, 0xed, 0x1f, 0x13, 0x43, 0xe3, 0x57, 0x43, 0x0d, + 0xad, 0x05, 0x73, 0x9a, 0xae, 0x3b, 0xd8, 0x75, 0xb9, 0x1c, 0xfe, 0x92, 0x7c, 0x79, 0x88, 0x1d, + 0xd7, 0x37, 0xf9, 0xbc, 0xe2, 0x2f, 0xd1, 0x5b, 0x50, 0x16, 0xe5, 0x74, 0x3e, 0xa9, 0x84, 0x0a, + 0xcb, 0xc9, 0x7b, 0x51, 0x41, 0x21, 0xff, 0x39, 0x07, 0x0d, 0xae, 0xb0, 0x5b, 0x3c, 0xa7, 0x8e, + 0x77, 0xbe, 0x5b, 0x50, 0xdb, 0x0d, 0x7c, 0x7f, 0xdc, 0xcc, 0x29, 0x1c, 0x22, 0x22, 0x34, 0x93, + 0x1c, 0x30, 0x9a, 0xd5, 0x0b, 0x33, 0x65, 0xf5, 0xe2, 0x51, 0x23, 0xd8, 0x68, 0x9d, 0x57, 0x4a, + 0xa8, 0xf3, 0xe4, 0x6f, 0x43, 0x35, 0xc4, 0x80, 0x46, 0x68, 0x36, 0xac, 0xe2, 0x1a, 0xf3, 0x97, + 0xe8, 0x46, 0x50, 0xdb, 0x30, 0x55, 0x9d, 0x4e, 0x90, 0x25, 0x56, 0xd6, 0xc8, 0xbf, 0x97, 0xa0, + 0xc4, 0x39, 0x9f, 0x87, 0x2a, 0x0f, 0x3a, 0xb4, 0xee, 0x63, 0xdc, 0x81, 0x83, 0x48, 0xe1, 0xf7, + 0xe4, 0xa2, 0xce, 0x69, 0x28, 0xc7, 0xe2, 0xcd, 0x1c, 0x4f, 0x0b, 0xfe, 0xa7, 0x50, 0x90, 0x21, + 0x9f, 0x68, 0x7c, 0xf9, 0x4a, 0xa2, 0x83, 0x76, 0x05, 0x77, 0xed, 0x87, 0xd8, 0x39, 0x9c, 0x7d, + 0x9c, 0xf9, 0x66, 0xc8, 0xa0, 0x33, 0xf6, 0x87, 0x82, 0x00, 0xbd, 0x19, 0xa8, 0x3b, 0x9f, 0x34, + 0xcd, 0x09, 0x47, 0x18, 0x6e, 0x8e, 0x81, 0xda, 0x7f, 0xca, 0x06, 0xb3, 0xd1, 0xa3, 0x4c, 0x5b, + 0xd7, 0x3c, 0x91, 0xb6, 0x43, 0xfe, 0xb9, 0x04, 0xa7, 0xd7, 0xb0, 0x77, 0x37, 0xda, 0xdc, 0x3f, + 0x6b, 0xa9, 0x4c, 0x68, 0x27, 0x09, 0x35, 0xcb, 0xad, 0xb7, 0xa1, 0x2c, 0xc6, 0x14, 0x6c, 0x64, + 0x2e, 0xd6, 0xf2, 0xf7, 0x25, 0x68, 0xf1, 0x5d, 0xe8, 0x9e, 0xa4, 0xa4, 0xee, 0x63, 0x0f, 0xeb, + 0x5f, 0x77, 0xdf, 0xfc, 0x77, 0x09, 0x9a, 0xe1, 0x88, 0x4f, 0x83, 0xf6, 0xeb, 0x50, 0xa4, 0xe3, + 0x09, 0x2e, 0xc1, 0x44, 0x63, 0x65, 0xd8, 0x24, 0x64, 0xd0, 0x32, 0x6f, 0x5b, 0x24, 0x27, 0xbe, + 0x0c, 0xd2, 0x4e, 0xfe, 0xe8, 0x69, 0x87, 0xa7, 0x61, 0x7b, 0x48, 0xf8, 0xb2, 0x71, 0x68, 0x00, + 0x90, 0xbf, 0xcc, 0x41, 0x2b, 0xe8, 0x47, 0xbe, 0xf6, 0xb8, 0x9f, 0x52, 0xad, 0xe6, 0x9f, 0x50, + 0xb5, 0x5a, 0x98, 0x3d, 0xd6, 0x17, 0x93, 0x62, 0xfd, 0xdf, 0x72, 0xd0, 0x08, 0xb4, 0xb6, 0xd9, + 0xd7, 0x2c, 0xb4, 0x00, 0xa5, 0x41, 0x5f, 0x0b, 0xa6, 0x8f, 0x7c, 0x85, 0xb6, 0x44, 0x9d, 0x13, + 0xd5, 0xd3, 0x4b, 0x49, 0x77, 0x98, 0x72, 0x11, 0x4a, 0x8c, 0x05, 0x69, 0x07, 0x59, 0x43, 0x41, + 0x9b, 0x7a, 0x5e, 0x5b, 0x31, 0x63, 0x21, 0xfd, 0xfc, 0x55, 0x40, 0xfc, 0x86, 0x55, 0xc3, 0x52, + 0x5d, 0xdc, 0xb5, 0x2d, 0x9d, 0xdd, 0x7d, 0x51, 0x69, 0xf2, 0x2f, 0x1d, 0x6b, 0x8b, 0xc1, 0xd1, + 0xeb, 0x50, 0xf0, 0x0e, 0x07, 0x2c, 0x8a, 0x37, 0x12, 0xa3, 0x63, 0x20, 0xd7, 0xf6, 0xe1, 0x00, + 0x2b, 0x14, 0x1d, 0x2d, 0x02, 0x10, 0x56, 0x9e, 0xa3, 0x3d, 0xe4, 0x29, 0xb1, 0xa0, 0x84, 0x20, + 0xc4, 0x9a, 0x7d, 0x1d, 0xce, 0xb1, 0xd4, 0xc1, 0x97, 0xf2, 0x5f, 0x72, 0xd0, 0x0c, 0x58, 0x2a, + 0xd8, 0x1d, 0xf6, 0xbd, 0x54, 0xfd, 0x8d, 0x6f, 0x06, 0x27, 0xd5, 0x0d, 0xef, 0x42, 0x95, 0xdf, + 0xe7, 0x11, 0xec, 0x01, 0x18, 0xc9, 0xfa, 0x18, 0x03, 0x2d, 0x3e, 0x21, 0x03, 0x2d, 0x1d, 0xd1, + 0x40, 0xe5, 0x2d, 0x58, 0xf0, 0xe3, 0x5e, 0x80, 0xb0, 0x81, 0x3d, 0x6d, 0x4c, 0xc1, 0x71, 0x1e, + 0xaa, 0x2c, 0x9f, 0xb1, 0x44, 0xce, 0x4a, 0x75, 0xd8, 0x11, 0x1d, 0xae, 0xfc, 0x1d, 0x38, 0x49, + 0xe3, 0x46, 0x7c, 0x94, 0x9b, 0x65, 0xae, 0x2e, 0x8b, 0x46, 0x80, 0x14, 0xfd, 0xcc, 0xba, 0x2b, + 0x4a, 0x04, 0x26, 0xaf, 0xc3, 0x73, 0x31, 0xfe, 0x33, 0xe4, 0x05, 0x52, 0x0a, 0x2d, 0x6c, 0x45, + 0x7f, 0x26, 0x9e, 0x3e, 0xfb, 0x9d, 0x13, 0x93, 0x5b, 0xd5, 0xd0, 0xe3, 0xf6, 0xa5, 0xa3, 0x77, + 0xa0, 0x62, 0xe1, 0x03, 0x35, 0x1c, 0x7c, 0x33, 0x0c, 0xe8, 0xca, 0x16, 0x3e, 0xa0, 0x7f, 0xc9, + 0xf7, 0xe0, 0xd4, 0x88, 0xa8, 0xb3, 0x9c, 0xfd, 0xaf, 0x12, 0x9c, 0x5e, 0x75, 0xec, 0xc1, 0x47, + 0x86, 0xe3, 0x0d, 0xb5, 0x7e, 0xf4, 0x87, 0xa2, 0xa7, 0xd3, 0xc6, 0xbd, 0x1f, 0x4a, 0xc3, 0x2c, + 0x2e, 0x5f, 0x4d, 0x30, 0xd7, 0x51, 0xa1, 0xf8, 0xa1, 0x43, 0x49, 0xfb, 0x3f, 0xf9, 0x24, 0xe1, + 0x39, 0xde, 0x84, 0x64, 0x93, 0xa5, 0x4a, 0x49, 0x9c, 0xfa, 0xe4, 0xa7, 0x9d, 0xfa, 0xa4, 0x78, + 0x7e, 0xe1, 0x09, 0x79, 0xfe, 0x91, 0xdb, 0x90, 0xf7, 0x21, 0x3a, 0x91, 0xa3, 0x21, 0x77, 0xaa, + 0x51, 0xde, 0x2d, 0x80, 0x60, 0x3a, 0xc5, 0x5f, 0xf9, 0x64, 0x61, 0x13, 0xa2, 0x22, 0xb7, 0x25, + 0xa2, 0x2c, 0x9d, 0x2a, 0x47, 0xe6, 0x25, 0x1f, 0x40, 0x3b, 0xc9, 0x4a, 0x67, 0xb1, 0xfc, 0x2f, + 0x73, 0x00, 0x1d, 0x3a, 0x1d, 0xda, 0xd6, 0xdc, 0xfd, 0xe9, 0x2a, 0xca, 0x8b, 0x50, 0x0f, 0x0c, + 0x26, 0xf0, 0xf7, 0xb0, 0x15, 0xe9, 0xc4, 0x25, 0x44, 0x61, 0x4b, 0x70, 0x46, 0x8a, 0x5d, 0x9d, + 0xf2, 0x09, 0x79, 0x0d, 0x33, 0x8a, 0x58, 0xd0, 0x43, 0x67, 0xa0, 0xe2, 0xd8, 0x07, 0x2a, 0x71, + 0x33, 0x9d, 0xe6, 0xd6, 0xb2, 0x52, 0x76, 0xec, 0x03, 0xe2, 0x7c, 0x3a, 0x3a, 0x05, 0x73, 0x9e, + 0xe6, 0xee, 0x13, 0xfe, 0x25, 0x96, 0xee, 0xc8, 0xb2, 0xa3, 0xa3, 0x93, 0x50, 0xdc, 0x35, 0xfa, + 0xd8, 0x6d, 0xcd, 0x51, 0x96, 0x6c, 0x81, 0xde, 0xf0, 0xdf, 0x73, 0x94, 0x33, 0xff, 0x1e, 0xcd, + 0x9e, 0x74, 0x7c, 0x25, 0xc1, 0x7c, 0xa0, 0x35, 0x1a, 0x80, 0x48, 0x4c, 0xa3, 0xf1, 0xec, 0xb6, + 0xad, 0xb3, 0x50, 0xd1, 0x48, 0xf9, 0x89, 0x85, 0x11, 0xb2, 0xa8, 0x15, 0x90, 0x8c, 0xab, 0xcb, + 0xc9, 0xb9, 0xc8, 0xa1, 0x0d, 0xdd, 0xff, 0x85, 0xa7, 0xe4, 0xd8, 0x07, 0x1d, 0x5d, 0x68, 0x83, + 0x3d, 0x6b, 0x63, 0x55, 0x28, 0xd1, 0xc6, 0x6d, 0xfa, 0xb2, 0xed, 0x22, 0xd4, 0xb1, 0xe3, 0xd8, + 0x8e, 0x6a, 0x62, 0xd7, 0xd5, 0x7a, 0x98, 0x17, 0x5d, 0x35, 0x0a, 0xdc, 0x60, 0x30, 0xf9, 0xdf, + 0x79, 0x68, 0x04, 0x47, 0xf1, 0x7f, 0xd7, 0x31, 0x74, 0xff, 0x77, 0x1d, 0x43, 0x27, 0xc1, 0xdc, + 0x61, 0xa1, 0x30, 0x14, 0xcc, 0x39, 0xa4, 0xa3, 0x93, 0x3c, 0x48, 0x1c, 0xcc, 0xb2, 0x75, 0x1c, + 0x5c, 0x2c, 0xf8, 0x20, 0x7e, 0xaf, 0x11, 0xfb, 0x28, 0x64, 0xb0, 0x8f, 0x62, 0x06, 0xfb, 0x28, + 0x25, 0xd8, 0xc7, 0x02, 0x94, 0x76, 0x86, 0xdd, 0x7d, 0xec, 0xf1, 0xf2, 0x88, 0xaf, 0xa2, 0x76, + 0x53, 0x8e, 0xd9, 0x8d, 0x30, 0x8f, 0x4a, 0xd8, 0x3c, 0xce, 0x40, 0x85, 0xfd, 0xb8, 0xa0, 0x7a, + 0x2e, 0x9d, 0xb2, 0xe6, 0x95, 0x32, 0x03, 0x6c, 0xbb, 0xe8, 0xa6, 0xdf, 0x3b, 0x54, 0x93, 0x1c, + 0x9d, 0x46, 0x9c, 0x98, 0x85, 0xf8, 0x9d, 0xc3, 0x4d, 0x68, 0xed, 0xe1, 0xa1, 0x43, 0xdf, 0x46, + 0xa8, 0xf4, 0xf1, 0xee, 0x83, 0x21, 0x76, 0x0e, 0xb5, 0x9d, 0x3e, 0x6e, 0xd5, 0xa8, 0x60, 0x0b, + 0xe2, 0xfb, 0xaa, 0xe6, 0x69, 0x1f, 0xf8, 0x5f, 0xd1, 0x6b, 0xb0, 0x10, 0xa3, 0x34, 0x2c, 0x1d, + 0x3f, 0xc2, 0x7a, 0xab, 0x4e, 0xe9, 0x4e, 0x46, 0xe8, 0x3a, 0xec, 0x9b, 0xfc, 0x09, 0xa0, 0x40, + 0x92, 0xd9, 0x7a, 0xc7, 0xd8, 0x55, 0xe7, 0xe2, 0x57, 0x2d, 0xff, 0x41, 0x82, 0xe3, 0xe1, 0xcd, + 0xa6, 0x4d, 0xa0, 0xef, 0x40, 0x95, 0xcd, 0xac, 0x55, 0xe2, 0xc0, 0xbc, 0x7b, 0x3c, 0x37, 0x56, + 0xc7, 0x0a, 0x18, 0x41, 0x1c, 0xbb, 0x08, 0xf5, 0x03, 0xdb, 0xd9, 0x37, 0xac, 0x9e, 0x4a, 0x24, + 0xf3, 0xdd, 0xa6, 0xc6, 0x81, 0xf7, 0x08, 0x4c, 0xfe, 0x42, 0x82, 0xc5, 0x0f, 0x07, 0xba, 0xe6, + 0xe1, 0x50, 0x25, 0x31, 0xeb, 0x03, 0x9b, 0xd7, 0xfd, 0x37, 0x2e, 0xb9, 0x6c, 0x73, 0x57, 0x86, + 0x2d, 0x6f, 0xc0, 0x69, 0x05, 0xbb, 0xd8, 0xd2, 0x23, 0x1f, 0xa7, 0x95, 0x42, 0x1e, 0x40, 0x3b, + 0x89, 0xdd, 0x2c, 0x77, 0xcf, 0x4a, 0x3a, 0xd5, 0x21, 0x6c, 0x3d, 0x1e, 0xa1, 0x48, 0x25, 0x41, + 0xf7, 0xf1, 0xae, 0xfc, 0x52, 0x82, 0xe3, 0x23, 0xcd, 0x32, 0x6a, 0x00, 0x7c, 0x68, 0x75, 0xf9, + 0x14, 0xa1, 0x79, 0x0c, 0xd5, 0xa0, 0xec, 0xcf, 0x14, 0x9a, 0x12, 0xaa, 0xc2, 0xdc, 0xb6, 0x4d, + 0xb1, 0x9b, 0x39, 0xd4, 0x84, 0x1a, 0x23, 0x1c, 0x76, 0xbb, 0xd8, 0x75, 0x9b, 0x79, 0x01, 0xb9, + 0xab, 0x19, 0xfd, 0xa1, 0x83, 0x9b, 0x05, 0x54, 0x87, 0xca, 0xb6, 0xcd, 0xdf, 0xca, 0x34, 0x8b, + 0x08, 0x41, 0xc3, 0x7f, 0x38, 0xc3, 0x89, 0x4a, 0x21, 0x98, 0x4f, 0x36, 0x77, 0x65, 0x37, 0xdc, + 0x56, 0x92, 0x5e, 0x0b, 0x9d, 0x82, 0x13, 0x1f, 0x5a, 0x3a, 0xde, 0x35, 0x2c, 0xac, 0x07, 0x9f, + 0x9a, 0xc7, 0xd0, 0x09, 0x98, 0xef, 0x58, 0x16, 0x76, 0x42, 0x40, 0x89, 0x00, 0x37, 0xb0, 0xd3, + 0xc3, 0x21, 0x60, 0x0e, 0x1d, 0x87, 0xfa, 0x86, 0xf1, 0x28, 0x04, 0xca, 0xaf, 0xfc, 0xe0, 0x14, + 0x54, 0x88, 0xe7, 0xdd, 0xb6, 0x6d, 0x47, 0x47, 0x03, 0x40, 0xf4, 0x3d, 0x99, 0x39, 0xb0, 0x2d, + 0xf1, 0xf0, 0x12, 0xbd, 0x92, 0x52, 0x11, 0x8c, 0xa2, 0xf2, 0xdb, 0x6f, 0x5f, 0x4e, 0xa1, 0x88, + 0xa1, 0xcb, 0xc7, 0x90, 0x49, 0x77, 0x24, 0xbd, 0xea, 0xb6, 0xd1, 0xdd, 0xf7, 0x7f, 0x29, 0x1f, + 0xb3, 0x63, 0x0c, 0xd5, 0xdf, 0x31, 0xf6, 0x9e, 0x93, 0x2f, 0xd8, 0xa3, 0x3f, 0xdf, 0x88, 0xe4, + 0x63, 0xe8, 0x01, 0x9c, 0x5c, 0xc3, 0x21, 0xa7, 0xf1, 0x37, 0x5c, 0x49, 0xdf, 0x70, 0x04, 0xf9, + 0x88, 0x5b, 0xae, 0x43, 0x91, 0x0e, 0xa6, 0x50, 0x92, 0x5f, 0x85, 0xff, 0x3b, 0xa1, 0xbd, 0x94, + 0x8e, 0x20, 0xb8, 0x7d, 0x02, 0xf3, 0xb1, 0xd7, 0xd5, 0xe8, 0xc5, 0x04, 0xb2, 0xe4, 0x77, 0xf2, + 0xed, 0x2b, 0x59, 0x50, 0xc5, 0x5e, 0x3d, 0x68, 0x44, 0x5f, 0xa3, 0xa1, 0xe5, 0x04, 0xfa, 0xc4, + 0x97, 0xb1, 0xed, 0x17, 0x33, 0x60, 0x8a, 0x8d, 0x4c, 0x68, 0xc6, 0x5f, 0xfb, 0xa2, 0x2b, 0x63, + 0x19, 0x44, 0xcd, 0xed, 0xa5, 0x4c, 0xb8, 0x62, 0xbb, 0x43, 0x6a, 0x04, 0x23, 0xaf, 0x4d, 0xd1, + 0xb5, 0x64, 0x36, 0x69, 0xcf, 0x60, 0xdb, 0xd7, 0x33, 0xe3, 0x8b, 0xad, 0xbf, 0xcb, 0x06, 0xe2, + 0x49, 0x2f, 0x36, 0xd1, 0xab, 0xc9, 0xec, 0xc6, 0x3c, 0x35, 0x6d, 0xaf, 0x1c, 0x85, 0x44, 0x08, + 0xf1, 0x19, 0x9d, 0x64, 0x27, 0xbc, 0x7a, 0x8c, 0xfb, 0x9d, 0xcf, 0x2f, 0xfd, 0x39, 0x67, 0xfb, + 0xd5, 0x23, 0x50, 0x08, 0x01, 0xec, 0xf8, 0x7b, 0x6a, 0xdf, 0x0d, 0xaf, 0x4f, 0xb4, 0x9a, 0xe9, + 0x7c, 0xf0, 0x63, 0x98, 0x8f, 0xbd, 0x49, 0x48, 0xf4, 0x9a, 0xe4, 0x77, 0x0b, 0xed, 0x71, 0xb9, + 0x86, 0xb9, 0x64, 0xec, 0x87, 0x01, 0x94, 0x62, 0xfd, 0x09, 0x3f, 0x1e, 0xb4, 0xaf, 0x64, 0x41, + 0x15, 0x07, 0x71, 0x69, 0xb8, 0x8c, 0x0d, 0xd7, 0xd1, 0xd5, 0x64, 0x1e, 0xc9, 0x3f, 0x0c, 0xb4, + 0x5f, 0xce, 0x88, 0x2d, 0x36, 0x55, 0x01, 0xd6, 0xb0, 0xb7, 0x81, 0x3d, 0x87, 0xd8, 0xc8, 0xe5, + 0x44, 0x95, 0x07, 0x08, 0xfe, 0x36, 0x2f, 0x4c, 0xc4, 0x13, 0x1b, 0x7c, 0x13, 0x90, 0x9f, 0x62, + 0x43, 0x2f, 0x62, 0x2e, 0x8e, 0x9d, 0x3f, 0xb2, 0x61, 0xe1, 0xa4, 0xbb, 0x79, 0x00, 0xcd, 0x0d, + 0xcd, 0x22, 0x9d, 0x67, 0xc0, 0xf7, 0x6a, 0xa2, 0x60, 0x71, 0xb4, 0x14, 0x6d, 0xa5, 0x62, 0x8b, + 0xc3, 0x1c, 0x88, 0x1c, 0xaa, 0x09, 0x17, 0xc4, 0xf1, 0xd8, 0x12, 0x68, 0x23, 0x86, 0x98, 0x12, + 0x5b, 0xc6, 0xe0, 0x8b, 0x8d, 0x1f, 0x4b, 0xf4, 0xd5, 0x7e, 0x0c, 0xe1, 0xbe, 0xe1, 0xed, 0x6d, + 0xf6, 0x35, 0xcb, 0xcd, 0x22, 0x02, 0x45, 0x3c, 0x82, 0x08, 0x1c, 0x5f, 0x88, 0xa0, 0x43, 0x3d, + 0x32, 0xde, 0x43, 0x49, 0xcf, 0x5a, 0x92, 0x06, 0x8c, 0xed, 0xe5, 0xc9, 0x88, 0x62, 0x97, 0x3d, + 0xa8, 0xfb, 0xf6, 0xca, 0x94, 0xfb, 0x62, 0x9a, 0xa4, 0x01, 0x4e, 0x8a, 0xbb, 0x25, 0xa3, 0x86, + 0xdd, 0x6d, 0x74, 0x7a, 0x81, 0xb2, 0x4d, 0xbd, 0xc6, 0xb9, 0x5b, 0xfa, 0x48, 0x84, 0xc5, 0x93, + 0xd8, 0xa4, 0x30, 0x39, 0x58, 0x25, 0x0e, 0x3e, 0x13, 0xe3, 0x49, 0xca, 0xe0, 0x51, 0x3e, 0x86, + 0xee, 0x43, 0x89, 0xb5, 0x23, 0xe8, 0xf9, 0xf1, 0x9d, 0x0a, 0xe7, 0x7e, 0x69, 0x02, 0x96, 0x60, + 0xbc, 0x0f, 0xa7, 0x52, 0xfa, 0x94, 0xc4, 0x3c, 0x37, 0xbe, 0xa7, 0x99, 0xe4, 0xe5, 0x1a, 0xa0, + 0xd1, 0x57, 0xf1, 0x89, 0xd7, 0x94, 0xfa, 0x78, 0x3e, 0xc3, 0x16, 0xa3, 0x0f, 0xde, 0x13, 0xb7, + 0x48, 0x7d, 0x17, 0x3f, 0x61, 0x8b, 0x95, 0x5f, 0x95, 0xa0, 0xec, 0xbf, 0xf8, 0x78, 0x06, 0x95, + 0xf8, 0x33, 0x28, 0x8d, 0x3f, 0x86, 0xf9, 0xd8, 0xd3, 0xf1, 0x44, 0x4b, 0x4f, 0x7e, 0x5e, 0x3e, + 0xe9, 0xc6, 0xee, 0xf3, 0x7f, 0xb0, 0x15, 0x59, 0xf2, 0x85, 0xb4, 0xf2, 0x3a, 0x9e, 0x20, 0x27, + 0x30, 0x7e, 0xea, 0xe9, 0xf0, 0x1e, 0x40, 0x28, 0x5d, 0x8d, 0xff, 0x19, 0x8e, 0x44, 0xe0, 0x49, + 0x02, 0x6f, 0x1c, 0xd1, 0xc9, 0x27, 0xb0, 0x73, 0x89, 0x2b, 0xc4, 0x1b, 0xf5, 0x14, 0x57, 0x48, + 0x19, 0x0f, 0x24, 0x06, 0xc5, 0xf4, 0xee, 0x5f, 0x3e, 0x76, 0xeb, 0xc6, 0xb7, 0x5e, 0xed, 0x19, + 0xde, 0xde, 0x70, 0x87, 0x88, 0x73, 0x9d, 0x11, 0xbf, 0x6c, 0xd8, 0xfc, 0xaf, 0xeb, 0xbe, 0x55, + 0x5e, 0xa7, 0xfc, 0xae, 0x13, 0x7e, 0x83, 0x9d, 0x9d, 0x12, 0x5d, 0xdd, 0xf8, 0x5f, 0x00, 0x00, + 0x00, 0xff, 0xff, 0x6e, 0xe3, 0xdf, 0x9f, 0x70, 0x3e, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -4138,6 +4254,8 @@ type DataCoordClient interface { // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load Import(ctx context.Context, in *ImportTaskRequest, opts ...grpc.CallOption) (*ImportTaskResponse, error) UpdateSegmentStatistics(ctx context.Context, in *UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + AcquireSegmentLock(ctx context.Context, in *AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) + ReleaseSegmentLock(ctx context.Context, in *ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) } type dataCoordClient struct { @@ -4373,6 +4491,24 @@ func (c *dataCoordClient) UpdateSegmentStatistics(ctx context.Context, in *Updat return out, nil } +func (c *dataCoordClient) AcquireSegmentLock(ctx context.Context, in *AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/AcquireSegmentLock", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *dataCoordClient) ReleaseSegmentLock(ctx context.Context, in *ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, "/milvus.proto.data.DataCoord/ReleaseSegmentLock", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // DataCoordServer is the server API for DataCoord service. type DataCoordServer interface { GetComponentStates(context.Context, *internalpb.GetComponentStatesRequest) (*internalpb.ComponentStates, error) @@ -4402,6 +4538,8 @@ type DataCoordServer interface { // https://wiki.lfaidata.foundation/display/MIL/MEP+24+--+Support+bulk+load Import(context.Context, *ImportTaskRequest) (*ImportTaskResponse, error) UpdateSegmentStatistics(context.Context, *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) + AcquireSegmentLock(context.Context, *AcquireSegmentLockRequest) (*commonpb.Status, error) + ReleaseSegmentLock(context.Context, *ReleaseSegmentLockRequest) (*commonpb.Status, error) } // UnimplementedDataCoordServer can be embedded to have forward compatible implementations. @@ -4483,6 +4621,12 @@ func (*UnimplementedDataCoordServer) Import(ctx context.Context, req *ImportTask func (*UnimplementedDataCoordServer) UpdateSegmentStatistics(ctx context.Context, req *UpdateSegmentStatisticsRequest) (*commonpb.Status, error) { return nil, status.Errorf(codes.Unimplemented, "method UpdateSegmentStatistics not implemented") } +func (*UnimplementedDataCoordServer) AcquireSegmentLock(ctx context.Context, req *AcquireSegmentLockRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method AcquireSegmentLock not implemented") +} +func (*UnimplementedDataCoordServer) ReleaseSegmentLock(ctx context.Context, req *ReleaseSegmentLockRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method ReleaseSegmentLock not implemented") +} func RegisterDataCoordServer(s *grpc.Server, srv DataCoordServer) { s.RegisterService(&_DataCoord_serviceDesc, srv) @@ -4938,6 +5082,42 @@ func _DataCoord_UpdateSegmentStatistics_Handler(srv interface{}, ctx context.Con return interceptor(ctx, in, info, handler) } +func _DataCoord_AcquireSegmentLock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(AcquireSegmentLockRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).AcquireSegmentLock(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataCoord/AcquireSegmentLock", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).AcquireSegmentLock(ctx, req.(*AcquireSegmentLockRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _DataCoord_ReleaseSegmentLock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ReleaseSegmentLockRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DataCoordServer).ReleaseSegmentLock(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/milvus.proto.data.DataCoord/ReleaseSegmentLock", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DataCoordServer).ReleaseSegmentLock(ctx, req.(*ReleaseSegmentLockRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _DataCoord_serviceDesc = grpc.ServiceDesc{ ServiceName: "milvus.proto.data.DataCoord", HandlerType: (*DataCoordServer)(nil), @@ -5042,6 +5222,14 @@ var _DataCoord_serviceDesc = grpc.ServiceDesc{ MethodName: "UpdateSegmentStatistics", Handler: _DataCoord_UpdateSegmentStatistics_Handler, }, + { + MethodName: "AcquireSegmentLock", + Handler: _DataCoord_AcquireSegmentLock_Handler, + }, + { + MethodName: "ReleaseSegmentLock", + Handler: _DataCoord_ReleaseSegmentLock_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "data_coord.proto", diff --git a/internal/proto/index_coord.proto b/internal/proto/index_coord.proto index b5e3cef567..d17512670e 100644 --- a/internal/proto/index_coord.proto +++ b/internal/proto/index_coord.proto @@ -80,6 +80,7 @@ message BuildIndexRequest { repeated common.KeyValuePair index_params = 7; int64 num_rows = 8; schema.FieldSchema field_schema = 9; + int64 segmentID = 10; } message BuildIndexResponse { diff --git a/internal/proto/indexpb/index_coord.pb.go b/internal/proto/indexpb/index_coord.pb.go index c0d9cf1d83..1672a1069f 100644 --- a/internal/proto/indexpb/index_coord.pb.go +++ b/internal/proto/indexpb/index_coord.pb.go @@ -391,6 +391,7 @@ type BuildIndexRequest struct { IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,7,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` NumRows int64 `protobuf:"varint,8,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` FieldSchema *schemapb.FieldSchema `protobuf:"bytes,9,opt,name=field_schema,json=fieldSchema,proto3" json:"field_schema,omitempty"` + SegmentID int64 `protobuf:"varint,10,opt,name=segmentID,proto3" json:"segmentID,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -477,6 +478,13 @@ func (m *BuildIndexRequest) GetFieldSchema() *schemapb.FieldSchema { return nil } +func (m *BuildIndexRequest) GetSegmentID() int64 { + if m != nil { + return m.SegmentID + } + return 0 +} + type BuildIndexResponse struct { Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` IndexBuildID int64 `protobuf:"varint,2,opt,name=indexBuildID,proto3" json:"indexBuildID,omitempty"` @@ -842,75 +850,75 @@ func init() { func init() { proto.RegisterFile("index_coord.proto", fileDescriptor_f9e019eb3fda53c2) } var fileDescriptor_f9e019eb3fda53c2 = []byte{ - // 1073 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x56, 0xcd, 0x6e, 0xdb, 0x46, - 0x10, 0x36, 0x4d, 0x5b, 0x3f, 0x23, 0xc5, 0x8d, 0xb7, 0x69, 0xc0, 0x28, 0x0d, 0x22, 0xb3, 0xf9, - 0x51, 0x8b, 0x44, 0x0e, 0x94, 0xa6, 0x3d, 0x15, 0x68, 0x6d, 0x21, 0x86, 0x51, 0x38, 0x30, 0xd6, - 0x46, 0x0f, 0x05, 0x0a, 0x61, 0x2d, 0x8e, 0xec, 0x45, 0xf8, 0x23, 0x73, 0x57, 0x49, 0xed, 0x63, - 0xd1, 0x7b, 0x6f, 0xe9, 0xa3, 0xf4, 0xd8, 0x67, 0xc8, 0xe3, 0xf4, 0x56, 0xec, 0x72, 0x29, 0x91, - 0x12, 0xe5, 0xc8, 0x75, 0xd3, 0x5e, 0x72, 0xe3, 0xcc, 0x7e, 0xb3, 0xb3, 0xf3, 0xed, 0xec, 0xc7, - 0x81, 0x75, 0x1e, 0x7a, 0xf8, 0x73, 0xaf, 0x1f, 0x45, 0xb1, 0xd7, 0x1e, 0xc6, 0x91, 0x8c, 0x08, - 0x09, 0xb8, 0xff, 0x6a, 0x24, 0x12, 0xab, 0xad, 0xd7, 0x1b, 0xf5, 0x7e, 0x14, 0x04, 0x51, 0x98, - 0xf8, 0x1a, 0x6b, 0x3c, 0x94, 0x18, 0x87, 0xcc, 0x37, 0x76, 0x3d, 0x1b, 0xd1, 0xa8, 0x8b, 0xfe, - 0x09, 0x06, 0x2c, 0xb1, 0xdc, 0xdf, 0x2d, 0xf8, 0x98, 0xe2, 0x31, 0x17, 0x12, 0xe3, 0x17, 0x91, - 0x87, 0x14, 0x4f, 0x47, 0x28, 0x24, 0x79, 0x02, 0x2b, 0x47, 0x4c, 0xa0, 0x63, 0x35, 0xad, 0x56, - 0xad, 0xf3, 0x69, 0x3b, 0x97, 0xd4, 0x64, 0xdb, 0x13, 0xc7, 0x5b, 0x4c, 0x20, 0xd5, 0x48, 0xf2, - 0x15, 0x94, 0x99, 0xe7, 0xc5, 0x28, 0x84, 0xb3, 0x7c, 0x41, 0xd0, 0x77, 0x09, 0x86, 0xa6, 0x60, - 0x72, 0x13, 0x4a, 0x61, 0xe4, 0xe1, 0x6e, 0xd7, 0xb1, 0x9b, 0x56, 0xcb, 0xa6, 0xc6, 0x72, 0x7f, - 0xb3, 0xe0, 0x46, 0xfe, 0x64, 0x62, 0x18, 0x85, 0x02, 0xc9, 0x53, 0x28, 0x09, 0xc9, 0xe4, 0x48, - 0x98, 0xc3, 0xdd, 0x2e, 0xcc, 0x73, 0xa0, 0x21, 0xd4, 0x40, 0xc9, 0x16, 0xd4, 0x78, 0xc8, 0x65, - 0x6f, 0xc8, 0x62, 0x16, 0xa4, 0x27, 0xdc, 0x68, 0x4f, 0x71, 0x69, 0x68, 0xdb, 0x0d, 0xb9, 0xdc, - 0xd7, 0x40, 0x0a, 0x7c, 0xfc, 0xed, 0x7e, 0x03, 0x9f, 0xec, 0xa0, 0xdc, 0x55, 0x8c, 0xab, 0xdd, - 0x51, 0xa4, 0x64, 0xdd, 0x83, 0x6b, 0xfa, 0x1e, 0xb6, 0x46, 0xdc, 0xf7, 0x76, 0xbb, 0xea, 0x60, - 0x76, 0xcb, 0xa6, 0x79, 0xa7, 0xfb, 0x87, 0x05, 0x55, 0x1d, 0xbc, 0x1b, 0x0e, 0x22, 0xf2, 0x0c, - 0x56, 0xd5, 0xd1, 0x12, 0x86, 0xd7, 0x3a, 0x77, 0x0b, 0x8b, 0x98, 0xe4, 0xa2, 0x09, 0x9a, 0xb8, - 0x50, 0xcf, 0xee, 0xaa, 0x0b, 0xb1, 0x69, 0xce, 0x47, 0x1c, 0x28, 0x6b, 0x7b, 0x4c, 0x69, 0x6a, - 0x92, 0x3b, 0x00, 0x49, 0x43, 0x85, 0x2c, 0x40, 0x67, 0xa5, 0x69, 0xb5, 0xaa, 0xb4, 0xaa, 0x3d, - 0x2f, 0x58, 0x80, 0xea, 0x2a, 0x62, 0x64, 0x22, 0x0a, 0x9d, 0x55, 0xbd, 0x64, 0x2c, 0xf7, 0x57, - 0x0b, 0x6e, 0x4e, 0x57, 0x7e, 0x95, 0xcb, 0x78, 0x96, 0x04, 0xa1, 0xba, 0x07, 0xbb, 0x55, 0xeb, - 0xdc, 0x69, 0xcf, 0xf6, 0x74, 0x7b, 0x4c, 0x15, 0x35, 0x60, 0xf7, 0xed, 0x32, 0x90, 0xed, 0x18, - 0x99, 0x44, 0xbd, 0x96, 0xb2, 0x3f, 0x4d, 0x89, 0x55, 0x40, 0x49, 0xbe, 0xf0, 0xe5, 0xe9, 0xc2, - 0xe7, 0x33, 0xe6, 0x40, 0xf9, 0x15, 0xc6, 0x82, 0x47, 0xa1, 0xa6, 0xcb, 0xa6, 0xa9, 0x49, 0x6e, - 0x43, 0x35, 0x40, 0xc9, 0x7a, 0x43, 0x26, 0x4f, 0x0c, 0x5f, 0x15, 0xe5, 0xd8, 0x67, 0xf2, 0x44, - 0xe5, 0xf3, 0x98, 0x59, 0x14, 0x4e, 0xa9, 0x69, 0xab, 0x7c, 0xca, 0xa3, 0x56, 0x75, 0x37, 0xca, - 0xb3, 0x21, 0xa6, 0xdd, 0x58, 0xd6, 0x2c, 0x6c, 0x14, 0x52, 0xf7, 0x3d, 0x9e, 0xfd, 0xc0, 0xfc, - 0x11, 0xee, 0x33, 0x1e, 0x53, 0x50, 0x51, 0x49, 0x37, 0x92, 0xae, 0x29, 0x3b, 0xdd, 0xa4, 0xb2, - 0xe8, 0x26, 0x35, 0x1d, 0x66, 0x7a, 0xfa, 0xaf, 0x65, 0x58, 0x4f, 0x48, 0xfa, 0xcf, 0x28, 0xcd, - 0x73, 0xb3, 0xfa, 0x0e, 0x6e, 0x4a, 0xff, 0x06, 0x37, 0xe5, 0x7f, 0xc2, 0x0d, 0xb9, 0x05, 0x95, - 0x70, 0x14, 0xf4, 0xe2, 0xe8, 0xb5, 0x62, 0x57, 0xd7, 0x10, 0x8e, 0x02, 0x1a, 0xbd, 0x16, 0x64, - 0x1b, 0xea, 0x03, 0x8e, 0xbe, 0xd7, 0x4b, 0xc4, 0xd4, 0xa9, 0xea, 0xe6, 0x6f, 0xe6, 0x13, 0x18, - 0xa1, 0x7d, 0xae, 0x80, 0x07, 0xfa, 0x9b, 0xd6, 0x06, 0x13, 0xc3, 0x0d, 0x80, 0x64, 0xa9, 0xbf, - 0xca, 0x8b, 0x5a, 0x40, 0x16, 0xdc, 0x6f, 0xc1, 0x49, 0x1f, 0xf1, 0x73, 0xee, 0xa3, 0x66, 0xfb, - 0x72, 0x0a, 0xf6, 0xa7, 0x05, 0xeb, 0xb9, 0x78, 0xad, 0x64, 0xef, 0xeb, 0xc0, 0xa4, 0x05, 0xd7, - 0x93, 0x5b, 0x1c, 0x70, 0x1f, 0x4d, 0xbb, 0xd8, 0xba, 0x5d, 0xd6, 0x78, 0xae, 0x0a, 0xf2, 0x10, - 0x3e, 0x12, 0x18, 0x73, 0xe6, 0xf3, 0x73, 0xf4, 0x7a, 0x82, 0x9f, 0x27, 0xe2, 0xb6, 0x42, 0xd7, - 0x26, 0xee, 0x03, 0x7e, 0x8e, 0xee, 0x1b, 0x0b, 0x6e, 0x15, 0x90, 0x70, 0x15, 0xea, 0xbb, 0x00, - 0x99, 0xf3, 0x25, 0x82, 0x76, 0x7f, 0xae, 0xa0, 0x65, 0x99, 0xa3, 0xd5, 0x41, 0x7a, 0x04, 0xf7, - 0x17, 0xdb, 0xfc, 0x1c, 0xf6, 0x50, 0xb2, 0x85, 0xde, 0xdf, 0xf8, 0x07, 0xb2, 0x7c, 0xa9, 0x1f, - 0xc8, 0x5d, 0xa8, 0x0d, 0x18, 0xf7, 0x7b, 0x46, 0xe8, 0x6d, 0xfd, 0x6e, 0x41, 0xb9, 0xa8, 0xf6, - 0x90, 0xaf, 0xc1, 0x8e, 0xf1, 0x54, 0xf3, 0x37, 0xa7, 0x90, 0x19, 0xbd, 0xa0, 0x2a, 0xa2, 0xf0, - 0xba, 0x56, 0x0b, 0xaf, 0x6b, 0x03, 0xea, 0x01, 0x8b, 0x5f, 0xf6, 0x3c, 0xf4, 0x51, 0xa2, 0xe7, - 0x94, 0x9a, 0x56, 0xab, 0x42, 0x6b, 0xca, 0xd7, 0x4d, 0x5c, 0x99, 0xa9, 0xa0, 0x9c, 0x9d, 0x0a, - 0xb2, 0x7a, 0x5c, 0xc9, 0xeb, 0x71, 0x03, 0x2a, 0x31, 0xf6, 0xcf, 0xfa, 0x3e, 0x7a, 0xfa, 0x39, - 0x56, 0xe8, 0xd8, 0x26, 0xf7, 0x61, 0xd2, 0x08, 0x49, 0x7b, 0x80, 0x6e, 0x8f, 0x6b, 0x63, 0xaf, - 0xee, 0x8e, 0x47, 0x70, 0xbd, 0x1b, 0x47, 0xc3, 0x9c, 0x14, 0x66, 0x74, 0xcc, 0xca, 0xe9, 0x58, - 0xe7, 0x6d, 0x09, 0x40, 0x43, 0xb7, 0xd5, 0x74, 0x46, 0x86, 0x40, 0x76, 0x50, 0x6e, 0x47, 0xc1, - 0x30, 0x0a, 0x31, 0x94, 0xc9, 0x7f, 0x92, 0x3c, 0x99, 0x33, 0x62, 0xcc, 0x42, 0x4d, 0xc2, 0xc6, - 0x83, 0x39, 0x11, 0x53, 0x70, 0x77, 0x89, 0x04, 0x3a, 0xe3, 0x21, 0x0f, 0xf0, 0x90, 0xf7, 0x5f, - 0x6e, 0x9f, 0xb0, 0x30, 0x44, 0xff, 0xa2, 0x8c, 0x53, 0xd0, 0x34, 0xe3, 0x67, 0xf9, 0x08, 0x63, - 0x1c, 0xc8, 0x98, 0x87, 0xc7, 0xe9, 0xdb, 0x70, 0x97, 0xc8, 0x29, 0xdc, 0xd8, 0x41, 0x9d, 0x9d, - 0x0b, 0xc9, 0xfb, 0x22, 0x4d, 0xd8, 0x99, 0x9f, 0x70, 0x06, 0x7c, 0xc9, 0x94, 0x3f, 0x01, 0x4c, - 0x9a, 0x8d, 0x2c, 0xd6, 0x8c, 0xb3, 0x04, 0x4e, 0xc3, 0xc6, 0xdb, 0x73, 0x58, 0xcb, 0x8f, 0x35, - 0xe4, 0xf3, 0xa2, 0xd8, 0xc2, 0xa1, 0xaf, 0xf1, 0xc5, 0x22, 0xd0, 0x71, 0xaa, 0x18, 0xd6, 0x67, - 0x74, 0x87, 0x3c, 0xba, 0x68, 0x8b, 0x69, 0x8d, 0x6e, 0x3c, 0x5e, 0x10, 0x3d, 0xce, 0xb9, 0x0f, - 0xd5, 0x71, 0x3b, 0x93, 0x7b, 0x45, 0xd1, 0xd3, 0xdd, 0xde, 0xb8, 0x48, 0xf1, 0xdc, 0x25, 0xd2, - 0x03, 0xd8, 0x41, 0xb9, 0x87, 0x32, 0xe6, 0x7d, 0x41, 0x1e, 0x14, 0x5e, 0xe2, 0x04, 0x90, 0x6e, - 0xfa, 0xf0, 0x9d, 0xb8, 0xf4, 0xc8, 0x9d, 0x37, 0x2b, 0x46, 0x06, 0xd5, 0xc4, 0xff, 0xe1, 0x49, - 0xbd, 0x87, 0x27, 0x75, 0x08, 0xb5, 0xcc, 0x0c, 0x4d, 0x0a, 0x1f, 0xcb, 0xec, 0x90, 0xfd, 0x7f, - 0x37, 0xc6, 0xd6, 0x97, 0x3f, 0x76, 0x8e, 0xb9, 0x3c, 0x19, 0x1d, 0xa9, 0xd4, 0x9b, 0x09, 0xf2, - 0x31, 0x8f, 0xcc, 0xd7, 0x66, 0xca, 0xd0, 0xa6, 0xde, 0x69, 0x53, 0x97, 0x31, 0x3c, 0x3a, 0x2a, - 0x69, 0xf3, 0xe9, 0xdf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xd8, 0x29, 0x0d, 0x3e, 0x47, 0x0f, 0x00, - 0x00, + // 1087 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x57, 0xcd, 0x6e, 0x1b, 0x37, + 0x10, 0xf6, 0x7a, 0x6d, 0xfd, 0x8c, 0x14, 0x37, 0x66, 0xd3, 0x60, 0xa3, 0x24, 0x88, 0xbc, 0xcd, + 0x8f, 0x5a, 0x24, 0x72, 0xa0, 0x34, 0xed, 0xa9, 0x40, 0x6b, 0x0b, 0x31, 0x84, 0xc2, 0x81, 0x41, + 0x1b, 0x3d, 0x14, 0x28, 0x04, 0x5a, 0x3b, 0xb2, 0x88, 0xec, 0x8f, 0xbc, 0xa4, 0x92, 0xda, 0xc7, + 0xa2, 0xf7, 0xde, 0x92, 0x47, 0xe9, 0xb1, 0xcf, 0x90, 0x37, 0x2a, 0xc8, 0xe5, 0x4a, 0x5a, 0x69, + 0xe5, 0xc8, 0x75, 0xd3, 0x5e, 0x7a, 0xd3, 0xcc, 0x7e, 0xc3, 0x8f, 0xf3, 0xcd, 0x70, 0x48, 0xc1, + 0x26, 0x0f, 0x3d, 0xfc, 0xa5, 0xdb, 0x8b, 0xa2, 0xd8, 0x6b, 0x0e, 0xe3, 0x48, 0x46, 0x84, 0x04, + 0xdc, 0x7f, 0x3d, 0x12, 0x89, 0xd5, 0xd4, 0xdf, 0x6b, 0xd5, 0x5e, 0x14, 0x04, 0x51, 0x98, 0xf8, + 0x6a, 0x1b, 0x3c, 0x94, 0x18, 0x87, 0xcc, 0x37, 0x76, 0x75, 0x3a, 0xa2, 0x56, 0x15, 0xbd, 0x01, + 0x06, 0x2c, 0xb1, 0xdc, 0x77, 0x16, 0x7c, 0x4a, 0xf1, 0x84, 0x0b, 0x89, 0xf1, 0xcb, 0xc8, 0x43, + 0x8a, 0xa7, 0x23, 0x14, 0x92, 0x3c, 0x85, 0xb5, 0x63, 0x26, 0xd0, 0xb1, 0xea, 0x56, 0xa3, 0xd2, + 0xba, 0xd3, 0xcc, 0x90, 0x1a, 0xb6, 0x7d, 0x71, 0xb2, 0xc3, 0x04, 0x52, 0x8d, 0x24, 0x5f, 0x43, + 0x91, 0x79, 0x5e, 0x8c, 0x42, 0x38, 0xab, 0x17, 0x04, 0x7d, 0x9f, 0x60, 0x68, 0x0a, 0x26, 0x37, + 0xa1, 0x10, 0x46, 0x1e, 0x76, 0xda, 0x8e, 0x5d, 0xb7, 0x1a, 0x36, 0x35, 0x96, 0xfb, 0xbb, 0x05, + 0x37, 0xb2, 0x3b, 0x13, 0xc3, 0x28, 0x14, 0x48, 0x9e, 0x41, 0x41, 0x48, 0x26, 0x47, 0xc2, 0x6c, + 0xee, 0x76, 0x2e, 0xcf, 0xa1, 0x86, 0x50, 0x03, 0x25, 0x3b, 0x50, 0xe1, 0x21, 0x97, 0xdd, 0x21, + 0x8b, 0x59, 0x90, 0xee, 0x70, 0xab, 0x39, 0xa3, 0xa5, 0x91, 0xad, 0x13, 0x72, 0x79, 0xa0, 0x81, + 0x14, 0xf8, 0xf8, 0xb7, 0xfb, 0x2d, 0x7c, 0xb6, 0x87, 0xb2, 0xa3, 0x14, 0x57, 0xab, 0xa3, 0x48, + 0xc5, 0xba, 0x0f, 0xd7, 0x74, 0x1d, 0x76, 0x46, 0xdc, 0xf7, 0x3a, 0x6d, 0xb5, 0x31, 0xbb, 0x61, + 0xd3, 0xac, 0xd3, 0xfd, 0xc3, 0x82, 0xb2, 0x0e, 0xee, 0x84, 0xfd, 0x88, 0x3c, 0x87, 0x75, 0xb5, + 0xb5, 0x44, 0xe1, 0x8d, 0xd6, 0xbd, 0xdc, 0x24, 0x26, 0x5c, 0x34, 0x41, 0x13, 0x17, 0xaa, 0xd3, + 0xab, 0xea, 0x44, 0x6c, 0x9a, 0xf1, 0x11, 0x07, 0x8a, 0xda, 0x1e, 0x4b, 0x9a, 0x9a, 0xe4, 0x2e, + 0x40, 0xd2, 0x50, 0x21, 0x0b, 0xd0, 0x59, 0xab, 0x5b, 0x8d, 0x32, 0x2d, 0x6b, 0xcf, 0x4b, 0x16, + 0xa0, 0x2a, 0x45, 0x8c, 0x4c, 0x44, 0xa1, 0xb3, 0xae, 0x3f, 0x19, 0xcb, 0xfd, 0xcd, 0x82, 0x9b, + 0xb3, 0x99, 0x5f, 0xa5, 0x18, 0xcf, 0x93, 0x20, 0x54, 0x75, 0xb0, 0x1b, 0x95, 0xd6, 0xdd, 0xe6, + 0x7c, 0x4f, 0x37, 0xc7, 0x52, 0x51, 0x03, 0x76, 0xdf, 0xaf, 0x02, 0xd9, 0x8d, 0x91, 0x49, 0xd4, + 0xdf, 0x52, 0xf5, 0x67, 0x25, 0xb1, 0x72, 0x24, 0xc9, 0x26, 0xbe, 0x3a, 0x9b, 0xf8, 0x62, 0xc5, + 0x1c, 0x28, 0xbe, 0xc6, 0x58, 0xf0, 0x28, 0xd4, 0x72, 0xd9, 0x34, 0x35, 0xc9, 0x6d, 0x28, 0x07, + 0x28, 0x59, 0x77, 0xc8, 0xe4, 0xc0, 0xe8, 0x55, 0x52, 0x8e, 0x03, 0x26, 0x07, 0x8a, 0xcf, 0x63, + 0xe6, 0xa3, 0x70, 0x0a, 0x75, 0x5b, 0xf1, 0x29, 0x8f, 0xfa, 0xaa, 0xbb, 0x51, 0x9e, 0x0d, 0x31, + 0xed, 0xc6, 0xa2, 0x56, 0x61, 0x2b, 0x57, 0xba, 0x1f, 0xf0, 0xec, 0x47, 0xe6, 0x8f, 0xf0, 0x80, + 0xf1, 0x98, 0x82, 0x8a, 0x4a, 0xba, 0x91, 0xb4, 0x4d, 0xda, 0xe9, 0x22, 0xa5, 0x65, 0x17, 0xa9, + 0xe8, 0x30, 0xd3, 0xd3, 0xef, 0x6c, 0xd8, 0x4c, 0x44, 0xfa, 0xd7, 0x24, 0xcd, 0x6a, 0xb3, 0xfe, + 0x01, 0x6d, 0x0a, 0xff, 0x84, 0x36, 0xc5, 0xbf, 0xa3, 0x0d, 0xb9, 0x05, 0xa5, 0x70, 0x14, 0x74, + 0xe3, 0xe8, 0x8d, 0x52, 0x57, 0xe7, 0x10, 0x8e, 0x02, 0x1a, 0xbd, 0x11, 0x64, 0x17, 0xaa, 0x7d, + 0x8e, 0xbe, 0xd7, 0x4d, 0x86, 0xa9, 0x53, 0xd6, 0xcd, 0x5f, 0xcf, 0x12, 0x98, 0x41, 0xfb, 0x42, + 0x01, 0x0f, 0xf5, 0x6f, 0x5a, 0xe9, 0x4f, 0x0c, 0x72, 0x07, 0xca, 0x02, 0x4f, 0x02, 0x0c, 0x65, + 0xa7, 0xed, 0x80, 0x26, 0x98, 0x38, 0xdc, 0x00, 0xc8, 0x74, 0x61, 0xae, 0x72, 0xde, 0x96, 0x18, + 0x1a, 0xee, 0x77, 0xe0, 0xa4, 0x47, 0xfc, 0x05, 0xf7, 0x51, 0xd7, 0xe2, 0x72, 0xf3, 0xed, 0x4f, + 0x0b, 0x36, 0x33, 0xf1, 0x7a, 0xce, 0x7d, 0xac, 0x0d, 0x93, 0x06, 0x5c, 0x4f, 0x6a, 0xdc, 0xe7, + 0x3e, 0x9a, 0x66, 0xb2, 0x75, 0x33, 0x6d, 0xf0, 0x4c, 0x16, 0xe4, 0x11, 0x7c, 0x22, 0x30, 0xe6, + 0xcc, 0xe7, 0xe7, 0xe8, 0x75, 0x05, 0x3f, 0x4f, 0x46, 0xdf, 0x1a, 0xdd, 0x98, 0xb8, 0x0f, 0xf9, + 0x39, 0xba, 0x6f, 0x2d, 0xb8, 0x95, 0x23, 0xc2, 0x55, 0xa4, 0x6f, 0x03, 0x4c, 0xed, 0x2f, 0x19, + 0x77, 0x0f, 0x16, 0x8e, 0xbb, 0x69, 0xe5, 0x68, 0xb9, 0x9f, 0x6e, 0xc1, 0xfd, 0xd5, 0x36, 0x57, + 0xc7, 0x3e, 0x4a, 0xb6, 0xd4, 0xe9, 0x1c, 0x5f, 0x2f, 0xab, 0x97, 0xba, 0x5e, 0xee, 0x41, 0xa5, + 0xcf, 0xb8, 0xdf, 0x35, 0xd7, 0x80, 0xad, 0x4f, 0x35, 0x28, 0x17, 0xd5, 0x1e, 0xf2, 0x0d, 0xd8, + 0x31, 0x9e, 0x6a, 0xfd, 0x16, 0x24, 0x32, 0x37, 0x4d, 0xa8, 0x8a, 0xc8, 0x2d, 0xd7, 0x7a, 0x6e, + 0xb9, 0xb6, 0xa0, 0x1a, 0xb0, 0xf8, 0x55, 0xd7, 0x43, 0x1f, 0x25, 0x7a, 0x4e, 0xa1, 0x6e, 0x35, + 0x4a, 0xb4, 0xa2, 0x7c, 0xed, 0xc4, 0x35, 0xf5, 0x66, 0x28, 0x4e, 0xbf, 0x19, 0xa6, 0xa7, 0x75, + 0x29, 0x3b, 0xad, 0x6b, 0x50, 0x8a, 0xb1, 0x77, 0xd6, 0xf3, 0xd1, 0xd3, 0x87, 0xb5, 0x44, 0xc7, + 0x36, 0x79, 0x00, 0x93, 0x46, 0x48, 0xda, 0x03, 0x74, 0x7b, 0x5c, 0x1b, 0x7b, 0x75, 0x77, 0x3c, + 0x86, 0xeb, 0xed, 0x38, 0x1a, 0x66, 0x06, 0xe5, 0xd4, 0x94, 0xb3, 0x32, 0x53, 0xae, 0xf5, 0xbe, + 0x00, 0xa0, 0xa1, 0xbb, 0xea, 0xed, 0x46, 0x86, 0x40, 0xf6, 0x50, 0xee, 0x46, 0xc1, 0x30, 0x0a, + 0x31, 0x94, 0xc9, 0x2d, 0x4a, 0x9e, 0x2e, 0x78, 0x80, 0xcc, 0x43, 0x0d, 0x61, 0xed, 0xe1, 0x82, + 0x88, 0x19, 0xb8, 0xbb, 0x42, 0x02, 0xcd, 0x78, 0xc4, 0x03, 0x3c, 0xe2, 0xbd, 0x57, 0xbb, 0x03, + 0x16, 0x86, 0xe8, 0x5f, 0xc4, 0x38, 0x03, 0x4d, 0x19, 0x3f, 0xcf, 0x46, 0x18, 0xe3, 0x50, 0xc6, + 0x3c, 0x3c, 0x49, 0xcf, 0x86, 0xbb, 0x42, 0x4e, 0xe1, 0xc6, 0x1e, 0x6a, 0x76, 0x2e, 0x24, 0xef, + 0x89, 0x94, 0xb0, 0xb5, 0x98, 0x70, 0x0e, 0x7c, 0x49, 0xca, 0x9f, 0x01, 0x26, 0xcd, 0x46, 0x96, + 0x6b, 0xc6, 0x79, 0x01, 0x67, 0x61, 0xe3, 0xe5, 0x39, 0x6c, 0x64, 0x1f, 0x3d, 0xe4, 0x8b, 0xbc, + 0xd8, 0xdc, 0x27, 0x61, 0xed, 0xcb, 0x65, 0xa0, 0x63, 0xaa, 0x18, 0x36, 0xe7, 0xe6, 0x0e, 0x79, + 0x7c, 0xd1, 0x12, 0xb3, 0x33, 0xba, 0xf6, 0x64, 0x49, 0xf4, 0x98, 0xf3, 0x00, 0xca, 0xe3, 0x76, + 0x26, 0xf7, 0xf3, 0xa2, 0x67, 0xbb, 0xbd, 0x76, 0xd1, 0xc4, 0x73, 0x57, 0x48, 0x17, 0x60, 0x0f, + 0xe5, 0x3e, 0xca, 0x98, 0xf7, 0x04, 0x79, 0x98, 0x5b, 0xc4, 0x09, 0x20, 0x5d, 0xf4, 0xd1, 0x07, + 0x71, 0xe9, 0x96, 0x5b, 0x6f, 0xd7, 0xcc, 0x18, 0x54, 0xff, 0x07, 0xfe, 0x3f, 0x52, 0x1f, 0xe1, + 0x48, 0x1d, 0x41, 0x65, 0xea, 0x85, 0x4d, 0x72, 0x0f, 0xcb, 0xfc, 0x13, 0xfc, 0xbf, 0x6e, 0x8c, + 0x9d, 0xaf, 0x7e, 0x6a, 0x9d, 0x70, 0x39, 0x18, 0x1d, 0x2b, 0xea, 0xed, 0x04, 0xf9, 0x84, 0x47, + 0xe6, 0xd7, 0x76, 0xaa, 0xd0, 0xb6, 0x5e, 0x69, 0x5b, 0xa7, 0x31, 0x3c, 0x3e, 0x2e, 0x68, 0xf3, + 0xd9, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0x71, 0x6a, 0x92, 0xdd, 0x65, 0x0f, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. diff --git a/internal/proxy/data_coord_mock_test.go b/internal/proxy/data_coord_mock_test.go index d75a72b9fd..8fff6192a1 100644 --- a/internal/proxy/data_coord_mock_test.go +++ b/internal/proxy/data_coord_mock_test.go @@ -221,6 +221,20 @@ func (coord *DataCoordMock) UpdateSegmentStatistics(ctx context.Context, req *da }, nil } +func (coord *DataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, nil +} + +func (coord *DataCoordMock) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, nil +} + func NewDataCoordMock() *DataCoordMock { return &DataCoordMock{ nodeID: typeutil.UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()), diff --git a/internal/querycoord/global_meta_broker.go b/internal/querycoord/global_meta_broker.go index 308eb95012..b2dc51d466 100644 --- a/internal/querycoord/global_meta_broker.go +++ b/internal/querycoord/global_meta_broker.go @@ -6,6 +6,8 @@ import ( "fmt" "path" + "github.com/milvus-io/milvus/internal/util/retry" + "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "go.uber.org/zap" @@ -476,3 +478,55 @@ func (broker *globalMetaBroker) getSegmentStates(ctx context.Context, segmentID return resp.States[0], nil } + +func (broker *globalMetaBroker) acquireSegmentsReferLock(ctx context.Context, segmentIDs []UniqueID) error { + ctx, cancel := context.WithTimeout(ctx, timeoutForRPC) + defer cancel() + acquireSegLockReq := &datapb.AcquireSegmentLockRequest{ + SegmentIDs: segmentIDs, + NodeID: Params.QueryCoordCfg.GetNodeID(), + } + status, err := broker.dataCoord.AcquireSegmentLock(ctx, acquireSegLockReq) + if err != nil { + log.Error("QueryCoord acquire the segment reference lock error", zap.Int64s("segIDs", segmentIDs), + zap.Error(err)) + return err + } + if status.ErrorCode != commonpb.ErrorCode_Success { + log.Error("QueryCoord acquire the segment reference lock error", zap.Int64s("segIDs", segmentIDs), + zap.String("failed reason", status.Reason)) + return fmt.Errorf(status.Reason) + } + + return nil +} + +func (broker *globalMetaBroker) releaseSegmentReferLock(ctx context.Context, segmentIDs []UniqueID) error { + ctx, cancel := context.WithTimeout(ctx, timeoutForRPC) + defer cancel() + + releaseSegReferLockReq := &datapb.ReleaseSegmentLockRequest{ + SegmentIDs: segmentIDs, + NodeID: Params.QueryCoordCfg.GetNodeID(), + } + + if err := retry.Do(ctx, func() error { + status, err := broker.dataCoord.ReleaseSegmentLock(ctx, releaseSegReferLockReq) + if err != nil { + log.Error("QueryCoord release reference lock on segments failed", zap.Int64s("segmentIDs", segmentIDs), + zap.Error(err)) + return err + } + + if status.ErrorCode != commonpb.ErrorCode_Success { + log.Error("QueryCoord release reference lock on segments failed", zap.Int64s("segmentIDs", segmentIDs), + zap.String("failed reason", status.Reason)) + return errors.New(status.Reason) + } + return nil + }, retry.Attempts(100)); err != nil { + return err + } + + return nil +} diff --git a/internal/querycoord/mock_3rd_component_test.go b/internal/querycoord/mock_3rd_component_test.go index afb7555079..35c47bf2bf 100644 --- a/internal/querycoord/mock_3rd_component_test.go +++ b/internal/querycoord/mock_3rd_component_test.go @@ -334,6 +334,7 @@ type dataCoordMock struct { returnError bool returnGrpcError bool segmentState commonpb.SegmentState + errLevel int } func newDataCoordMock(ctx context.Context) *dataCoordMock { @@ -448,6 +449,42 @@ func (data *dataCoordMock) GetSegmentStates(ctx context.Context, req *datapb.Get }, nil } +func (data *dataCoordMock) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) { + if data.errLevel == 2 { + data.errLevel++ + return nil, errors.New("AcquireSegmentLock failed") + + } + if data.errLevel == 1 { + data.errLevel++ + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "AcquireSegmentLock failed", + }, nil + } + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, nil +} +func (data *dataCoordMock) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) { + if data.errLevel == 4 { + data.errLevel++ + return nil, errors.New("ReleaseSegmentLock failed") + } + + if data.errLevel == 3 { + data.errLevel++ + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Reason: "ReleaseSegmentLock failed", + }, nil + } + + return &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, nil +} + type indexCoordMock struct { types.IndexCoord chunkManager storage.ChunkManager diff --git a/internal/querycoord/task.go b/internal/querycoord/task.go index 7a29f13b63..32fbe402e1 100644 --- a/internal/querycoord/task.go +++ b/internal/querycoord/task.go @@ -509,7 +509,7 @@ func (lct *loadCollectionTask) execute(ctx context.Context) error { watchDmChannelReqs = append(watchDmChannelReqs, watchRequest) } - internalTasks, err := assignInternalTask(ctx, lct, lct.meta, lct.cluster, loadSegmentReqs, watchDmChannelReqs, false, nil, replica.GetNodeIds(), -1) + internalTasks, err := assignInternalTask(ctx, lct, lct.meta, lct.cluster, loadSegmentReqs, watchDmChannelReqs, false, nil, replica.GetNodeIds(), -1, lct.broker) if err != nil { log.Error("loadCollectionTask: assign child task failed", zap.Int64("collectionID", collectionID), zap.Int64("msgID", lct.Base.MsgID), zap.Error(err)) lct.setResultInfo(err) @@ -939,7 +939,7 @@ func (lpt *loadPartitionTask) execute(ctx context.Context) error { watchDmChannelReqs = append(watchDmChannelReqs, watchRequest) } - internalTasks, err := assignInternalTask(ctx, lpt, lpt.meta, lpt.cluster, loadSegmentReqs, watchDmChannelReqs, false, nil, replica.GetNodeIds(), -1) + internalTasks, err := assignInternalTask(ctx, lpt, lpt.meta, lpt.cluster, loadSegmentReqs, watchDmChannelReqs, false, nil, replica.GetNodeIds(), -1, lpt.broker) if err != nil { log.Error("loadPartitionTask: assign child task failed", zap.Int64("collectionID", collectionID), zap.Int64s("partitionIDs", partitionIDs), zap.Int64("msgID", lpt.Base.MsgID), zap.Error(err)) lpt.setResultInfo(err) @@ -1176,6 +1176,8 @@ type loadSegmentTask struct { meta Meta cluster Cluster excludeNodeIDs []int64 + + broker *globalMetaBroker } func (lst *loadSegmentTask) msgBase() *commonpb.MsgBase { @@ -1212,7 +1214,7 @@ func (lst *loadSegmentTask) updateTaskProcess() { parentTask.updateTaskProcess() } -func (lst *loadSegmentTask) preExecute(context.Context) error { +func (lst *loadSegmentTask) preExecute(ctx context.Context) error { segmentIDs := make([]UniqueID, 0) for _, info := range lst.Infos { segmentIDs = append(segmentIDs, info.SegmentID) @@ -1222,6 +1224,12 @@ func (lst *loadSegmentTask) preExecute(context.Context) error { zap.Int64s("segmentIDs", segmentIDs), zap.Int64("loaded nodeID", lst.DstNodeID), zap.Int64("taskID", lst.getTaskID())) + + if err := lst.broker.acquireSegmentsReferLock(ctx, segmentIDs); err != nil { + log.Error("acquire reference lock on segments failed", zap.Int64s("segmentIDs", segmentIDs), + zap.Error(err)) + return err + } return nil } @@ -1241,6 +1249,14 @@ func (lst *loadSegmentTask) execute(ctx context.Context) error { } func (lst *loadSegmentTask) postExecute(context.Context) error { + segmentIDs := make([]UniqueID, 0) + for _, info := range lst.Infos { + segmentIDs = append(segmentIDs, info.SegmentID) + } + if err := lst.broker.releaseSegmentReferLock(lst.ctx, segmentIDs); err != nil { + panic(err) + } + log.Info("loadSegmentTask postExecute done", zap.Int64("taskID", lst.getTaskID())) return nil @@ -1275,7 +1291,7 @@ func (lst *loadSegmentTask) reschedule(ctx context.Context) ([]task, error) { if lst.getParentTask().getTriggerCondition() == querypb.TriggerCondition_NodeDown { wait2AssignTaskSuccess = true } - reScheduledTasks, err := assignInternalTask(ctx, lst.getParentTask(), lst.meta, lst.cluster, loadSegmentReqs, nil, wait2AssignTaskSuccess, lst.excludeNodeIDs, nil, lst.ReplicaID) + reScheduledTasks, err := assignInternalTask(ctx, lst.getParentTask(), lst.meta, lst.cluster, loadSegmentReqs, nil, wait2AssignTaskSuccess, lst.excludeNodeIDs, nil, lst.ReplicaID, lst.broker) if err != nil { log.Error("loadSegment reschedule failed", zap.Int64s("excludeNodes", lst.excludeNodeIDs), zap.Int64("taskID", lst.getTaskID()), zap.Error(err)) return nil, err @@ -1456,7 +1472,7 @@ func (wdt *watchDmChannelTask) reschedule(ctx context.Context) ([]task, error) { if wdt.getParentTask().getTriggerCondition() == querypb.TriggerCondition_NodeDown { wait2AssignTaskSuccess = true } - reScheduledTasks, err := assignInternalTask(ctx, wdt.parentTask, wdt.meta, wdt.cluster, nil, watchDmChannelReqs, wait2AssignTaskSuccess, wdt.excludeNodeIDs, nil, wdt.ReplicaID) + reScheduledTasks, err := assignInternalTask(ctx, wdt.parentTask, wdt.meta, wdt.cluster, nil, watchDmChannelReqs, wait2AssignTaskSuccess, wdt.excludeNodeIDs, nil, wdt.ReplicaID, nil) if err != nil { log.Error("watchDmChannel reschedule failed", zap.Int64("taskID", wdt.getTaskID()), zap.Int64s("excludeNodes", wdt.excludeNodeIDs), zap.Error(err)) return nil, err @@ -1591,7 +1607,7 @@ func (ht *handoffTask) execute(ctx context.Context) error { } if collectionInfo.LoadType == querypb.LoadType_LoadCollection && ht.meta.hasReleasePartition(collectionID, partitionID) { - log.Warn("handoffTask: partition has not been released", zap.Int64("collectionID", collectionID), zap.Int64("partitionID", partitionID)) + log.Warn("handoffTask: partition has been released", zap.Int64("collectionID", collectionID), zap.Int64("partitionID", partitionID)) continue } @@ -1687,7 +1703,7 @@ func (ht *handoffTask) execute(ctx context.Context) error { // we should copy a request because assignInternalTask will change DstNodeID of LoadSegmentRequest clonedReq := proto.Clone(loadSegmentReq).(*querypb.LoadSegmentsRequest) clonedReq.ReplicaID = replica.ReplicaID - tasks, err := assignInternalTask(ctx, ht, ht.meta, ht.cluster, []*querypb.LoadSegmentsRequest{clonedReq}, nil, true, nil, nil, replica.GetReplicaID()) + tasks, err := assignInternalTask(ctx, ht, ht.meta, ht.cluster, []*querypb.LoadSegmentsRequest{clonedReq}, nil, true, nil, nil, replica.GetReplicaID(), ht.broker) if err != nil { log.Error("handoffTask: assign child task failed", zap.Int64("collectionID", collectionID), zap.Int64("segmentID", segmentID), zap.Error(err)) ht.setResultInfo(err) @@ -1960,7 +1976,7 @@ func (lbt *loadBalanceTask) execute(ctx context.Context) error { } } - tasks, err := assignInternalTask(ctx, lbt, lbt.meta, lbt.cluster, loadSegmentReqs, watchDmChannelReqs, true, lbt.SourceNodeIDs, lbt.DstNodeIDs, replica.GetReplicaID()) + tasks, err := assignInternalTask(ctx, lbt, lbt.meta, lbt.cluster, loadSegmentReqs, watchDmChannelReqs, true, lbt.SourceNodeIDs, lbt.DstNodeIDs, replica.GetReplicaID(), lbt.broker) if err != nil { log.Error("loadBalanceTask: assign child task failed", zap.Int64("sourceNodeID", nodeID)) lbt.setResultInfo(err) @@ -2144,7 +2160,7 @@ func (lbt *loadBalanceTask) execute(ctx context.Context) error { return err } } - internalTasks, err := assignInternalTask(ctx, lbt, lbt.meta, lbt.cluster, loadSegmentReqs, nil, false, lbt.SourceNodeIDs, lbt.DstNodeIDs, lbt.replicaID) + internalTasks, err := assignInternalTask(ctx, lbt, lbt.meta, lbt.cluster, loadSegmentReqs, nil, false, lbt.SourceNodeIDs, lbt.DstNodeIDs, lbt.replicaID, lbt.broker) if err != nil { log.Error("loadBalanceTask: assign child task failed", zap.Any("balance request", lbt.LoadBalanceRequest)) lbt.setResultInfo(err) @@ -2350,7 +2366,8 @@ func assignInternalTask(ctx context.Context, parentTask task, meta Meta, cluster Cluster, loadSegmentRequests []*querypb.LoadSegmentsRequest, watchDmChannelRequests []*querypb.WatchDmChannelsRequest, - wait bool, excludeNodeIDs []int64, includeNodeIDs []int64, replicaID int64) ([]task, error) { + wait bool, excludeNodeIDs []int64, includeNodeIDs []int64, replicaID int64, + broker *globalMetaBroker) ([]task, error) { internalTasks := make([]task, 0) err := cluster.allocateSegmentsToQueryNode(ctx, loadSegmentRequests, wait, excludeNodeIDs, includeNodeIDs, replicaID) @@ -2387,6 +2404,7 @@ func assignInternalTask(ctx context.Context, meta: meta, cluster: cluster, excludeNodeIDs: excludeNodeIDs, + broker: broker, } internalTasks = append(internalTasks, loadSegmentTask) @@ -2407,6 +2425,7 @@ func assignInternalTask(ctx context.Context, meta: meta, cluster: cluster, excludeNodeIDs: excludeNodeIDs, + broker: broker, } internalTasks = append(internalTasks, loadSegmentTask) } diff --git a/internal/querycoord/task_scheduler.go b/internal/querycoord/task_scheduler.go index a7cf173139..abd865b757 100644 --- a/internal/querycoord/task_scheduler.go +++ b/internal/querycoord/task_scheduler.go @@ -336,6 +336,7 @@ func (scheduler *TaskScheduler) unmarshalTask(taskID UniqueID, t string) (task, cluster: scheduler.cluster, meta: scheduler.meta, excludeNodeIDs: []int64{}, + broker: scheduler.broker, } newTask = loadSegmentTask case commonpb.MsgType_ReleaseSegments: @@ -521,7 +522,11 @@ func (scheduler *TaskScheduler) processTask(t task) error { // task preExecute span.LogFields(oplog.Int64("processTask: scheduler process PreExecute", t.getTaskID())) - t.preExecute(ctx) + err = t.preExecute(ctx) + if err != nil { + log.Warn("failed to preExecute task", zap.Error(err)) + return err + } taskInfoKey = fmt.Sprintf("%s/%d", taskInfoPrefix, t.getTaskID()) err = scheduler.client.Save(taskInfoKey, strconv.Itoa(int(taskDoing))) if err != nil { diff --git a/internal/querycoord/task_test.go b/internal/querycoord/task_test.go index 9450b37b58..d7c2fdacc6 100644 --- a/internal/querycoord/task_test.go +++ b/internal/querycoord/task_test.go @@ -18,9 +18,13 @@ package querycoord import ( "context" + "math/rand" "testing" "time" + "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/internal/util/etcd" + "github.com/stretchr/testify/assert" "go.uber.org/zap" @@ -213,13 +217,14 @@ func genLoadSegmentTask(ctx context.Context, queryCoord *QueryCoord, nodeID int6 }, } baseTask := newBaseTask(ctx, querypb.TriggerCondition_GrpcRequest) - baseTask.taskID = 100 + baseTask.taskID = rand.Int63() loadSegmentTask := &loadSegmentTask{ baseTask: baseTask, LoadSegmentsRequest: req, cluster: queryCoord.cluster, meta: queryCoord.meta, excludeNodeIDs: []int64{}, + broker: queryCoord.broker, } parentReq := &querypb.LoadCollectionRequest{ @@ -764,7 +769,7 @@ func Test_AssignInternalTask(t *testing.T) { loadSegmentRequests = append(loadSegmentRequests, req) } - internalTasks, err := assignInternalTask(queryCoord.loopCtx, loadCollectionTask, queryCoord.meta, queryCoord.cluster, loadSegmentRequests, nil, false, nil, nil, -1) + internalTasks, err := assignInternalTask(queryCoord.loopCtx, loadCollectionTask, queryCoord.meta, queryCoord.cluster, loadSegmentRequests, nil, false, nil, nil, -1, nil) assert.Nil(t, err) assert.NotEqual(t, 1, len(internalTasks)) @@ -1377,3 +1382,82 @@ func TestUpdateTaskProcessWhenWatchDmChannel(t *testing.T) { err = removeAllSession() assert.Nil(t, err) } + +func startMockCoord(ctx context.Context) (*QueryCoord, error) { + factory := dependency.NewDefaultFactory(true) + + coord, err := NewQueryCoordTest(ctx, factory) + if err != nil { + return nil, err + } + + rootCoord := newRootCoordMock(ctx) + rootCoord.createCollection(defaultCollectionID) + rootCoord.createPartition(defaultCollectionID, defaultPartitionID) + + dataCoord := &dataCoordMock{ + collections: make([]UniqueID, 0), + col2DmChannels: make(map[UniqueID][]*datapb.VchannelInfo), + partitionID2Segment: make(map[UniqueID][]UniqueID), + Segment2Binlog: make(map[UniqueID]*datapb.SegmentBinlogs), + baseSegmentID: defaultSegmentID, + channelNumPerCol: defaultChannelNum, + segmentState: commonpb.SegmentState_Flushed, + errLevel: 1, + } + indexCoord, err := newIndexCoordMock(queryCoordTestDir) + if err != nil { + return nil, err + } + + coord.SetRootCoord(rootCoord) + coord.SetDataCoord(dataCoord) + coord.SetIndexCoord(indexCoord) + etcd, err := etcd.GetEtcdClient(&Params.EtcdCfg) + if err != nil { + return nil, err + } + coord.SetEtcdClient(etcd) + err = coord.Init() + if err != nil { + return nil, err + } + err = coord.Start() + if err != nil { + return nil, err + } + err = coord.Register() + if err != nil { + return nil, err + } + return coord, nil +} + +func Test_LoadSegment(t *testing.T) { + refreshParams() + ctx := context.Background() + queryCoord, err := startMockCoord(ctx) + assert.Nil(t, err) + + node1, err := startQueryNodeServer(ctx) + assert.Nil(t, err) + + waitQueryNodeOnline(queryCoord.cluster, node1.queryNodeID) + + loadSegmentTask := genLoadSegmentTask(ctx, queryCoord, node1.queryNodeID) + err = loadSegmentTask.meta.setDeltaChannel(111, []*datapb.VchannelInfo{}) + assert.Nil(t, err) + loadCollectionTask := loadSegmentTask.parentTask + queryCoord.scheduler.triggerTaskQueue.addTask(loadCollectionTask) + + // 1. Acquire segment reference lock failed, and reschedule task. + // 2. Acquire segment reference lock successfully, but release reference lock failed, and retry release the lock. + // 3. Release segment reference lock successfully, and task done. + waitTaskFinalState(loadSegmentTask, taskDone) + + err = queryCoord.Stop() + assert.Nil(t, err) + + err = removeAllSession() + assert.Nil(t, err) +} diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index 19eb4b80e3..550f486e83 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -128,13 +128,12 @@ type Core struct { //setMsgStreams, send drop partition into dd channel SendDdDropPartitionReq func(ctx context.Context, req *internalpb.DropPartitionRequest, channelNames []string) error - //get binlog file path from data service, - CallGetBinlogFilePathsService func(ctx context.Context, segID typeutil.UniqueID, fieldID typeutil.UniqueID) ([]string, error) - CallGetNumRowsService func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (int64, error) + //get segment info from data service CallGetFlushedSegmentsService func(ctx context.Context, collID, partID typeutil.UniqueID) ([]typeutil.UniqueID, error) + CallGetRecoveryInfoService func(ctx context.Context, collID, partID UniqueID) ([]*datapb.SegmentBinlogs, error) //call index builder's client to build index, return build id or get index state. - CallBuildIndexService func(ctx context.Context, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) + CallBuildIndexService func(ctx context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) CallDropIndexService func(ctx context.Context, indexID typeutil.UniqueID) error CallGetIndexStatesService func(ctx context.Context, IndexBuildIDs []int64) ([]*indexpb.IndexInfo, error) @@ -264,11 +263,11 @@ func (c *Core) checkInit() error { if c.SendDdDropPartitionReq == nil { return fmt.Errorf("sendDdDropPartitionReq is nil") } - if c.CallGetBinlogFilePathsService == nil { - return fmt.Errorf("callGetBinlogFilePathsService is nil") + if c.CallGetFlushedSegmentsService == nil { + return fmt.Errorf("callGetFlushedSegmentsService is nil") } - if c.CallGetNumRowsService == nil { - return fmt.Errorf("callGetNumRowsService is nil") + if c.CallGetRecoveryInfoService == nil { + return fmt.Errorf("CallGetRecoveryInfoService is nil") } if c.CallBuildIndexService == nil { return fmt.Errorf("callBuildIndexService is nil") @@ -276,9 +275,6 @@ func (c *Core) checkInit() error { if c.CallDropIndexService == nil { return fmt.Errorf("callDropIndexService is nil") } - if c.CallGetFlushedSegmentsService == nil { - return fmt.Errorf("callGetFlushedSegmentsService is nil") - } if c.CallWatchChannels == nil { return fmt.Errorf("callWatchChannels is nil") } @@ -369,7 +365,7 @@ func (c *Core) checkFlushedSegments(ctx context.Context) { } for _, partID := range collMeta.PartitionIDs { ctx2, cancel2 := context.WithTimeout(ctx, 3*time.Minute) - segIDs, err := c.CallGetFlushedSegmentsService(ctx2, collMeta.ID, partID) + segBinlogs, err := c.CallGetRecoveryInfoService(ctx2, collMeta.ID, partID) if err != nil { log.Debug("failed to get flushed segments from dataCoord", zap.Int64("collection ID", collMeta.GetID()), @@ -378,7 +374,8 @@ func (c *Core) checkFlushedSegments(ctx context.Context) { cancel2() continue } - for _, segID := range segIDs { + for _, segBinlog := range segBinlogs { + segID := segBinlog.SegmentID var indexInfos []*etcdpb.FieldIndexInfo indexMeta, ok := segID2IndexMeta[segID] if !ok { @@ -416,7 +413,7 @@ func (c *Core) checkFlushedSegments(ctx context.Context) { zap.Int64("segment_id", segID), zap.Int64("index_id", indexMeta.IndexID), zap.Int64("collection_id", collMeta.ID)) - info.BuildID, err = c.BuildIndex(ctx2, segID, field, &indexMeta, false) + info.BuildID, err = c.BuildIndex(ctx2, segID, segBinlog.GetNumOfRows(), segBinlog.GetFieldBinlogs(), field, &indexMeta, false) if err != nil { log.Debug("build index failed", zap.Int64("segment_id", segID), @@ -441,27 +438,29 @@ func (c *Core) checkFlushedSegments(ctx context.Context) { } } -func (c *Core) getSegments(ctx context.Context, collID typeutil.UniqueID) (map[typeutil.UniqueID]typeutil.UniqueID, error) { +func (c *Core) getSegments(ctx context.Context, collID typeutil.UniqueID) (map[UniqueID]UniqueID, map[UniqueID]*datapb.SegmentBinlogs, error) { collMeta, err := c.MetaTable.GetCollectionByID(collID, 0) if err != nil { - return nil, err + return nil, nil, err } - segID2PartID := map[typeutil.UniqueID]typeutil.UniqueID{} + segID2PartID := make(map[UniqueID]UniqueID) + segID2Binlog := make(map[UniqueID]*datapb.SegmentBinlogs) for _, partID := range collMeta.PartitionIDs { - if seg, err := c.CallGetFlushedSegmentsService(ctx, collID, partID); err == nil { - for _, s := range seg { - segID2PartID[s] = partID + if segs, err := c.CallGetRecoveryInfoService(ctx, collID, partID); err == nil { + for _, s := range segs { + segID2PartID[s.SegmentID] = partID + segID2Binlog[s.SegmentID] = s } } else { - log.Error("failed to get flushed segments from dataCoord", + log.Error("failed to get flushed segments info from dataCoord", zap.Int64("collection ID", collID), zap.Int64("partition ID", partID), zap.Error(err)) - return nil, err + return nil, nil, err } } - return segID2PartID, nil + return segID2PartID, segID2Binlog, nil } func (c *Core) setMsgStreams() error { @@ -580,80 +579,6 @@ func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error { log.Debug("Retrying RootCoord connection to DataCoord") } }() - c.CallGetBinlogFilePathsService = func(ctx context.Context, segID typeutil.UniqueID, fieldID typeutil.UniqueID) (retFiles []string, retErr error) { - defer func() { - if err := recover(); err != nil { - retErr = fmt.Errorf("get bin log file paths panic, msg = %v", err) - } - }() - <-initCh //wait connect to DataCoord - ts, err := c.TSOAllocator(1) - if err != nil { - return nil, err - } - binlog, err := s.GetInsertBinlogPaths(ctx, &datapb.GetInsertBinlogPathsRequest{ - Base: &commonpb.MsgBase{ - MsgType: 0, //TODO, msg type - MsgID: 0, - Timestamp: ts, - SourceID: c.session.ServerID, - }, - SegmentID: segID, - }) - if err != nil { - return nil, err - } - if binlog.Status.ErrorCode != commonpb.ErrorCode_Success { - return nil, fmt.Errorf("getInsertBinlogPaths from data service failed, error = %s", binlog.Status.Reason) - } - binlogPaths := make([]string, 0) - for i := range binlog.FieldIDs { - if binlog.FieldIDs[i] == fieldID { - binlogPaths = append(binlogPaths, binlog.Paths[i].Values...) - } - } - if len(binlogPaths) == 0 { - return nil, fmt.Errorf("binlog file does not exist, segment id = %d, field id = %d", segID, fieldID) - } - return binlogPaths, nil - } - - c.CallGetNumRowsService = func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (retRows int64, retErr error) { - defer func() { - if err := recover(); err != nil { - retErr = fmt.Errorf("get num rows panic, msg = %v", err) - } - }() - <-initCh - ts, err := c.TSOAllocator(1) - if err != nil { - return retRows, err - } - segInfo, err := s.GetSegmentInfo(ctx, &datapb.GetSegmentInfoRequest{ - Base: &commonpb.MsgBase{ - MsgType: 0, //TODO, msg type - MsgID: 0, - Timestamp: ts, - SourceID: c.session.ServerID, - }, - SegmentIDs: []typeutil.UniqueID{segID}, - }) - if err != nil { - return retRows, err - } - if segInfo.Status.ErrorCode != commonpb.ErrorCode_Success { - return retRows, fmt.Errorf("getSegmentInfo from data service failed, error = %s", segInfo.Status.Reason) - } - if len(segInfo.Infos) != 1 { - log.Debug("get segment info empty") - return retRows, nil - } - if !isFromFlushedChan && segInfo.Infos[0].State != commonpb.SegmentState_Flushed { - log.Debug("segment id not flushed", zap.Int64("segment id", segID)) - return retRows, nil - } - return segInfo.Infos[0].NumOfRows, nil - } c.CallGetFlushedSegmentsService = func(ctx context.Context, collID, partID typeutil.UniqueID) (retSegIDs []typeutil.UniqueID, retErr error) { defer func() { @@ -682,6 +607,27 @@ func (c *Core) SetDataCoord(ctx context.Context, s types.DataCoord) error { return rsp.Segments, nil } + c.CallGetRecoveryInfoService = func(ctx context.Context, collID, partID typeutil.UniqueID) ([]*datapb.SegmentBinlogs, error) { + getSegmentInfoReq := &datapb.GetRecoveryInfoRequest{ + Base: &commonpb.MsgBase{ + MsgType: 0, //TODO, msg type + MsgID: 0, + Timestamp: 0, + SourceID: c.session.ServerID, + }, + CollectionID: collID, + PartitionID: partID, + } + resp, err := s.GetRecoveryInfo(ctx, getSegmentInfoReq) + if err != nil { + return nil, err + } + if resp.Status.ErrorCode != commonpb.ErrorCode_Success { + return nil, errors.New(resp.Status.Reason) + } + return resp.Binlogs, nil + } + c.CallWatchChannels = func(ctx context.Context, collectionID int64, channelNames []string) (retErr error) { defer func() { if err := recover(); err != nil { @@ -758,7 +704,7 @@ func (c *Core) SetIndexCoord(s types.IndexCoord) error { } }() - c.CallBuildIndexService = func(ctx context.Context, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (retID typeutil.UniqueID, retErr error) { + c.CallBuildIndexService = func(ctx context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (retID typeutil.UniqueID, retErr error) { defer func() { if err := recover(); err != nil { retErr = fmt.Errorf("build index panic, msg = %v", err) @@ -773,6 +719,7 @@ func (c *Core) SetIndexCoord(s types.IndexCoord) error { IndexName: idxInfo.IndexName, NumRows: numRows, FieldSchema: field, + SegmentID: segID, }) if err != nil { return retID, err @@ -917,7 +864,7 @@ func (c *Core) SetQueryCoord(s types.QueryCoord) error { } // BuildIndex will check row num and call build index service -func (c *Core) BuildIndex(ctx context.Context, segID typeutil.UniqueID, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, isFlush bool) (typeutil.UniqueID, error) { +func (c *Core) BuildIndex(ctx context.Context, segID UniqueID, numRows int64, binlogs []*datapb.FieldBinlog, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, isFlush bool) (typeutil.UniqueID, error) { log.Debug("start build index", zap.String("index name", idxInfo.IndexName), zap.String("field name", field.Name), zap.Int64("segment id", segID)) sp, ctx := trace.StartSpanFromContext(ctx) @@ -926,27 +873,24 @@ func (c *Core) BuildIndex(ctx context.Context, segID typeutil.UniqueID, field *s info, err := c.MetaTable.GetSegmentIndexInfoByID(segID, field.FieldID, idxInfo.GetIndexName()) return info.BuildID, err } - rows, err := c.CallGetNumRowsService(ctx, segID, isFlush) - if err != nil { - return 0, err - } - var bldID typeutil.UniqueID - if rows < Params.RootCoordCfg.MinSegmentSizeToEnableIndex { - log.Debug("num of rows is less than MinSegmentSizeToEnableIndex", zap.Int64("num rows", rows)) + var bldID UniqueID + var err error + if numRows < Params.RootCoordCfg.MinSegmentSizeToEnableIndex { + log.Debug("num of rows is less than MinSegmentSizeToEnableIndex", zap.Int64("num rows", numRows)) } else { - binlogs, err := c.CallGetBinlogFilePathsService(ctx, segID, field.FieldID) - if err != nil { - return 0, err + binLogs := make([]string, 0) + for _, fieldBinLog := range binlogs { + if fieldBinLog.GetFieldID() == field.GetFieldID() { + for _, binLog := range fieldBinLog.GetBinlogs() { + binLogs = append(binLogs, binLog.LogPath) + } + break + } } - bldID, err = c.CallBuildIndexService(ctx, binlogs, field, idxInfo, rows) - if err != nil { - return 0, err - } - - log.Debug("CallBuildIndex finished", zap.String("index name", idxInfo.IndexName), - zap.String("field name", field.Name), zap.Int64("segment id", segID), zap.Int64("num rows", rows)) + bldID, err = c.CallBuildIndexService(ctx, segID, binLogs, field, idxInfo, numRows) } - return bldID, nil + + return bldID, err } // RemoveIndex will call drop index service @@ -2136,7 +2080,7 @@ func (c *Core) SegmentFlushCompleted(ctx context.Context, in *datapb.SegmentFlus IndexID: idxInfo.IndexID, EnableIndex: false, } - info.BuildID, err = c.BuildIndex(ctx, segID, fieldSch, idxInfo, true) + info.BuildID, err = c.BuildIndex(ctx, segID, in.Segment.GetNumOfRows(), in.Segment.GetBinlogs(), fieldSch, idxInfo, true) if err == nil && info.BuildID != 0 { info.EnableIndex = true } else { diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 0f10ed6c3e..0600826e17 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -199,6 +199,47 @@ func (d *dataMock) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInf }, nil } +func (d *dataMock) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) { + var fieldBinlog []*datapb.FieldBinlog + for i := 0; i < 200; i++ { + binlog := &datapb.FieldBinlog{ + FieldID: int64(i), + Binlogs: []*datapb.Binlog{ + { + LogPath: fmt.Sprintf("file0-%d", i), + }, + { + LogPath: fmt.Sprintf("file1-%d", i), + }, + { + LogPath: fmt.Sprintf("file2-%d", i), + }, + }, + } + fieldBinlog = append(fieldBinlog, binlog) + } + + d.mu.Lock() + segmentBinlogs := make([]*datapb.SegmentBinlogs, 0, len(d.segs)) + for _, segID := range d.segs { + segmentBinlog := &datapb.SegmentBinlogs{ + SegmentID: segID, + NumOfRows: Params.RootCoordCfg.MinSegmentSizeToEnableIndex, + FieldBinlogs: fieldBinlog, + } + segmentBinlogs = append(segmentBinlogs, segmentBinlog) + } + d.mu.Unlock() + + return &datapb.GetRecoveryInfoResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + Reason: "", + }, + Binlogs: segmentBinlogs, + }, nil +} + func (d *dataMock) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) { d.mu.Lock() defer d.mu.Unlock() @@ -2939,25 +2980,21 @@ func TestCheckInit(t *testing.T) { err = c.checkInit() assert.Error(t, err) - c.CallGetBinlogFilePathsService = func(ctx context.Context, segID, fieldID typeutil.UniqueID) ([]string, error) { - return []string{}, nil - } - err = c.checkInit() - assert.Error(t, err) - - c.CallGetNumRowsService = func(ctx context.Context, segID typeutil.UniqueID, isFromFlushedChan bool) (int64, error) { - return 0, nil - } - err = c.checkInit() - assert.Error(t, err) - c.CallGetFlushedSegmentsService = func(ctx context.Context, collID, partID typeutil.UniqueID) ([]typeutil.UniqueID, error) { return nil, nil } + err = c.checkInit() assert.Error(t, err) - c.CallBuildIndexService = func(ctx context.Context, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) { + c.CallGetRecoveryInfoService = func(ctx context.Context, collID, partID UniqueID) ([]*datapb.SegmentBinlogs, error) { + return nil, nil + } + + err = c.checkInit() + assert.Error(t, err) + + c.CallBuildIndexService = func(ctx context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idxInfo *etcdpb.IndexInfo, numRows int64) (typeutil.UniqueID, error) { return 0, nil } err = c.checkInit() @@ -3142,7 +3179,7 @@ func TestCheckFlushedSegments(t *testing.T) { core.MetaTable.indexID2Meta[indexID] = etcdpb.IndexInfo{ IndexID: indexID, } - core.CallBuildIndexService = func(_ context.Context, binlog []string, field *schemapb.FieldSchema, idx *etcdpb.IndexInfo, numRows int64) (int64, error) { + core.CallBuildIndexService = func(_ context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idx *etcdpb.IndexInfo, numRows int64) (int64, error) { assert.Equal(t, fieldID, field.FieldID) assert.Equal(t, indexID, idx.IndexID) return -1, errors.New("build index build") @@ -3151,7 +3188,7 @@ func TestCheckFlushedSegments(t *testing.T) { core.checkFlushedSegments(ctx) var indexBuildID int64 = 10001 - core.CallBuildIndexService = func(_ context.Context, binlog []string, field *schemapb.FieldSchema, idx *etcdpb.IndexInfo, numRows int64) (int64, error) { + core.CallBuildIndexService = func(_ context.Context, segID UniqueID, binlog []string, field *schemapb.FieldSchema, idx *etcdpb.IndexInfo, numRows int64) (int64, error) { return indexBuildID, nil } core.checkFlushedSegments(core.ctx) diff --git a/internal/rootcoord/task.go b/internal/rootcoord/task.go index 56d12d4246..c5fbd201c7 100644 --- a/internal/rootcoord/task.go +++ b/internal/rootcoord/task.go @@ -959,7 +959,7 @@ func (t *CreateIndexReqTask) Execute(ctx context.Context) error { if err != nil { return err } - segID2PartID, err := t.core.getSegments(ctx, collMeta.ID) + segID2PartID, segID2Binlog, err := t.core.getSegments(ctx, collMeta.ID) flushedSegs := make([]typeutil.UniqueID, 0, len(segID2PartID)) for k := range segID2PartID { flushedSegs = append(flushedSegs, k) @@ -987,7 +987,7 @@ func (t *CreateIndexReqTask) Execute(ctx context.Context) error { IndexID: idxInfo.IndexID, EnableIndex: false, } - info.BuildID, err = t.core.BuildIndex(ctx, segID, &field, idxInfo, false) + info.BuildID, err = t.core.BuildIndex(ctx, segID, segID2Binlog[segID].GetNumOfRows(), segID2Binlog[segID].GetFieldBinlogs(), &field, idxInfo, false) if err != nil { return err } diff --git a/internal/types/types.go b/internal/types/types.go index e878318e28..53ef200a58 100644 --- a/internal/types/types.go +++ b/internal/types/types.go @@ -281,6 +281,9 @@ type DataCoord interface { // UpdateSegmentStatistics updates a segment's stats. UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) (*commonpb.Status, error) + + AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest) (*commonpb.Status, error) + ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest) (*commonpb.Status, error) } // DataCoordComponent defines the interface of DataCoord component. @@ -350,6 +353,8 @@ type IndexCoordComponent interface { // SetEtcdClient set etcd client for IndexCoordComponent SetEtcdClient(etcdClient *clientv3.Client) + SetDataCoord(dataCoor DataCoord) error + // UpdateStateCode updates state code for IndexCoordComponent // `stateCode` is current statement of this IndexCoordComponent, indicating whether it's healthy. UpdateStateCode(stateCode internalpb.StateCode) diff --git a/internal/util/mock/datacoord_client.go b/internal/util/mock/datacoord_client.go new file mode 100644 index 0000000000..5b8c69c747 --- /dev/null +++ b/internal/util/mock/datacoord_client.go @@ -0,0 +1,140 @@ +// 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 mock + +import ( + "context" + + "google.golang.org/grpc" + + "github.com/milvus-io/milvus/internal/proto/commonpb" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/internalpb" + "github.com/milvus-io/milvus/internal/proto/milvuspb" +) + +// DataCoordClient mocks of DataCoordClient +type DataCoordClient struct { + Err error +} + +func (m *DataCoordClient) GetComponentStates(ctx context.Context, in *internalpb.GetComponentStatesRequest, opts ...grpc.CallOption) (*internalpb.ComponentStates, error) { + return &internalpb.ComponentStates{}, m.Err +} + +func (m *DataCoordClient) GetTimeTickChannel(ctx context.Context, in *internalpb.GetTimeTickChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + return &milvuspb.StringResponse{}, m.Err +} + +func (m *DataCoordClient) GetStatisticsChannel(ctx context.Context, in *internalpb.GetStatisticsChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + return &milvuspb.StringResponse{}, m.Err +} + +func (m *DataCoordClient) Flush(ctx context.Context, in *datapb.FlushRequest, opts ...grpc.CallOption) (*datapb.FlushResponse, error) { + return &datapb.FlushResponse{}, m.Err +} + +func (m *DataCoordClient) AssignSegmentID(ctx context.Context, in *datapb.AssignSegmentIDRequest, opts ...grpc.CallOption) (*datapb.AssignSegmentIDResponse, error) { + return &datapb.AssignSegmentIDResponse{}, m.Err +} + +func (m *DataCoordClient) GetSegmentInfo(ctx context.Context, in *datapb.GetSegmentInfoRequest, opts ...grpc.CallOption) (*datapb.GetSegmentInfoResponse, error) { + return &datapb.GetSegmentInfoResponse{}, m.Err +} + +func (m *DataCoordClient) GetSegmentStates(ctx context.Context, in *datapb.GetSegmentStatesRequest, opts ...grpc.CallOption) (*datapb.GetSegmentStatesResponse, error) { + return &datapb.GetSegmentStatesResponse{}, m.Err +} + +func (m *DataCoordClient) GetInsertBinlogPaths(ctx context.Context, in *datapb.GetInsertBinlogPathsRequest, opts ...grpc.CallOption) (*datapb.GetInsertBinlogPathsResponse, error) { + return &datapb.GetInsertBinlogPathsResponse{}, m.Err +} + +func (m *DataCoordClient) GetCollectionStatistics(ctx context.Context, in *datapb.GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetCollectionStatisticsResponse, error) { + return &datapb.GetCollectionStatisticsResponse{}, m.Err +} + +func (m *DataCoordClient) GetPartitionStatistics(ctx context.Context, in *datapb.GetPartitionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetPartitionStatisticsResponse, error) { + return &datapb.GetPartitionStatisticsResponse{}, m.Err +} + +func (m *DataCoordClient) GetSegmentInfoChannel(ctx context.Context, in *datapb.GetSegmentInfoChannelRequest, opts ...grpc.CallOption) (*milvuspb.StringResponse, error) { + return &milvuspb.StringResponse{}, m.Err +} + +func (m *DataCoordClient) SaveBinlogPaths(ctx context.Context, in *datapb.SaveBinlogPathsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} + +func (m *DataCoordClient) GetRecoveryInfo(ctx context.Context, in *datapb.GetRecoveryInfoRequest, opts ...grpc.CallOption) (*datapb.GetRecoveryInfoResponse, error) { + return &datapb.GetRecoveryInfoResponse{}, m.Err +} + +func (m *DataCoordClient) GetFlushedSegments(ctx context.Context, in *datapb.GetFlushedSegmentsRequest, opts ...grpc.CallOption) (*datapb.GetFlushedSegmentsResponse, error) { + return &datapb.GetFlushedSegmentsResponse{}, m.Err +} + +func (m *DataCoordClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { + return &milvuspb.GetMetricsResponse{}, m.Err +} + +func (m *DataCoordClient) CompleteCompaction(ctx context.Context, req *datapb.CompactionResult, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} + +func (m *DataCoordClient) ManualCompaction(ctx context.Context, in *milvuspb.ManualCompactionRequest, opts ...grpc.CallOption) (*milvuspb.ManualCompactionResponse, error) { + return &milvuspb.ManualCompactionResponse{}, m.Err +} + +func (m *DataCoordClient) GetCompactionState(ctx context.Context, in *milvuspb.GetCompactionStateRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionStateResponse, error) { + return &milvuspb.GetCompactionStateResponse{}, m.Err +} + +func (m *DataCoordClient) GetCompactionStateWithPlans(ctx context.Context, req *milvuspb.GetCompactionPlansRequest, opts ...grpc.CallOption) (*milvuspb.GetCompactionPlansResponse, error) { + return &milvuspb.GetCompactionPlansResponse{}, m.Err +} + +func (m *DataCoordClient) WatchChannels(ctx context.Context, req *datapb.WatchChannelsRequest, opts ...grpc.CallOption) (*datapb.WatchChannelsResponse, error) { + return &datapb.WatchChannelsResponse{}, m.Err +} +func (m *DataCoordClient) GetFlushState(ctx context.Context, req *milvuspb.GetFlushStateRequest, opts ...grpc.CallOption) (*milvuspb.GetFlushStateResponse, error) { + return &milvuspb.GetFlushStateResponse{}, m.Err +} + +func (m *DataCoordClient) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest, opts ...grpc.CallOption) (*datapb.DropVirtualChannelResponse, error) { + return &datapb.DropVirtualChannelResponse{}, m.Err +} + +func (m *DataCoordClient) SetSegmentState(ctx context.Context, req *datapb.SetSegmentStateRequest, opts ...grpc.CallOption) (*datapb.SetSegmentStateResponse, error) { + return &datapb.SetSegmentStateResponse{}, m.Err +} + +func (m *DataCoordClient) Import(ctx context.Context, req *datapb.ImportTaskRequest, opts ...grpc.CallOption) (*datapb.ImportTaskResponse, error) { + return &datapb.ImportTaskResponse{}, m.Err +} + +func (m *DataCoordClient) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} + +func (m *DataCoordClient) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} + +func (m *DataCoordClient) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} diff --git a/internal/util/mock/grpc_datacoord_client.go b/internal/util/mock/grpc_datacoord_client.go index 99e59a1b38..d0585c1315 100644 --- a/internal/util/mock/grpc_datacoord_client.go +++ b/internal/util/mock/grpc_datacoord_client.go @@ -132,3 +132,11 @@ func (m *GrpcDataCoordClient) Import(ctx context.Context, req *datapb.ImportTask func (m *GrpcDataCoordClient) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { return &commonpb.Status{}, m.Err } + +func (m *GrpcDataCoordClient) AcquireSegmentLock(ctx context.Context, req *datapb.AcquireSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} + +func (m *GrpcDataCoordClient) ReleaseSegmentLock(ctx context.Context, req *datapb.ReleaseSegmentLockRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +}