2021-04-19 11:12:56 +08:00
|
|
|
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
|
|
|
//
|
|
|
|
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
//
|
|
|
|
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
|
|
|
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
|
|
|
// or implied. See the License for the specific language governing permissions and limitations under the License.
|
|
|
|
|
2021-01-19 14:44:03 +08:00
|
|
|
package masterservice
|
|
|
|
|
|
|
|
import (
|
2021-05-12 15:33:53 +08:00
|
|
|
"encoding/json"
|
2021-03-05 10:15:27 +08:00
|
|
|
"fmt"
|
2021-01-20 09:36:50 +08:00
|
|
|
"path"
|
2021-01-19 14:44:03 +08:00
|
|
|
"strconv"
|
|
|
|
"sync"
|
|
|
|
|
|
|
|
"github.com/golang/protobuf/proto"
|
2021-02-27 10:11:52 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2021-03-05 10:15:27 +08:00
|
|
|
"errors"
|
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/kv"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
|
|
|
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
|
|
|
pb "github.com/milvus-io/milvus/internal/proto/etcdpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
2021-01-19 14:44:03 +08:00
|
|
|
)
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
const (
|
2021-02-09 10:36:31 +08:00
|
|
|
ComponentPrefix = "master-service"
|
|
|
|
TenantMetaPrefix = ComponentPrefix + "/tenant"
|
|
|
|
ProxyMetaPrefix = ComponentPrefix + "/proxy"
|
|
|
|
CollectionMetaPrefix = ComponentPrefix + "/collection"
|
|
|
|
PartitionMetaPrefix = ComponentPrefix + "/partition"
|
|
|
|
SegmentIndexMetaPrefix = ComponentPrefix + "/segment-index"
|
|
|
|
IndexMetaPrefix = ComponentPrefix + "/index"
|
2021-05-12 15:33:53 +08:00
|
|
|
|
|
|
|
DDMsgPrefix = ComponentPrefix + "/dd-msg"
|
|
|
|
DDMsgTypePrefix = ComponentPrefix + "/dd-msg-type"
|
|
|
|
DDMsgFlagPrefix = ComponentPrefix + "/dd-msg-flag"
|
|
|
|
|
|
|
|
CreateCollectionMsgType = "CreateCollection"
|
|
|
|
DropCollectionMsgType = "DropCollection"
|
|
|
|
CreatePartitionMsgType = "CreatePartition"
|
|
|
|
DropPartitionMsgType = "DropPartition"
|
2021-01-20 09:36:50 +08:00
|
|
|
)
|
|
|
|
|
2021-01-19 14:44:03 +08:00
|
|
|
type metaTable struct {
|
2021-04-12 18:09:28 +08:00
|
|
|
client kv.TxnKV // client of a reliable kv service, i.e. etcd client
|
2021-01-21 10:01:29 +08:00
|
|
|
tenantID2Meta map[typeutil.UniqueID]pb.TenantMeta // tenant id to tenant meta
|
|
|
|
proxyID2Meta map[typeutil.UniqueID]pb.ProxyMeta // proxy id to proxy meta
|
2021-03-06 16:00:41 +08:00
|
|
|
collID2Meta map[typeutil.UniqueID]pb.CollectionInfo // collection_id -> meta
|
2021-01-21 10:01:29 +08:00
|
|
|
collName2ID map[string]typeutil.UniqueID // collection name to collection id
|
2021-03-06 16:00:41 +08:00
|
|
|
partitionID2Meta map[typeutil.UniqueID]pb.PartitionInfo // collection_id/partition_id -> meta
|
|
|
|
segID2IndexMeta map[typeutil.UniqueID]*map[typeutil.UniqueID]pb.SegmentIndexInfo // collection_id/index_id/partition_id/segment_id -> meta
|
|
|
|
indexID2Meta map[typeutil.UniqueID]pb.IndexInfo // collection_id/index_id -> meta
|
2021-01-21 10:01:29 +08:00
|
|
|
segID2CollID map[typeutil.UniqueID]typeutil.UniqueID // segment id -> collection id
|
2021-03-06 16:00:41 +08:00
|
|
|
segID2PartitionID map[typeutil.UniqueID]typeutil.UniqueID // segment id -> partition id
|
2021-03-10 14:45:35 +08:00
|
|
|
flushedSegID map[typeutil.UniqueID]bool // flushed segment id
|
2021-01-21 10:01:29 +08:00
|
|
|
partitionID2CollID map[typeutil.UniqueID]typeutil.UniqueID // partition id -> collection id
|
2021-01-19 14:44:03 +08:00
|
|
|
|
|
|
|
tenantLock sync.RWMutex
|
|
|
|
proxyLock sync.RWMutex
|
|
|
|
ddLock sync.RWMutex
|
|
|
|
}
|
|
|
|
|
2021-04-12 18:09:28 +08:00
|
|
|
func NewMetaTable(kv kv.TxnKV) (*metaTable, error) {
|
2021-01-19 14:44:03 +08:00
|
|
|
mt := &metaTable{
|
|
|
|
client: kv,
|
|
|
|
tenantLock: sync.RWMutex{},
|
|
|
|
proxyLock: sync.RWMutex{},
|
|
|
|
ddLock: sync.RWMutex{},
|
|
|
|
}
|
|
|
|
err := mt.reloadFromKV()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return mt, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) reloadFromKV() error {
|
|
|
|
|
|
|
|
mt.tenantID2Meta = make(map[typeutil.UniqueID]pb.TenantMeta)
|
|
|
|
mt.proxyID2Meta = make(map[typeutil.UniqueID]pb.ProxyMeta)
|
2021-01-20 09:36:50 +08:00
|
|
|
mt.collID2Meta = make(map[typeutil.UniqueID]pb.CollectionInfo)
|
2021-01-19 14:44:03 +08:00
|
|
|
mt.collName2ID = make(map[string]typeutil.UniqueID)
|
2021-01-20 09:36:50 +08:00
|
|
|
mt.partitionID2Meta = make(map[typeutil.UniqueID]pb.PartitionInfo)
|
|
|
|
mt.segID2IndexMeta = make(map[typeutil.UniqueID]*map[typeutil.UniqueID]pb.SegmentIndexInfo)
|
|
|
|
mt.indexID2Meta = make(map[typeutil.UniqueID]pb.IndexInfo)
|
2021-01-21 10:01:29 +08:00
|
|
|
mt.partitionID2CollID = make(map[typeutil.UniqueID]typeutil.UniqueID)
|
|
|
|
mt.segID2CollID = make(map[typeutil.UniqueID]typeutil.UniqueID)
|
2021-03-06 16:00:41 +08:00
|
|
|
mt.segID2PartitionID = make(map[typeutil.UniqueID]typeutil.UniqueID)
|
2021-03-10 14:45:35 +08:00
|
|
|
mt.flushedSegID = make(map[typeutil.UniqueID]bool)
|
2021-01-19 14:44:03 +08:00
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
_, values, err := mt.client.LoadWithPrefix(TenantMetaPrefix)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, value := range values {
|
|
|
|
tenantMeta := pb.TenantMeta{}
|
|
|
|
err := proto.UnmarshalText(value, &tenantMeta)
|
|
|
|
if err != nil {
|
2021-03-30 20:19:30 +08:00
|
|
|
return fmt.Errorf("MasterService UnmarshalText pb.TenantMeta err:%w", err)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
mt.tenantID2Meta[tenantMeta.ID] = tenantMeta
|
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
_, values, err = mt.client.LoadWithPrefix(ProxyMetaPrefix)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, value := range values {
|
|
|
|
proxyMeta := pb.ProxyMeta{}
|
|
|
|
err = proto.UnmarshalText(value, &proxyMeta)
|
|
|
|
if err != nil {
|
2021-03-30 20:19:30 +08:00
|
|
|
return fmt.Errorf("MasterService UnmarshalText pb.ProxyMeta err:%w", err)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
mt.proxyID2Meta[proxyMeta.ID] = proxyMeta
|
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
_, values, err = mt.client.LoadWithPrefix(CollectionMetaPrefix)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, value := range values {
|
2021-01-20 09:36:50 +08:00
|
|
|
collectionInfo := pb.CollectionInfo{}
|
|
|
|
err = proto.UnmarshalText(value, &collectionInfo)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
2021-03-30 20:19:30 +08:00
|
|
|
return fmt.Errorf("MasterService UnmarshalText pb.CollectionInfo err:%w", err)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
mt.collID2Meta[collectionInfo.ID] = collectionInfo
|
|
|
|
mt.collName2ID[collectionInfo.Schema.Name] = collectionInfo.ID
|
2021-01-21 10:01:29 +08:00
|
|
|
for _, partID := range collectionInfo.PartitionIDs {
|
|
|
|
mt.partitionID2CollID[partID] = collectionInfo.ID
|
|
|
|
}
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
_, values, err = mt.client.LoadWithPrefix(PartitionMetaPrefix)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
for _, value := range values {
|
2021-01-20 09:36:50 +08:00
|
|
|
partitionInfo := pb.PartitionInfo{}
|
|
|
|
err = proto.UnmarshalText(value, &partitionInfo)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
2021-04-10 10:53:58 +08:00
|
|
|
return fmt.Errorf("MasterService UnmarshalText pb.PartitionInfo err:%w", err)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-21 10:01:29 +08:00
|
|
|
collID, ok := mt.partitionID2CollID[partitionInfo.PartitionID]
|
|
|
|
if !ok {
|
2021-02-24 16:25:40 +08:00
|
|
|
log.Warn("partition does not belong to any collection", zap.Int64("partition id", partitionInfo.PartitionID))
|
2021-01-21 10:01:29 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
mt.partitionID2Meta[partitionInfo.PartitionID] = partitionInfo
|
2021-01-21 10:01:29 +08:00
|
|
|
for _, segID := range partitionInfo.SegmentIDs {
|
|
|
|
mt.segID2CollID[segID] = collID
|
2021-03-06 16:00:41 +08:00
|
|
|
mt.segID2PartitionID[segID] = partitionInfo.PartitionID
|
2021-03-10 14:45:35 +08:00
|
|
|
mt.flushedSegID[segID] = true
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
_, values, err = mt.client.LoadWithPrefix(SegmentIndexMetaPrefix)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
for _, value := range values {
|
|
|
|
segmentIndexInfo := pb.SegmentIndexInfo{}
|
|
|
|
err = proto.UnmarshalText(value, &segmentIndexInfo)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
2021-03-30 20:19:30 +08:00
|
|
|
return fmt.Errorf("MasterService UnmarshalText pb.SegmentIndexInfo err:%w", err)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
idx, ok := mt.segID2IndexMeta[segmentIndexInfo.SegmentID]
|
2021-01-19 14:44:03 +08:00
|
|
|
if ok {
|
2021-01-20 09:36:50 +08:00
|
|
|
(*idx)[segmentIndexInfo.IndexID] = segmentIndexInfo
|
|
|
|
} else {
|
|
|
|
meta := make(map[typeutil.UniqueID]pb.SegmentIndexInfo)
|
|
|
|
meta[segmentIndexInfo.IndexID] = segmentIndexInfo
|
|
|
|
mt.segID2IndexMeta[segmentIndexInfo.SegmentID] = &meta
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
_, values, err = mt.client.LoadWithPrefix(IndexMetaPrefix)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
2021-01-19 18:32:57 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
for _, value := range values {
|
|
|
|
meta := pb.IndexInfo{}
|
|
|
|
err = proto.UnmarshalText(value, &meta)
|
|
|
|
if err != nil {
|
2021-03-30 20:19:30 +08:00
|
|
|
return fmt.Errorf("MasterService UnmarshalText pb.IndexInfo err:%w", err)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
mt.indexID2Meta[meta.IndexID] = meta
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
return nil
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
2021-04-08 17:31:39 +08:00
|
|
|
func (mt *metaTable) AddTenant(te *pb.TenantMeta) error {
|
|
|
|
mt.tenantLock.Lock()
|
|
|
|
defer mt.tenantLock.Unlock()
|
|
|
|
|
|
|
|
k := fmt.Sprintf("%s/%d", TenantMetaPrefix, te.ID)
|
|
|
|
v := proto.MarshalTextString(te)
|
|
|
|
|
|
|
|
if err := mt.client.Save(k, v); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
mt.tenantID2Meta[te.ID] = *te
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) AddProxy(po *pb.ProxyMeta) error {
|
|
|
|
mt.proxyLock.Lock()
|
|
|
|
defer mt.proxyLock.Unlock()
|
|
|
|
|
|
|
|
k := fmt.Sprintf("%s/%d", ProxyMetaPrefix, po.ID)
|
|
|
|
v := proto.MarshalTextString(po)
|
|
|
|
|
|
|
|
if err := mt.client.Save(k, v); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
mt.proxyID2Meta[po.ID] = *po
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
func (mt *metaTable) AddCollection(coll *pb.CollectionInfo, part *pb.PartitionInfo, idx []*pb.IndexInfo) error {
|
2021-01-19 14:44:03 +08:00
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
2021-01-20 09:36:50 +08:00
|
|
|
|
|
|
|
if len(part.SegmentIDs) != 0 {
|
2021-03-05 10:15:27 +08:00
|
|
|
return errors.New("segment should be empty when creating collection")
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
if len(coll.PartitionIDs) != 0 {
|
2021-03-05 10:15:27 +08:00
|
|
|
return errors.New("partitions should be empty when creating collection")
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-22 15:41:54 +08:00
|
|
|
if _, ok := mt.collName2ID[coll.Schema.Name]; ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("collection %s exist", coll.Schema.Name)
|
2021-01-22 15:41:54 +08:00
|
|
|
}
|
2021-02-11 08:41:59 +08:00
|
|
|
if len(coll.FieldIndexes) != len(idx) {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("incorrect index id when creating collection")
|
2021-02-11 08:41:59 +08:00
|
|
|
}
|
2021-01-22 15:41:54 +08:00
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
coll.PartitionIDs = append(coll.PartitionIDs, part.PartitionID)
|
|
|
|
mt.collID2Meta[coll.ID] = *coll
|
|
|
|
mt.collName2ID[coll.Schema.Name] = coll.ID
|
|
|
|
mt.partitionID2Meta[part.PartitionID] = *part
|
2021-01-21 10:01:29 +08:00
|
|
|
mt.partitionID2CollID[part.PartitionID] = coll.ID
|
2021-02-11 08:41:59 +08:00
|
|
|
for _, i := range idx {
|
|
|
|
mt.indexID2Meta[i.IndexID] = *i
|
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
|
2021-03-06 16:00:41 +08:00
|
|
|
k1 := fmt.Sprintf("%s/%d", CollectionMetaPrefix, coll.ID)
|
2021-01-20 09:36:50 +08:00
|
|
|
v1 := proto.MarshalTextString(coll)
|
2021-03-06 16:00:41 +08:00
|
|
|
k2 := fmt.Sprintf("%s/%d/%d", PartitionMetaPrefix, coll.ID, part.PartitionID)
|
2021-01-20 09:36:50 +08:00
|
|
|
v2 := proto.MarshalTextString(part)
|
|
|
|
meta := map[string]string{k1: v1, k2: v2}
|
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
for _, i := range idx {
|
2021-03-06 16:00:41 +08:00
|
|
|
k := fmt.Sprintf("%s/%d/%d", IndexMetaPrefix, coll.ID, i.IndexID)
|
2021-02-11 08:41:59 +08:00
|
|
|
v := proto.MarshalTextString(i)
|
|
|
|
meta[k] = v
|
|
|
|
}
|
|
|
|
|
2021-05-12 15:33:53 +08:00
|
|
|
// record ddmsg info and type
|
|
|
|
ddmsg, err := json.Marshal(meta)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
meta[DDMsgPrefix] = string(ddmsg)
|
|
|
|
meta[DDMsgTypePrefix] = CreateCollectionMsgType
|
|
|
|
meta[DDMsgFlagPrefix] = "false"
|
|
|
|
|
|
|
|
err = mt.client.MultiSave(meta)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
|
|
|
return err
|
|
|
|
}
|
2021-05-12 15:33:53 +08:00
|
|
|
|
2021-01-19 14:44:03 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) DeleteCollection(collID typeutil.UniqueID) error {
|
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
|
|
|
|
|
|
|
collMeta, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("can't find collection. id = %d", collID)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
delete(mt.collID2Meta, collID)
|
|
|
|
delete(mt.collName2ID, collMeta.Schema.Name)
|
|
|
|
for _, partID := range collMeta.PartitionIDs {
|
|
|
|
partMeta, ok := mt.partitionID2Meta[partID]
|
|
|
|
if !ok {
|
2021-02-24 16:25:40 +08:00
|
|
|
log.Warn("partition id not exist", zap.Int64("partition id", partID))
|
2021-01-20 09:36:50 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
delete(mt.partitionID2Meta, partID)
|
|
|
|
for _, segID := range partMeta.SegmentIDs {
|
2021-03-10 14:45:35 +08:00
|
|
|
delete(mt.segID2CollID, segID)
|
|
|
|
delete(mt.segID2PartitionID, segID)
|
|
|
|
delete(mt.flushedSegID, segID)
|
2021-03-06 16:00:41 +08:00
|
|
|
_, ok := mt.segID2IndexMeta[segID]
|
2021-01-20 09:36:50 +08:00
|
|
|
if !ok {
|
2021-02-24 16:25:40 +08:00
|
|
|
log.Warn("segment id not exist", zap.Int64("segment id", segID))
|
2021-01-20 09:36:50 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
delete(mt.segID2IndexMeta, segID)
|
|
|
|
}
|
|
|
|
}
|
2021-03-06 16:00:41 +08:00
|
|
|
for _, idxInfo := range collMeta.FieldIndexes {
|
|
|
|
_, ok := mt.indexID2Meta[idxInfo.IndexID]
|
|
|
|
if !ok {
|
|
|
|
log.Warn("index id not exist", zap.Int64("index id", idxInfo.IndexID))
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
delete(mt.indexID2Meta, idxInfo.IndexID)
|
|
|
|
}
|
2021-05-12 15:33:53 +08:00
|
|
|
delMetakeys := []string{
|
2021-03-06 16:00:41 +08:00
|
|
|
fmt.Sprintf("%s/%d", CollectionMetaPrefix, collID),
|
|
|
|
fmt.Sprintf("%s/%d", PartitionMetaPrefix, collID),
|
|
|
|
fmt.Sprintf("%s/%d", SegmentIndexMetaPrefix, collID),
|
|
|
|
fmt.Sprintf("%s/%d", IndexMetaPrefix, collID),
|
|
|
|
}
|
2021-05-12 15:33:53 +08:00
|
|
|
|
|
|
|
// record ddmsg info and type
|
|
|
|
ddmsg, err := json.Marshal(collID)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
saveMeta := map[string]string{
|
|
|
|
DDMsgPrefix: string(ddmsg),
|
|
|
|
DDMsgTypePrefix: DropCollectionMsgType,
|
|
|
|
DDMsgFlagPrefix: "false",
|
|
|
|
}
|
|
|
|
|
|
|
|
err = mt.client.MultiSaveAndRemoveWithPrefix(saveMeta, delMetakeys)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) HasCollection(collID typeutil.UniqueID) bool {
|
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
|
|
|
_, ok := mt.collID2Meta[collID]
|
|
|
|
return ok
|
|
|
|
}
|
|
|
|
|
2021-02-02 10:09:10 +08:00
|
|
|
func (mt *metaTable) GetCollectionByID(collectionID typeutil.UniqueID) (*pb.CollectionInfo, error) {
|
2021-01-21 10:01:29 +08:00
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
|
|
|
|
|
|
|
col, ok := mt.collID2Meta[collectionID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return nil, fmt.Errorf("can't find collection id : %d", collectionID)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
2021-02-02 10:09:10 +08:00
|
|
|
colCopy := proto.Clone(&col)
|
|
|
|
|
|
|
|
return colCopy.(*pb.CollectionInfo), nil
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
|
2021-02-02 10:09:10 +08:00
|
|
|
func (mt *metaTable) GetCollectionByName(collectionName string) (*pb.CollectionInfo, error) {
|
2021-01-19 14:44:03 +08:00
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
|
|
|
|
|
|
|
vid, ok := mt.collName2ID[collectionName]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return nil, fmt.Errorf("can't find collection: " + collectionName)
|
2021-02-02 10:09:10 +08:00
|
|
|
}
|
|
|
|
col, ok := mt.collID2Meta[vid]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return nil, fmt.Errorf("can't find collection: " + collectionName)
|
2021-02-02 10:09:10 +08:00
|
|
|
}
|
|
|
|
colCopy := proto.Clone(&col)
|
|
|
|
return colCopy.(*pb.CollectionInfo), nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) GetCollectionBySegmentID(segID typeutil.UniqueID) (*pb.CollectionInfo, error) {
|
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
|
|
|
|
|
|
|
vid, ok := mt.segID2CollID[segID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return nil, fmt.Errorf("segment id %d not belong to any collection", segID)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
col, ok := mt.collID2Meta[vid]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return nil, fmt.Errorf("can't find collection id: %d", vid)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-02-02 10:09:10 +08:00
|
|
|
colCopy := proto.Clone(&col)
|
|
|
|
return colCopy.(*pb.CollectionInfo), nil
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) ListCollections() ([]string, error) {
|
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
|
|
|
|
|
|
|
colls := make([]string, 0, len(mt.collName2ID))
|
|
|
|
for name := range mt.collName2ID {
|
|
|
|
colls = append(colls, name)
|
|
|
|
}
|
|
|
|
return colls, nil
|
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
func (mt *metaTable) AddPartition(collID typeutil.UniqueID, partitionName string, partitionID typeutil.UniqueID) error {
|
2021-01-19 14:44:03 +08:00
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
|
|
|
coll, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("can't find collection. id = %d", collID)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// number of partition tags (except _default) should be limited to 4096 by default
|
2021-04-10 10:53:58 +08:00
|
|
|
if int64(len(coll.PartitionIDs)) >= Params.MaxPartitionNum {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("maximum partition's number should be limit to %d", Params.MaxPartitionNum)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
for _, t := range coll.PartitionIDs {
|
|
|
|
part, ok := mt.partitionID2Meta[t]
|
|
|
|
if !ok {
|
2021-02-24 16:25:40 +08:00
|
|
|
log.Warn("partition id not exist", zap.Int64("partition id", t))
|
2021-01-20 09:36:50 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
if part.PartitionName == partitionName {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("partition name = %s already exists", partitionName)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
if part.PartitionID == partitionID {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("partition id = %d already exists", partitionID)
|
2021-01-20 09:36:50 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
partMeta := pb.PartitionInfo{
|
|
|
|
PartitionName: partitionName,
|
|
|
|
PartitionID: partitionID,
|
|
|
|
SegmentIDs: make([]typeutil.UniqueID, 0, 16),
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
coll.PartitionIDs = append(coll.PartitionIDs, partitionID)
|
2021-01-20 09:36:50 +08:00
|
|
|
mt.partitionID2Meta[partitionID] = partMeta
|
|
|
|
mt.collID2Meta[collID] = coll
|
2021-01-21 10:01:29 +08:00
|
|
|
mt.partitionID2CollID[partitionID] = collID
|
2021-01-20 09:36:50 +08:00
|
|
|
|
2021-03-06 16:00:41 +08:00
|
|
|
k1 := fmt.Sprintf("%s/%d", CollectionMetaPrefix, collID)
|
2021-01-20 09:36:50 +08:00
|
|
|
v1 := proto.MarshalTextString(&coll)
|
2021-03-06 16:00:41 +08:00
|
|
|
k2 := fmt.Sprintf("%s/%d/%d", PartitionMetaPrefix, collID, partitionID)
|
2021-01-20 09:36:50 +08:00
|
|
|
v2 := proto.MarshalTextString(&partMeta)
|
|
|
|
meta := map[string]string{k1: v1, k2: v2}
|
|
|
|
|
2021-05-12 15:33:53 +08:00
|
|
|
// record ddmsg info and type
|
|
|
|
ddmsg, err := json.Marshal(meta)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
meta[DDMsgPrefix] = string(ddmsg)
|
|
|
|
meta[DDMsgTypePrefix] = CreatePartitionMsgType
|
|
|
|
meta[DDMsgFlagPrefix] = "false"
|
2021-01-19 14:44:03 +08:00
|
|
|
|
2021-05-12 15:33:53 +08:00
|
|
|
err = mt.client.MultiSave(meta)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
func (mt *metaTable) HasPartition(collID typeutil.UniqueID, partitionName string) bool {
|
2021-01-19 14:44:03 +08:00
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
|
|
|
col, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
|
|
|
return false
|
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
for _, partitionID := range col.PartitionIDs {
|
|
|
|
meta, ok := mt.partitionID2Meta[partitionID]
|
|
|
|
if ok {
|
|
|
|
if meta.PartitionName == partitionName {
|
|
|
|
return true
|
|
|
|
}
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
func (mt *metaTable) DeletePartition(collID typeutil.UniqueID, partitionName string) (typeutil.UniqueID, error) {
|
2021-01-19 14:44:03 +08:00
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
if partitionName == Params.DefaultPartitionName {
|
2021-04-12 15:03:23 +08:00
|
|
|
return 0, fmt.Errorf("default partition cannot be deleted")
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
collMeta, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return 0, fmt.Errorf("can't find collection id = %d", collID)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// check tag exists
|
|
|
|
exist := false
|
|
|
|
|
|
|
|
pd := make([]typeutil.UniqueID, 0, len(collMeta.PartitionIDs))
|
2021-01-20 09:36:50 +08:00
|
|
|
var partMeta pb.PartitionInfo
|
|
|
|
for _, t := range collMeta.PartitionIDs {
|
|
|
|
pm, ok := mt.partitionID2Meta[t]
|
|
|
|
if ok {
|
|
|
|
if pm.PartitionName != partitionName {
|
|
|
|
pd = append(pd, pm.PartitionID)
|
|
|
|
} else {
|
|
|
|
partMeta = pm
|
|
|
|
exist = true
|
|
|
|
}
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
if !exist {
|
2021-03-05 10:15:27 +08:00
|
|
|
return 0, fmt.Errorf("partition %s does not exist", partitionName)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
delete(mt.partitionID2Meta, partMeta.PartitionID)
|
|
|
|
collMeta.PartitionIDs = pd
|
|
|
|
mt.collID2Meta[collID] = collMeta
|
2021-01-19 14:44:03 +08:00
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
for _, segID := range partMeta.SegmentIDs {
|
2021-03-10 14:45:35 +08:00
|
|
|
delete(mt.segID2CollID, segID)
|
|
|
|
delete(mt.segID2PartitionID, segID)
|
|
|
|
delete(mt.flushedSegID, segID)
|
|
|
|
|
2021-03-06 16:00:41 +08:00
|
|
|
_, ok := mt.segID2IndexMeta[segID]
|
2021-01-19 14:44:03 +08:00
|
|
|
if !ok {
|
2021-02-24 16:25:40 +08:00
|
|
|
log.Warn("segment has no index meta", zap.Int64("segment id", segID))
|
2021-01-20 09:36:50 +08:00
|
|
|
continue
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
delete(mt.segID2IndexMeta, segID)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
collKV := map[string]string{path.Join(CollectionMetaPrefix, strconv.FormatInt(collID, 10)): proto.MarshalTextString(&collMeta)}
|
2021-03-06 16:00:41 +08:00
|
|
|
delMetaKeys := []string{
|
|
|
|
fmt.Sprintf("%s/%d/%d", PartitionMetaPrefix, collMeta.ID, partMeta.PartitionID),
|
|
|
|
}
|
|
|
|
for _, idxInfo := range collMeta.FieldIndexes {
|
|
|
|
k := fmt.Sprintf("%s/%d/%d/%d", SegmentIndexMetaPrefix, collMeta.ID, idxInfo.IndexID, partMeta.PartitionID)
|
|
|
|
delMetaKeys = append(delMetaKeys, k)
|
|
|
|
}
|
|
|
|
|
2021-05-12 15:33:53 +08:00
|
|
|
// record ddmsg info and type
|
|
|
|
ddmsg, err := json.Marshal(collKV)
|
|
|
|
if err != nil {
|
|
|
|
return 0, err
|
|
|
|
}
|
|
|
|
collKV[DDMsgPrefix] = string(ddmsg)
|
|
|
|
collKV[DDMsgTypePrefix] = DropPartitionMsgType
|
|
|
|
collKV[DDMsgFlagPrefix] = "false"
|
2021-01-19 14:44:03 +08:00
|
|
|
|
2021-05-12 15:33:53 +08:00
|
|
|
err = mt.client.MultiSaveAndRemoveWithPrefix(collKV, delMetaKeys)
|
2021-01-19 14:44:03 +08:00
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
2021-01-20 09:36:50 +08:00
|
|
|
return 0, err
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
return partMeta.PartitionID, nil
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
func (mt *metaTable) GetPartitionByID(partitionID typeutil.UniqueID) (pb.PartitionInfo, error) {
|
2021-01-19 14:44:03 +08:00
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
2021-01-20 09:36:50 +08:00
|
|
|
partMeta, ok := mt.partitionID2Meta[partitionID]
|
2021-01-19 14:44:03 +08:00
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return pb.PartitionInfo{}, fmt.Errorf("partition id = %d not exist", partitionID)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
return partMeta, nil
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
|
|
|
|
2021-01-20 09:36:50 +08:00
|
|
|
func (mt *metaTable) AddSegment(seg *datapb.SegmentInfo) error {
|
2021-01-19 14:44:03 +08:00
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
2021-01-20 09:36:50 +08:00
|
|
|
collMeta, ok := mt.collID2Meta[seg.CollectionID]
|
2021-01-19 14:44:03 +08:00
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("can't find collection id = %d", seg.CollectionID)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
partMeta, ok := mt.partitionID2Meta[seg.PartitionID]
|
2021-01-19 14:44:03 +08:00
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("can't find partition id = %d", seg.PartitionID)
|
2021-01-19 14:44:03 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
exist := false
|
|
|
|
for _, partID := range collMeta.PartitionIDs {
|
|
|
|
if partID == seg.PartitionID {
|
|
|
|
exist = true
|
|
|
|
break
|
2021-01-19 15:35:40 +08:00
|
|
|
}
|
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
if !exist {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("partition id = %d, not belong to collection id = %d", seg.PartitionID, seg.CollectionID)
|
2021-01-19 18:32:57 +08:00
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
exist = false
|
|
|
|
for _, segID := range partMeta.SegmentIDs {
|
2021-04-09 09:55:04 +08:00
|
|
|
if segID == seg.ID {
|
2021-01-20 09:36:50 +08:00
|
|
|
exist = true
|
2021-01-19 18:32:57 +08:00
|
|
|
}
|
|
|
|
}
|
2021-01-20 09:36:50 +08:00
|
|
|
if exist {
|
2021-04-09 09:55:04 +08:00
|
|
|
return fmt.Errorf("segment id = %d exist", seg.ID)
|
2021-01-19 15:35:40 +08:00
|
|
|
}
|
2021-04-09 09:55:04 +08:00
|
|
|
partMeta.SegmentIDs = append(partMeta.SegmentIDs, seg.ID)
|
2021-01-20 09:36:50 +08:00
|
|
|
mt.partitionID2Meta[seg.PartitionID] = partMeta
|
2021-04-09 09:55:04 +08:00
|
|
|
mt.segID2CollID[seg.ID] = seg.CollectionID
|
|
|
|
mt.segID2PartitionID[seg.ID] = seg.PartitionID
|
2021-03-06 16:00:41 +08:00
|
|
|
k := fmt.Sprintf("%s/%d/%d", PartitionMetaPrefix, seg.CollectionID, seg.PartitionID)
|
|
|
|
v := proto.MarshalTextString(&partMeta)
|
|
|
|
|
|
|
|
err := mt.client.Save(k, v)
|
2021-01-19 18:32:57 +08:00
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
2021-01-21 10:01:29 +08:00
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
func (mt *metaTable) AddIndex(seg *pb.SegmentIndexInfo) error {
|
2021-01-21 10:01:29 +08:00
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
collID, ok := mt.segID2CollID[seg.SegmentID]
|
2021-02-09 13:11:55 +08:00
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("segment id = %d not belong to any collection", seg.SegmentID)
|
2021-02-11 08:41:59 +08:00
|
|
|
}
|
|
|
|
collMeta, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("collection id = %d not found", collID)
|
2021-02-11 08:41:59 +08:00
|
|
|
}
|
2021-03-06 16:00:41 +08:00
|
|
|
partID, ok := mt.segID2PartitionID[seg.SegmentID]
|
|
|
|
if !ok {
|
|
|
|
return fmt.Errorf("segment id = %d not belong to any partition", seg.SegmentID)
|
|
|
|
}
|
2021-02-11 08:41:59 +08:00
|
|
|
exist := false
|
|
|
|
for _, i := range collMeta.FieldIndexes {
|
|
|
|
if i.IndexID == seg.IndexID {
|
|
|
|
exist = true
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if !exist {
|
2021-03-05 10:15:27 +08:00
|
|
|
return fmt.Errorf("index id = %d not found", seg.IndexID)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
|
2021-04-12 15:03:23 +08:00
|
|
|
segIdxMap, ok := mt.segID2IndexMeta[seg.SegmentID]
|
|
|
|
if !ok {
|
|
|
|
idxMap := map[typeutil.UniqueID]pb.SegmentIndexInfo{seg.IndexID: *seg}
|
|
|
|
mt.segID2IndexMeta[seg.SegmentID] = &idxMap
|
|
|
|
} else {
|
|
|
|
_, ok := (*segIdxMap)[seg.IndexID]
|
|
|
|
if ok {
|
|
|
|
return fmt.Errorf("index id = %d exist", seg.IndexID)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
(*(mt.segID2IndexMeta[seg.SegmentID]))[seg.IndexID] = *seg
|
2021-03-06 16:00:41 +08:00
|
|
|
k := fmt.Sprintf("%s/%d/%d/%d/%d", SegmentIndexMetaPrefix, collID, seg.IndexID, partID, seg.SegmentID)
|
2021-02-11 08:41:59 +08:00
|
|
|
v := proto.MarshalTextString(seg)
|
|
|
|
|
|
|
|
err := mt.client.Save(k, v)
|
2021-01-21 10:01:29 +08:00
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
|
|
|
return err
|
|
|
|
}
|
2021-03-10 14:45:35 +08:00
|
|
|
|
|
|
|
if _, ok := mt.flushedSegID[seg.SegmentID]; !ok {
|
|
|
|
mt.flushedSegID[seg.SegmentID] = true
|
|
|
|
}
|
|
|
|
|
2021-01-21 10:01:29 +08:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-02-20 15:38:44 +08:00
|
|
|
func (mt *metaTable) DropIndex(collName, fieldName, indexName string) (typeutil.UniqueID, bool, error) {
|
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
|
|
|
|
|
|
|
collID, ok := mt.collName2ID[collName]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return 0, false, fmt.Errorf("collection name = %s not exist", collName)
|
2021-02-20 15:38:44 +08:00
|
|
|
}
|
|
|
|
collMeta, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return 0, false, fmt.Errorf("collection name = %s not has meta", collName)
|
2021-02-20 15:38:44 +08:00
|
|
|
}
|
|
|
|
fieldSch, err := mt.unlockGetFieldSchema(collName, fieldName)
|
|
|
|
if err != nil {
|
|
|
|
return 0, false, err
|
|
|
|
}
|
|
|
|
fieldIdxInfo := make([]*pb.FieldIndexInfo, 0, len(collMeta.FieldIndexes))
|
|
|
|
var dropIdxID typeutil.UniqueID
|
|
|
|
for i, info := range collMeta.FieldIndexes {
|
|
|
|
if info.FiledID != fieldSch.FieldID {
|
|
|
|
fieldIdxInfo = append(fieldIdxInfo, info)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
idxMeta, ok := mt.indexID2Meta[info.IndexID]
|
|
|
|
if !ok {
|
|
|
|
fieldIdxInfo = append(fieldIdxInfo, info)
|
2021-02-24 16:25:40 +08:00
|
|
|
log.Warn("index id not has meta", zap.Int64("index id", info.IndexID))
|
2021-02-20 15:38:44 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
if idxMeta.IndexName != indexName {
|
|
|
|
fieldIdxInfo = append(fieldIdxInfo, info)
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
dropIdxID = info.IndexID
|
|
|
|
fieldIdxInfo = append(fieldIdxInfo, collMeta.FieldIndexes[i+1:]...)
|
|
|
|
break
|
|
|
|
}
|
|
|
|
if len(fieldIdxInfo) == len(collMeta.FieldIndexes) {
|
2021-02-24 16:25:40 +08:00
|
|
|
log.Warn("drop index,index not found", zap.String("collection name", collName), zap.String("filed name", fieldName), zap.String("index name", indexName))
|
2021-02-20 15:38:44 +08:00
|
|
|
return 0, false, nil
|
|
|
|
}
|
|
|
|
collMeta.FieldIndexes = fieldIdxInfo
|
|
|
|
mt.collID2Meta[collID] = collMeta
|
|
|
|
saveMeta := map[string]string{path.Join(CollectionMetaPrefix, strconv.FormatInt(collID, 10)): proto.MarshalTextString(&collMeta)}
|
|
|
|
|
|
|
|
delete(mt.indexID2Meta, dropIdxID)
|
|
|
|
|
|
|
|
for _, partID := range collMeta.PartitionIDs {
|
|
|
|
partMeta, ok := mt.partitionID2Meta[partID]
|
|
|
|
if !ok {
|
2021-02-24 16:25:40 +08:00
|
|
|
log.Warn("partition not exist", zap.Int64("partition id", partID))
|
2021-02-20 15:38:44 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
for _, segID := range partMeta.SegmentIDs {
|
|
|
|
segInfo, ok := mt.segID2IndexMeta[segID]
|
|
|
|
if ok {
|
|
|
|
_, ok := (*segInfo)[dropIdxID]
|
|
|
|
if ok {
|
|
|
|
delete(*segInfo, dropIdxID)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-03-06 16:00:41 +08:00
|
|
|
delMeta := []string{
|
|
|
|
fmt.Sprintf("%s/%d/%d", SegmentIndexMetaPrefix, collMeta.ID, dropIdxID),
|
|
|
|
fmt.Sprintf("%s/%d/%d", IndexMetaPrefix, collMeta.ID, dropIdxID),
|
|
|
|
}
|
2021-02-20 15:38:44 +08:00
|
|
|
|
2021-03-06 16:00:41 +08:00
|
|
|
err = mt.client.MultiSaveAndRemoveWithPrefix(saveMeta, delMeta)
|
2021-02-20 15:38:44 +08:00
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
|
|
|
return 0, false, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return dropIdxID, true, nil
|
|
|
|
}
|
|
|
|
|
2021-01-21 10:01:29 +08:00
|
|
|
func (mt *metaTable) GetSegmentIndexInfoByID(segID typeutil.UniqueID, filedID int64, idxName string) (pb.SegmentIndexInfo, error) {
|
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
|
|
|
|
2021-03-10 14:45:35 +08:00
|
|
|
_, ok := mt.flushedSegID[segID]
|
|
|
|
if !ok {
|
|
|
|
return pb.SegmentIndexInfo{}, fmt.Errorf("segment id %d hasn't flused, there is no index meta", segID)
|
|
|
|
}
|
|
|
|
|
2021-01-21 10:01:29 +08:00
|
|
|
segIdxMap, ok := mt.segID2IndexMeta[segID]
|
|
|
|
if !ok {
|
2021-03-10 14:45:35 +08:00
|
|
|
return pb.SegmentIndexInfo{
|
|
|
|
SegmentID: segID,
|
|
|
|
FieldID: filedID,
|
|
|
|
IndexID: 0,
|
|
|
|
BuildID: 0,
|
|
|
|
EnableIndex: false,
|
|
|
|
}, nil
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
if len(*segIdxMap) == 0 {
|
2021-03-05 10:15:27 +08:00
|
|
|
return pb.SegmentIndexInfo{}, fmt.Errorf("segment id %d not has any index", segID)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
|
2021-01-27 14:41:56 +08:00
|
|
|
if filedID == -1 && idxName == "" { // return default index
|
2021-01-21 10:01:29 +08:00
|
|
|
for _, seg := range *segIdxMap {
|
2021-01-27 14:41:56 +08:00
|
|
|
info, ok := mt.indexID2Meta[seg.IndexID]
|
|
|
|
if ok && info.IndexName == Params.DefaultIndexName {
|
|
|
|
return seg, nil
|
|
|
|
}
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
} else {
|
|
|
|
for idxID, seg := range *segIdxMap {
|
|
|
|
idxMeta, ok := mt.indexID2Meta[idxID]
|
|
|
|
if ok {
|
|
|
|
if idxMeta.IndexName != idxName {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
if seg.FieldID != filedID {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
return seg, nil
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-03-05 10:15:27 +08:00
|
|
|
return pb.SegmentIndexInfo{}, fmt.Errorf("can't find index name = %s on segment = %d, with filed id = %d", idxName, segID, filedID)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) GetFieldSchema(collName string, fieldName string) (schemapb.FieldSchema, error) {
|
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
|
|
|
|
2021-02-02 10:09:10 +08:00
|
|
|
return mt.unlockGetFieldSchema(collName, fieldName)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) unlockGetFieldSchema(collName string, fieldName string) (schemapb.FieldSchema, error) {
|
2021-01-21 10:01:29 +08:00
|
|
|
collID, ok := mt.collName2ID[collName]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return schemapb.FieldSchema{}, fmt.Errorf("collection %s not found", collName)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
collMeta, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return schemapb.FieldSchema{}, fmt.Errorf("collection %s not found", collName)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
for _, field := range collMeta.Schema.Fields {
|
|
|
|
if field.Name == fieldName {
|
|
|
|
return *field, nil
|
|
|
|
}
|
|
|
|
}
|
2021-03-05 10:15:27 +08:00
|
|
|
return schemapb.FieldSchema{}, fmt.Errorf("collection %s doesn't have filed %s", collName, fieldName)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
//return true/false
|
|
|
|
func (mt *metaTable) IsSegmentIndexed(segID typeutil.UniqueID, fieldSchema *schemapb.FieldSchema, indexParams []*commonpb.KeyValuePair) bool {
|
|
|
|
mt.ddLock.RLock()
|
|
|
|
defer mt.ddLock.RUnlock()
|
2021-02-02 10:09:10 +08:00
|
|
|
return mt.unlockIsSegmentIndexed(segID, fieldSchema, indexParams)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (mt *metaTable) unlockIsSegmentIndexed(segID typeutil.UniqueID, fieldSchema *schemapb.FieldSchema, indexParams []*commonpb.KeyValuePair) bool {
|
2021-01-21 10:01:29 +08:00
|
|
|
segIdx, ok := mt.segID2IndexMeta[segID]
|
|
|
|
if !ok {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
exist := false
|
|
|
|
for idxID, meta := range *segIdx {
|
|
|
|
if meta.FieldID != fieldSchema.FieldID {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
idxMeta, ok := mt.indexID2Meta[idxID]
|
|
|
|
if !ok {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
if EqualKeyPairArray(indexParams, idxMeta.IndexParams) {
|
|
|
|
exist = true
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return exist
|
|
|
|
}
|
|
|
|
|
|
|
|
// return segment ids, type params, error
|
2021-02-09 13:11:55 +08:00
|
|
|
func (mt *metaTable) GetNotIndexedSegments(collName string, fieldName string, idxInfo *pb.IndexInfo) ([]typeutil.UniqueID, schemapb.FieldSchema, error) {
|
2021-02-02 10:09:10 +08:00
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
|
|
|
|
2021-03-10 16:21:49 +08:00
|
|
|
if idxInfo.IndexParams == nil {
|
|
|
|
return nil, schemapb.FieldSchema{}, fmt.Errorf("index param is nil")
|
|
|
|
}
|
2021-01-21 10:01:29 +08:00
|
|
|
collID, ok := mt.collName2ID[collName]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return nil, schemapb.FieldSchema{}, fmt.Errorf("collection %s not found", collName)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
collMeta, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return nil, schemapb.FieldSchema{}, fmt.Errorf("collection %s not found", collName)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
2021-02-02 10:09:10 +08:00
|
|
|
fieldSchema, err := mt.unlockGetFieldSchema(collName, fieldName)
|
2021-01-21 10:01:29 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, fieldSchema, err
|
|
|
|
}
|
|
|
|
|
2021-03-10 16:21:49 +08:00
|
|
|
var dupIdx typeutil.UniqueID = 0
|
|
|
|
for _, f := range collMeta.FieldIndexes {
|
2021-04-26 15:53:47 +08:00
|
|
|
if info, ok := mt.indexID2Meta[f.IndexID]; ok {
|
|
|
|
if info.IndexName == idxInfo.IndexName {
|
|
|
|
dupIdx = info.IndexID
|
|
|
|
break
|
2021-02-02 10:09:10 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-03-10 16:21:49 +08:00
|
|
|
|
|
|
|
exist := false
|
|
|
|
var existInfo pb.IndexInfo
|
|
|
|
for _, f := range collMeta.FieldIndexes {
|
|
|
|
if f.FiledID == fieldSchema.FieldID {
|
|
|
|
existInfo, ok = mt.indexID2Meta[f.IndexID]
|
|
|
|
if !ok {
|
|
|
|
return nil, schemapb.FieldSchema{}, fmt.Errorf("index id = %d not found", f.IndexID)
|
|
|
|
}
|
|
|
|
if EqualKeyPairArray(existInfo.IndexParams, idxInfo.IndexParams) {
|
|
|
|
exist = true
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if !exist {
|
2021-02-09 13:11:55 +08:00
|
|
|
idx := &pb.FieldIndexInfo{
|
2021-02-11 08:41:59 +08:00
|
|
|
FiledID: fieldSchema.FieldID,
|
|
|
|
IndexID: idxInfo.IndexID,
|
2021-02-09 13:11:55 +08:00
|
|
|
}
|
|
|
|
collMeta.FieldIndexes = append(collMeta.FieldIndexes, idx)
|
2021-02-02 10:09:10 +08:00
|
|
|
mt.collID2Meta[collMeta.ID] = collMeta
|
|
|
|
k1 := path.Join(CollectionMetaPrefix, strconv.FormatInt(collMeta.ID, 10))
|
|
|
|
v1 := proto.MarshalTextString(&collMeta)
|
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
mt.indexID2Meta[idx.IndexID] = *idxInfo
|
|
|
|
k2 := path.Join(IndexMetaPrefix, strconv.FormatInt(idx.IndexID, 10))
|
2021-02-19 09:52:06 +08:00
|
|
|
v2 := proto.MarshalTextString(idxInfo)
|
2021-02-11 08:41:59 +08:00
|
|
|
meta := map[string]string{k1: v1, k2: v2}
|
|
|
|
|
2021-03-10 16:21:49 +08:00
|
|
|
if dupIdx != 0 {
|
|
|
|
dupInfo := mt.indexID2Meta[dupIdx]
|
|
|
|
dupInfo.IndexName = dupInfo.IndexName + "_bak"
|
|
|
|
mt.indexID2Meta[dupIdx] = dupInfo
|
|
|
|
k := path.Join(IndexMetaPrefix, strconv.FormatInt(dupInfo.IndexID, 10))
|
|
|
|
v := proto.MarshalTextString(&dupInfo)
|
|
|
|
meta[k] = v
|
|
|
|
}
|
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
err = mt.client.MultiSave(meta)
|
2021-02-02 10:09:10 +08:00
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
|
|
|
return nil, schemapb.FieldSchema{}, err
|
|
|
|
}
|
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
} else {
|
|
|
|
idxInfo.IndexID = existInfo.IndexID
|
|
|
|
if existInfo.IndexName != idxInfo.IndexName { //replace index name
|
|
|
|
existInfo.IndexName = idxInfo.IndexName
|
|
|
|
mt.indexID2Meta[existInfo.IndexID] = existInfo
|
|
|
|
k := path.Join(IndexMetaPrefix, strconv.FormatInt(existInfo.IndexID, 10))
|
|
|
|
v := proto.MarshalTextString(&existInfo)
|
2021-03-10 16:21:49 +08:00
|
|
|
meta := map[string]string{k: v}
|
|
|
|
if dupIdx != 0 {
|
|
|
|
dupInfo := mt.indexID2Meta[dupIdx]
|
|
|
|
dupInfo.IndexName = dupInfo.IndexName + "_bak"
|
|
|
|
mt.indexID2Meta[dupIdx] = dupInfo
|
|
|
|
k := path.Join(IndexMetaPrefix, strconv.FormatInt(dupInfo.IndexID, 10))
|
|
|
|
v := proto.MarshalTextString(&dupInfo)
|
|
|
|
meta[k] = v
|
|
|
|
}
|
|
|
|
|
|
|
|
err = mt.client.MultiSave(meta)
|
2021-02-11 08:41:59 +08:00
|
|
|
if err != nil {
|
|
|
|
_ = mt.reloadFromKV()
|
|
|
|
return nil, schemapb.FieldSchema{}, err
|
|
|
|
}
|
|
|
|
}
|
2021-02-02 10:09:10 +08:00
|
|
|
}
|
|
|
|
|
2021-01-21 10:01:29 +08:00
|
|
|
rstID := make([]typeutil.UniqueID, 0, 16)
|
|
|
|
for _, partID := range collMeta.PartitionIDs {
|
|
|
|
partMeta, ok := mt.partitionID2Meta[partID]
|
|
|
|
if ok {
|
|
|
|
for _, segID := range partMeta.SegmentIDs {
|
2021-02-09 13:11:55 +08:00
|
|
|
if exist := mt.unlockIsSegmentIndexed(segID, &fieldSchema, idxInfo.IndexParams); !exist {
|
2021-01-21 10:01:29 +08:00
|
|
|
rstID = append(rstID, segID)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return rstID, fieldSchema, nil
|
|
|
|
}
|
|
|
|
|
2021-04-27 10:30:55 +08:00
|
|
|
func (mt *metaTable) GetIndexByName(collName, indexName string) (pb.CollectionInfo, []pb.IndexInfo, error) {
|
2021-01-21 10:01:29 +08:00
|
|
|
mt.ddLock.RLock()
|
2021-02-08 15:19:48 +08:00
|
|
|
defer mt.ddLock.RUnlock()
|
2021-01-21 10:01:29 +08:00
|
|
|
|
|
|
|
collID, ok := mt.collName2ID[collName]
|
|
|
|
if !ok {
|
2021-04-27 10:30:55 +08:00
|
|
|
return pb.CollectionInfo{}, nil, fmt.Errorf("collection %s not found", collName)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
collMeta, ok := mt.collID2Meta[collID]
|
|
|
|
if !ok {
|
2021-04-27 10:30:55 +08:00
|
|
|
return pb.CollectionInfo{}, nil, fmt.Errorf("collection %s not found", collName)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
|
2021-02-11 08:41:59 +08:00
|
|
|
rstIndex := make([]pb.IndexInfo, 0, len(collMeta.FieldIndexes))
|
2021-02-19 09:52:06 +08:00
|
|
|
for _, idx := range collMeta.FieldIndexes {
|
2021-04-26 15:53:47 +08:00
|
|
|
idxInfo, ok := mt.indexID2Meta[idx.IndexID]
|
|
|
|
if !ok {
|
2021-04-27 10:30:55 +08:00
|
|
|
return pb.CollectionInfo{}, nil, fmt.Errorf("index id = %d not found", idx.IndexID)
|
2021-04-26 15:53:47 +08:00
|
|
|
}
|
|
|
|
if indexName == "" || idxInfo.IndexName == indexName {
|
|
|
|
rstIndex = append(rstIndex, idxInfo)
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
|
|
|
}
|
2021-04-27 10:30:55 +08:00
|
|
|
return collMeta, rstIndex, nil
|
2021-01-21 10:01:29 +08:00
|
|
|
}
|
2021-01-27 14:41:56 +08:00
|
|
|
|
|
|
|
func (mt *metaTable) GetIndexByID(indexID typeutil.UniqueID) (*pb.IndexInfo, error) {
|
|
|
|
mt.ddLock.RLock()
|
2021-02-08 15:19:48 +08:00
|
|
|
defer mt.ddLock.RUnlock()
|
2021-01-27 14:41:56 +08:00
|
|
|
|
|
|
|
indexInfo, ok := mt.indexID2Meta[indexID]
|
|
|
|
if !ok {
|
2021-03-05 10:15:27 +08:00
|
|
|
return nil, fmt.Errorf("cannot find index, id = %d", indexID)
|
2021-01-27 14:41:56 +08:00
|
|
|
}
|
|
|
|
return &indexInfo, nil
|
|
|
|
}
|
2021-03-10 14:45:35 +08:00
|
|
|
|
|
|
|
func (mt *metaTable) AddFlushedSegment(segID typeutil.UniqueID) error {
|
|
|
|
mt.ddLock.Lock()
|
|
|
|
defer mt.ddLock.Unlock()
|
|
|
|
|
|
|
|
_, ok := mt.flushedSegID[segID]
|
|
|
|
if ok {
|
|
|
|
return fmt.Errorf("segment id = %d exist", segID)
|
|
|
|
}
|
|
|
|
mt.flushedSegID[segID] = true
|
|
|
|
return nil
|
|
|
|
}
|