Add the standard of golangci-lint

Signed-off-by: cai.zhang <cai.zhang@zilliz.com>
pull/4973/head^2
cai.zhang 2020-11-12 12:04:12 +08:00 committed by yefu.chen
parent 4a11a6431b
commit 1cd2fd9954
65 changed files with 912 additions and 1447 deletions

View File

@ -9,7 +9,7 @@ run:
linters-settings:
golint:
min-confidence: 0
min-confidence: 0.8
misspell:
locale: US
@ -25,6 +25,15 @@ linters:
- ineffassign
- gosimple
issues:
exclude-use-default: false
exclude:
- should have a package comment
- should have comment
- should be of the form
- should not use dot imports
- which can be annoying to use
- AllocId
service:
golangci-lint-version: 1.27.0 # use the fixed version to not introduce new linters unexpectedly

View File

@ -7,8 +7,9 @@ import (
"os/signal"
"syscall"
mockmaster "github.com/zilliztech/milvus-distributed/internal/master/mock"
"go.uber.org/zap"
mockmaster "github.com/zilliztech/milvus-distributed/internal/master/mock"
)
func main() {

View File

@ -15,6 +15,8 @@ import (
func main() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
var yamlFile string
flag.StringVar(&yamlFile, "yaml", "", "yaml file")
flag.Parse()

1
go.sum
View File

@ -246,6 +246,7 @@ github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsT
github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A=
github.com/prometheus/procfs v0.1.3 h1:F0+tqvhOksq22sc6iCHF5WGlWjdwj92p0udFh1VFBS8=
github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU=
github.com/protocolbuffers/protobuf v3.13.0+incompatible h1:omZA3Tuq+U2kJ2uMuqMR9c1VO5qLEgZ19m9878fXNtg=
github.com/remyoudompheng/bigfft v0.0.0-20170806203942-52369c62f446/go.mod h1:uYEyJGbgTkfkS4+E/PavXkNJcbFIpEtjt2B0KDQ5+9M=
github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=

View File

@ -115,7 +115,7 @@ type Allocator struct {
masterAddress string
masterConn *grpc.ClientConn
masterClient masterpb.MasterClient
countPerRpc uint32
countPerRPC uint32
tChan tickerChan
syncFunc func()
@ -135,7 +135,8 @@ func (ta *Allocator) Start() error {
func (ta *Allocator) connectMaster() error {
log.Printf("Connected to master, master_addr=%s", ta.masterAddress)
ctx, _ := context.WithTimeout(context.Background(), 2*time.Second)
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
conn, err := grpc.DialContext(ctx, ta.masterAddress, grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
log.Printf("Connect to master failed, error= %v", err)

View File

@ -12,40 +12,40 @@ import (
type UniqueID = typeutil.UniqueID
type IdAllocator struct {
type IDAllocator struct {
Allocator
idStart UniqueID
idEnd UniqueID
}
func NewIdAllocator(ctx context.Context) (*IdAllocator, error) {
func NewIDAllocator(ctx context.Context) (*IDAllocator, error) {
ctx1, cancel := context.WithCancel(ctx)
a := &IdAllocator{
a := &IDAllocator{
Allocator: Allocator{reqs: make(chan request, maxMergeRequests),
ctx: ctx1,
cancel: cancel,
},
}
a.tChan = &emptyTicker{}
a.Allocator.syncFunc = a.syncId
a.Allocator.syncFunc = a.syncID
a.Allocator.processFunc = a.processFunc
return a, nil
}
func (ta *IdAllocator) syncId() {
fmt.Println("syncId")
func (ta *IDAllocator) syncID() {
fmt.Println("syncID")
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
req := &internalpb.IdRequest{
PeerId: 1,
Role: internalpb.PeerRole_Proxy,
Count: ta.countPerRpc,
Count: ta.countPerRPC,
}
resp, err := ta.masterClient.AllocId(ctx, req)
cancel()
if err != nil {
log.Panic("syncId Failed!!!!!")
log.Panic("syncID Failed!!!!!")
return
}
ta.idStart = resp.GetId()
@ -53,13 +53,13 @@ func (ta *IdAllocator) syncId() {
}
func (ta *IdAllocator) processFunc(req request) {
func (ta *IDAllocator) processFunc(req request) {
idRequest := req.(*idRequest)
idRequest.id = 1
fmt.Println("process Id")
fmt.Println("process ID")
}
func (ta *IdAllocator) AllocOne() (UniqueID, error) {
func (ta *IDAllocator) AllocOne() (UniqueID, error) {
ret, _, err := ta.Alloc(1)
if err != nil {
return 0, err
@ -67,7 +67,7 @@ func (ta *IdAllocator) AllocOne() (UniqueID, error) {
return ret, nil
}
func (ta *IdAllocator) Alloc(count uint32) (UniqueID, UniqueID, error) {
func (ta *IDAllocator) Alloc(count uint32) (UniqueID, UniqueID, error) {
req := &idRequest{baseRequest: baseRequest{done: make(chan error), valid: false}}
req.count = count

View File

@ -46,13 +46,13 @@ func (ta *TimestampAllocator) syncTs() {
req := &internalpb.TsoRequest{
PeerId: 1,
Role: internalpb.PeerRole_Proxy,
Count: ta.countPerRpc,
Count: ta.countPerRPC,
}
resp, err := ta.masterClient.AllocTimestamp(ctx, req)
cancel()
if err != nil {
log.Panic("syncId Failed!!!!!")
log.Panic("syncID Failed!!!!!")
return
}
ta.lastTsBegin = resp.GetTimestamp()

View File

@ -21,7 +21,7 @@ type MasterConfig struct {
PulsarMoniterInterval int32
PulsarTopic string
SegmentThreshole float32
ProxyIdList []UniqueID
ProxyIDList []UniqueID
QueryNodeNum int
WriteNodeNum int
}
@ -56,7 +56,7 @@ type PulsarConfig struct {
type ProxyConfig struct {
Timezone string `yaml:"timezone"`
ProxyId int `yaml:"proxy_id"`
ProxyID int `yaml:"proxy_id"`
NumReaderNodes int `yaml:"numReaderNodes"`
TosSaveInterval int `yaml:"tsoSaveInterval"`
TimeTickInterval int `yaml:"timeTickInterval"`
@ -87,7 +87,7 @@ type ProxyConfig struct {
}
type Reader struct {
ClientId int
ClientID int
StopFlag int64
ReaderQueueSize int
SearchChanSize int
@ -97,10 +97,10 @@ type Reader struct {
}
type Writer struct {
ClientId int
ClientID int
StopFlag int64
ReaderQueueSize int
SearchByIdChanSize int
SearchByIDChanSize int
Parallelism int
TopicStart int
TopicEnd int

View File

@ -34,7 +34,8 @@ func (kv *EtcdKV) Close() {
func (kv *EtcdKV) LoadWithPrefix(key string) ([]string, []string, error) {
key = path.Join(kv.rootPath, key)
log.Printf("LoadWithPrefix %s", key)
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key, clientv3.WithPrefix(), clientv3.WithSort(clientv3.SortByKey, clientv3.SortAscend))
if err != nil {
return nil, nil, err
@ -50,7 +51,8 @@ func (kv *EtcdKV) LoadWithPrefix(key string) ([]string, []string, error) {
func (kv *EtcdKV) Load(key string) (string, error) {
key = path.Join(kv.rootPath, key)
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
resp, err := kv.client.Get(ctx, key)
if err != nil {
return "", err
@ -64,11 +66,12 @@ func (kv *EtcdKV) Load(key string) (string, error) {
func (kv *EtcdKV) MultiLoad(keys []string) ([]string, error) {
ops := make([]clientv3.Op, 0, len(keys))
for _, key_load := range keys {
ops = append(ops, clientv3.OpGet(path.Join(kv.rootPath, key_load)))
for _, keyLoad := range keys {
ops = append(ops, clientv3.OpGet(path.Join(kv.rootPath, keyLoad)))
}
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
resp, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
if err != nil {
return []string{}, err
@ -96,7 +99,8 @@ func (kv *EtcdKV) MultiLoad(keys []string) ([]string, error) {
func (kv *EtcdKV) Save(key, value string) error {
key = path.Join(kv.rootPath, key)
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
_, err := kv.client.Put(ctx, key, value)
return err
}
@ -107,14 +111,18 @@ func (kv *EtcdKV) MultiSave(kvs map[string]string) error {
ops = append(ops, clientv3.OpPut(path.Join(kv.rootPath, key), value))
}
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
return err
}
func (kv *EtcdKV) Remove(key string) error {
key = path.Join(kv.rootPath, key)
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
_, err := kv.client.Delete(ctx, key)
return err
}
@ -125,7 +133,9 @@ func (kv *EtcdKV) MultiRemove(keys []string) error {
ops = append(ops, clientv3.OpDelete(path.Join(kv.rootPath, key)))
}
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
return err
}
@ -136,12 +146,14 @@ func (kv *EtcdKV) MultiSaveAndRemove(saves map[string]string, removals []string)
ops = append(ops, clientv3.OpPut(path.Join(kv.rootPath, key), value))
}
for _, key_delete := range removals {
ops = append(ops, clientv3.OpDelete(path.Join(kv.rootPath, key_delete)))
for _, keyDelete := range removals {
ops = append(ops, clientv3.OpDelete(path.Join(kv.rootPath, keyDelete)))
}
log.Printf("MultiSaveAndRemove")
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
_, err := kv.client.Txn(ctx).If().Then(ops...).Commit()
return err
}

View File

@ -14,7 +14,8 @@ func TestEtcdKV_Load(t *testing.T) {
assert.Nil(t, err)
rootpath := "/etcd/test/root"
kv := NewEtcdKV(cli, rootpath)
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
defer kv.Close()
defer kv.client.Delete(ctx, rootpath, clientv3.WithPrefix())
@ -67,7 +68,8 @@ func TestEtcdKV_MultiSave(t *testing.T) {
assert.Nil(t, err)
rootpath := "/etcd/test/root"
kv := NewEtcdKV(cli, rootpath)
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
defer kv.Close()
defer kv.client.Delete(ctx, rootpath, clientv3.WithPrefix())
@ -93,7 +95,8 @@ func TestEtcdKV_Remove(t *testing.T) {
assert.Nil(t, err)
rootpath := "/etcd/test/root"
kv := NewEtcdKV(cli, rootpath)
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
defer kv.Close()
defer kv.client.Delete(ctx, rootpath, clientv3.WithPrefix())
@ -159,8 +162,8 @@ func TestEtcdKV_MultiSaveAndRemove(t *testing.T) {
assert.Nil(t, err)
rootpath := "/etcd/test/root"
kv := NewEtcdKV(cli, rootpath)
ctx, _ := context.WithTimeout(context.TODO(), requestTimeout)
ctx, cancel := context.WithTimeout(context.TODO(), requestTimeout)
defer cancel()
defer kv.Close()
defer kv.client.Delete(ctx, rootpath, clientv3.WithPrefix())

View File

@ -1,6 +1,6 @@
package kv
type KVBase interface {
type Base interface {
Load(key string) (string, error)
MultiLoad(keys []string) ([]string, error)
LoadWithPrefix(key string) ([]string, []string, error)

View File

@ -1,10 +1,9 @@
package kv
import (
"github.com/google/btree"
"sync"
"github.com/google/btree"
)
type MemoryKV struct {
@ -12,7 +11,6 @@ type MemoryKV struct {
tree *btree.BTree
}
// NewMemoryKV returns an in-memory kvBase for testing.
func NewMemoryKV() *MemoryKV {
return &MemoryKV{
@ -24,13 +22,10 @@ type memoryKVItem struct {
key, value string
}
func (s memoryKVItem) Less(than btree.Item) bool {
return s.key < than.(memoryKVItem).key
}
func (kv *MemoryKV) Load(key string) (string, error) {
kv.RLock()
defer kv.RUnlock()
@ -72,12 +67,11 @@ func (kv *MemoryKV) Remove(key string) error {
return nil
}
func (kv *MemoryKV) MultiLoad(keys []string) ([]string, error) {
kv.RLock()
defer kv.RUnlock()
result := make([]string, 0, len(keys))
for _,key := range keys {
for _, key := range keys {
item := kv.tree.Get(memoryKVItem{key, ""})
result = append(result, item.(memoryKVItem).value)
}
@ -102,7 +96,6 @@ func (kv *MemoryKV) MultiRemove(keys []string) error {
return nil
}
func (kv *MemoryKV) MultiSaveAndRemove(saves map[string]string, removals []string) error {
kv.Lock()
defer kv.Unlock()
@ -121,4 +114,4 @@ func (kv *MemoryKV) LoadWithPrefix(key string) ([]string, []string, error) {
}
func (kv *MemoryKV) Close() {
}
}

View File

@ -50,7 +50,7 @@ func GrpcMarshal(c *Collection) *Collection {
}
pbSchema.Fields = schemaSlice
grpcCollection := &etcdpb.CollectionMeta{
Id: c.ID,
ID: c.ID,
Schema: pbSchema,
CreateTime: c.CreateTime,
SegmentIds: c.SegmentIDs,
@ -62,16 +62,14 @@ func GrpcMarshal(c *Collection) *Collection {
}
func NewCollection(id UniqueID, name string, createTime time.Time,
schema []*schemapb.FieldSchema, sIds []UniqueID, ptags []string) Collection {
schema []*schemapb.FieldSchema, sIDs []UniqueID, ptags []string) Collection {
segementIDs := []UniqueID{}
newSchema := []FieldMeta{}
for _, v := range schema {
newSchema = append(newSchema, FieldMeta{FieldName: v.Name, Type: v.DataType, DIM: 16})
}
for _, sid := range sIds {
segementIDs = append(segementIDs, sid)
}
segementIDs = append(segementIDs, sIDs...)
return Collection{
ID: id,
Name: name,

View File

@ -13,26 +13,26 @@ var (
name = "test-segment"
createTime = time.Now()
schema = []*schemapb.FieldSchema{}
sIds = []UniqueID{111111, 222222}
sIDs = []UniqueID{111111, 222222}
ptags = []string{"default", "test"}
)
func TestNewCollection(t *testing.T) {
assert := assert.New(t)
c := NewCollection(cid, name, createTime, schema, sIds, ptags)
c := NewCollection(cid, name, createTime, schema, sIDs, ptags)
assert.Equal(cid, c.ID)
assert.Equal(name, c.Name)
for k, v := range schema {
assert.Equal(v.Name, c.Schema[k].FieldName)
assert.Equal(v.DataType, c.Schema[k].Type)
}
assert.Equal(sIds, c.SegmentIDs)
assert.Equal(sIDs, c.SegmentIDs)
assert.Equal(ptags, c.PartitionTags)
}
func TestGrpcMarshal(t *testing.T) {
assert := assert.New(t)
c := NewCollection(cid, name, createTime, schema, sIds, ptags)
c := NewCollection(cid, name, createTime, schema, sIDs, ptags)
newc := GrpcMarshal(&c)
assert.NotEqual("", newc.GrpcMarshalString)
}

View File

@ -74,7 +74,7 @@ func (t *createCollectionTask) Execute() error {
return err
}
collectionId, err := allocGlobalId()
collectionID, err := allocGlobalId()
if err != nil {
return err
}
@ -85,7 +85,7 @@ func (t *createCollectionTask) Execute() error {
}
collection := etcdpb.CollectionMeta{
Id: collectionId,
ID: collectionID,
Schema: &schema,
CreateTime: ts,
// TODO: initial segment?
@ -124,14 +124,14 @@ func (t *dropCollectionTask) Execute() error {
return err
}
collectionId := collectionMeta.Id
collectionID := collectionMeta.ID
err = (*t.kvBase).Remove(collectionMetaPrefix + strconv.FormatInt(collectionId, 10))
err = (*t.kvBase).Remove(collectionMetaPrefix + strconv.FormatInt(collectionID, 10))
if err != nil {
return err
}
delete(t.mt.collId2Meta, collectionId)
delete(t.mt.collID2Meta, collectionID)
return nil
}
@ -227,7 +227,7 @@ func (t *showCollectionsTask) Execute() error {
}
collections := make([]string, 0)
for _, collection := range t.mt.collId2Meta {
for _, collection := range t.mt.collID2Meta {
collections = append(collections, collection.Schema.Name)
}

View File

@ -357,7 +357,6 @@ func (s *Master) AllocTimestamp(ctx context.Context, request *internalpb.TsoRequ
}
func (s *Master) AllocId(ctx context.Context, request *internalpb.IdRequest) (*internalpb.IdResponse, error) {
panic("implement me")
count := request.GetCount()
ts, err := id.AllocOne()
@ -369,7 +368,7 @@ func (s *Master) AllocId(ctx context.Context, request *internalpb.IdRequest) (*i
response := &internalpb.IdResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
Id: ts,
ID: ts,
Count: count,
}

View File

@ -2,6 +2,8 @@ package master
import (
"context"
"testing"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
@ -10,7 +12,6 @@ import (
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"go.etcd.io/etcd/clientv3"
"google.golang.org/grpc"
"testing"
)
func TestMaster_CreateCollection(t *testing.T) {
@ -105,7 +106,7 @@ func TestMaster_CreateCollection(t *testing.T) {
coll_meta, err := svr.mt.GetCollectionByName(sch.Name)
assert.Nil(t, err)
t.Logf("collection id = %d", coll_meta.Id)
t.Logf("collection id = %d", coll_meta.ID)
assert.Equal(t, coll_meta.CreateTime, uint64(11))
assert.Equal(t, coll_meta.Schema.Name, "col1")
assert.Equal(t, coll_meta.Schema.AutoId, false)

View File

@ -10,35 +10,35 @@ import (
type UniqueID = typeutil.UniqueID
// GlobalTSOAllocator is the global single point TSO allocator.
type GlobalIdAllocator struct {
type GlobalIDAllocator struct {
allocator tso.Allocator
}
var allocator *GlobalIdAllocator
var allocator *GlobalIDAllocator
func Init() {
InitGlobalIdAllocator("idTimestamp", tsoutil.NewTSOKVBase("gid"))
InitGlobalIDAllocator("idTimestamp", tsoutil.NewTSOKVBase("gid"))
}
func InitGlobalIdAllocator(key string, base kv.KVBase) {
allocator = NewGlobalIdAllocator(key, base)
func InitGlobalIDAllocator(key string, base kv.Base) {
allocator = NewGlobalIDAllocator(key, base)
allocator.Initialize()
}
func NewGlobalIdAllocator(key string, base kv.KVBase) *GlobalIdAllocator {
return &GlobalIdAllocator{
func NewGlobalIDAllocator(key string, base kv.Base) *GlobalIDAllocator {
return &GlobalIDAllocator{
allocator: tso.NewGlobalTSOAllocator(key, base),
}
}
// Initialize will initialize the created global TSO allocator.
func (gia *GlobalIdAllocator) Initialize() error {
func (gia *GlobalIDAllocator) Initialize() error {
return gia.allocator.Initialize()
}
// GenerateTSO is used to generate a given number of TSOs.
// Make sure you have initialized the TSO allocator before calling.
func (gia *GlobalIdAllocator) Alloc(count uint32) (UniqueID, UniqueID, error) {
func (gia *GlobalIDAllocator) Alloc(count uint32) (UniqueID, UniqueID, error) {
timestamp, err := gia.allocator.GenerateTSO(count)
if err != nil {
return 0, 0, err
@ -48,7 +48,7 @@ func (gia *GlobalIdAllocator) Alloc(count uint32) (UniqueID, UniqueID, error) {
return idStart, idEnd, nil
}
func (gia *GlobalIdAllocator) AllocOne() (UniqueID, error) {
func (gia *GlobalIDAllocator) AllocOne() (UniqueID, error) {
timestamp, err := gia.allocator.GenerateTSO(1)
if err != nil {
return 0, err

View File

@ -5,15 +5,16 @@ import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
)
var GIdAllocator *GlobalIdAllocator
var GIdAllocator *GlobalIDAllocator
func TestMain(m *testing.M) {
conf.LoadConfig("config.yaml")
GIdAllocator = NewGlobalIdAllocator("idTimestamp", tsoutil.NewTSOKVBase("gid"))
GIdAllocator = NewGlobalIDAllocator("idTimestamp", tsoutil.NewTSOKVBase("gid"))
exitCode := m.Run()
os.Exit(exitCode)
}

View File

@ -3,7 +3,6 @@ package master
import (
"context"
"fmt"
"github.com/zilliztech/milvus-distributed/internal/errors"
"log"
"math/rand"
"net"
@ -12,20 +11,20 @@ import (
"sync/atomic"
"time"
"github.com/zilliztech/milvus-distributed/internal/master/id"
"github.com/zilliztech/milvus-distributed/internal/master/tso"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/master/controller"
"github.com/zilliztech/milvus-distributed/internal/master/informer"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"go.etcd.io/etcd/clientv3"
"google.golang.org/grpc"
"go.etcd.io/etcd/clientv3"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/master/controller"
"github.com/zilliztech/milvus-distributed/internal/master/id"
"github.com/zilliztech/milvus-distributed/internal/master/informer"
"github.com/zilliztech/milvus-distributed/internal/master/tso"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
)
// Server is the pd server.
@ -58,12 +57,12 @@ type Master struct {
closeCallbacks []func()
}
func newKVBase(kv_root string, etcdAddr []string) *kv.EtcdKV {
func newKVBase(kvRoot string, etcdAddr []string) *kv.EtcdKV {
cli, _ := clientv3.New(clientv3.Config{
Endpoints: etcdAddr,
DialTimeout: 5 * time.Second,
})
kvBase := kv.NewEtcdKV(cli, kv_root)
kvBase := kv.NewEtcdKV(cli, kvRoot)
return kvBase
}
@ -265,7 +264,8 @@ func (s *Master) pulsarLoop() {
func (s *Master) tasksExecutionLoop() {
defer s.serverLoopWg.Done()
ctx, _ := context.WithCancel(s.serverLoopCtx)
ctx, cancel := context.WithCancel(s.serverLoopCtx)
defer cancel()
for {
select {

View File

@ -15,12 +15,12 @@ import (
type UniqueID = typeutil.UniqueID
type metaTable struct {
client *kv.EtcdKV // client of a reliable kv service, i.e. etcd client
tenantId2Meta map[UniqueID]pb.TenantMeta // tenant id to tenant meta
proxyId2Meta map[UniqueID]pb.ProxyMeta // proxy id to proxy meta
collId2Meta map[UniqueID]pb.CollectionMeta // collection id to collection meta
collName2Id map[string]UniqueID // collection name to collection id
segId2Meta map[UniqueID]pb.SegmentMeta // segment id to segment meta
client *kv.EtcdKV // client of a reliable kv service, i.e. etcd client
tenantID2Meta map[UniqueID]pb.TenantMeta // tenant id to tenant meta
proxyID2Meta map[UniqueID]pb.ProxyMeta // proxy id to proxy meta
collID2Meta map[UniqueID]pb.CollectionMeta // collection id to collection meta
collName2ID map[string]UniqueID // collection name to collection id
segID2Meta map[UniqueID]pb.SegmentMeta // segment id to segment meta
tenantLock sync.RWMutex
proxyLock sync.RWMutex
@ -43,11 +43,11 @@ func NewMetaTable(kv *kv.EtcdKV) (*metaTable, error) {
func (mt *metaTable) reloadFromKV() error {
mt.tenantId2Meta = make(map[UniqueID]pb.TenantMeta)
mt.proxyId2Meta = make(map[UniqueID]pb.ProxyMeta)
mt.collId2Meta = make(map[UniqueID]pb.CollectionMeta)
mt.collName2Id = make(map[string]UniqueID)
mt.segId2Meta = make(map[UniqueID]pb.SegmentMeta)
mt.tenantID2Meta = make(map[UniqueID]pb.TenantMeta)
mt.proxyID2Meta = make(map[UniqueID]pb.ProxyMeta)
mt.collID2Meta = make(map[UniqueID]pb.CollectionMeta)
mt.collName2ID = make(map[string]UniqueID)
mt.segID2Meta = make(map[UniqueID]pb.SegmentMeta)
_, values, err := mt.client.LoadWithPrefix("tenant")
if err != nil {
@ -55,12 +55,12 @@ func (mt *metaTable) reloadFromKV() error {
}
for _, value := range values {
tenant_meta := pb.TenantMeta{}
err := proto.Unmarshal([]byte(value), &tenant_meta)
tenantMeta := pb.TenantMeta{}
err := proto.Unmarshal([]byte(value), &tenantMeta)
if err != nil {
return err
}
mt.tenantId2Meta[tenant_meta.Id] = tenant_meta
mt.tenantID2Meta[tenantMeta.ID] = tenantMeta
}
_, values, err = mt.client.LoadWithPrefix("proxy")
@ -69,12 +69,12 @@ func (mt *metaTable) reloadFromKV() error {
}
for _, value := range values {
proxy_meta := pb.ProxyMeta{}
err = proto.Unmarshal([]byte(value), &proxy_meta)
proxyMeta := pb.ProxyMeta{}
err = proto.Unmarshal([]byte(value), &proxyMeta)
if err != nil {
return err
}
mt.proxyId2Meta[proxy_meta.Id] = proxy_meta
mt.proxyID2Meta[proxyMeta.ID] = proxyMeta
}
_, values, err = mt.client.LoadWithPrefix("collection")
@ -83,13 +83,13 @@ func (mt *metaTable) reloadFromKV() error {
}
for _, value := range values {
collection_meta := pb.CollectionMeta{}
err = proto.Unmarshal([]byte(value), &collection_meta)
collectionMeta := pb.CollectionMeta{}
err = proto.Unmarshal([]byte(value), &collectionMeta)
if err != nil {
return err
}
mt.collId2Meta[collection_meta.Id] = collection_meta
mt.collName2Id[collection_meta.Schema.Name] = collection_meta.Id
mt.collID2Meta[collectionMeta.ID] = collectionMeta
mt.collName2ID[collectionMeta.Schema.Name] = collectionMeta.ID
}
_, values, err = mt.client.LoadWithPrefix("segment")
@ -98,12 +98,12 @@ func (mt *metaTable) reloadFromKV() error {
}
for _, value := range values {
segment_meta := pb.SegmentMeta{}
err = proto.Unmarshal([]byte(value), &segment_meta)
segmentMeta := pb.SegmentMeta{}
err = proto.Unmarshal([]byte(value), &segmentMeta)
if err != nil {
return err
}
mt.segId2Meta[segment_meta.SegmentId] = segment_meta
mt.segID2Meta[segmentMeta.SegmentId] = segmentMeta
}
return nil
@ -111,43 +111,43 @@ func (mt *metaTable) reloadFromKV() error {
// mt.ddLock.Lock() before call this function
func (mt *metaTable) saveCollectionMeta(coll *pb.CollectionMeta) error {
coll_bytes, err := proto.Marshal(coll)
collBytes, err := proto.Marshal(coll)
if err != nil {
return err
}
mt.collId2Meta[coll.Id] = *coll
mt.collName2Id[coll.Schema.Name] = coll.Id
return mt.client.Save("/collection/"+strconv.FormatInt(coll.Id, 10), string(coll_bytes))
mt.collID2Meta[coll.ID] = *coll
mt.collName2ID[coll.Schema.Name] = coll.ID
return mt.client.Save("/collection/"+strconv.FormatInt(coll.ID, 10), string(collBytes))
}
// mt.ddLock.Lock() before call this function
func (mt *metaTable) saveSegmentMeta(seg *pb.SegmentMeta) error {
seg_bytes, err := proto.Marshal(seg)
segBytes, err := proto.Marshal(seg)
if err != nil {
return err
}
mt.segId2Meta[seg.SegmentId] = *seg
mt.segID2Meta[seg.SegmentId] = *seg
return mt.client.Save("/segment/"+strconv.FormatInt(seg.SegmentId, 10), string(seg_bytes))
return mt.client.Save("/segment/"+strconv.FormatInt(seg.SegmentId, 10), string(segBytes))
}
// mt.ddLock.Lock() before call this function
func (mt *metaTable) deleteSegmentMeta(segId UniqueID) error {
_, ok := mt.segId2Meta[segId]
func (mt *metaTable) deleteSegmentMeta(segID UniqueID) error {
_, ok := mt.segID2Meta[segID]
if ok {
delete(mt.segId2Meta, segId)
delete(mt.segID2Meta, segID)
}
return mt.client.Remove("/segment/" + strconv.FormatInt(segId, 10))
return mt.client.Remove("/segment/" + strconv.FormatInt(segID, 10))
}
// mt.ddLock.Lock() before call this function
func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta, segIds []UniqueID) error {
segIdStrs := make([]string, 0, len(segIds))
for _, segId := range segIds {
segIdStrs = append(segIdStrs, "/segment/"+strconv.FormatInt(segId, 10))
func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta, segIDs []UniqueID) error {
segIDStrs := make([]string, 0, len(segIDs))
for _, segID := range segIDs {
segIDStrs = append(segIDStrs, "/segment/"+strconv.FormatInt(segID, 10))
}
kvs := make(map[string]string)
@ -156,75 +156,75 @@ func (mt *metaTable) saveCollectionAndDeleteSegmentsMeta(coll *pb.CollectionMeta
return err
}
kvs["/collection/"+strconv.FormatInt(coll.Id, 10)] = string(collStrs)
kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = string(collStrs)
for _, segId := range segIds {
_, ok := mt.segId2Meta[segId]
for _, segID := range segIDs {
_, ok := mt.segID2Meta[segID]
if ok {
delete(mt.segId2Meta, segId)
delete(mt.segID2Meta, segID)
}
}
mt.collId2Meta[coll.Id] = *coll
mt.collID2Meta[coll.ID] = *coll
return mt.client.MultiSaveAndRemove(kvs, segIdStrs)
return mt.client.MultiSaveAndRemove(kvs, segIDStrs)
}
// mt.ddLock.Lock() before call this function
func (mt *metaTable) saveCollectionsAndSegmentsMeta(coll *pb.CollectionMeta, seg *pb.SegmentMeta) error {
kvs := make(map[string]string, 0)
coll_bytes, err := proto.Marshal(coll)
kvs := make(map[string]string)
collBytes, err := proto.Marshal(coll)
if err != nil {
return err
}
kvs["/collection/"+strconv.FormatInt(coll.Id, 10)] = string(coll_bytes)
kvs["/collection/"+strconv.FormatInt(coll.ID, 10)] = string(collBytes)
mt.collId2Meta[coll.Id] = *coll
mt.collName2Id[coll.Schema.Name] = coll.Id
mt.collID2Meta[coll.ID] = *coll
mt.collName2ID[coll.Schema.Name] = coll.ID
seg_bytes, err := proto.Marshal(seg)
segBytes, err := proto.Marshal(seg)
if err != nil {
return err
}
kvs["/segment/"+strconv.FormatInt(seg.SegmentId, 10)] = string(seg_bytes)
kvs["/segment/"+strconv.FormatInt(seg.SegmentId, 10)] = string(segBytes)
mt.segId2Meta[seg.SegmentId] = *seg
mt.segID2Meta[seg.SegmentId] = *seg
return mt.client.MultiSave(kvs)
}
// mt.ddLock.Lock() before call this function
func (mt *metaTable) deleteCollectionsAndSegmentsMeta(collId UniqueID, segIds []UniqueID) error {
collIdStr := "/collection/" + strconv.FormatInt(collId, 10)
func (mt *metaTable) deleteCollectionsAndSegmentsMeta(collID UniqueID, segIDs []UniqueID) error {
collIDStr := "/collection/" + strconv.FormatInt(collID, 10)
totalIdStrs := make([]string, 0, 1+len(segIds))
totalIdStrs = append(totalIdStrs, collIdStr)
for _, singleId := range segIds {
totalIdStrs = append(totalIdStrs, "/segment/"+strconv.FormatInt(singleId, 10))
totalIDStrs := make([]string, 0, 1+len(segIDs))
totalIDStrs = append(totalIDStrs, collIDStr)
for _, singleID := range segIDs {
totalIDStrs = append(totalIDStrs, "/segment/"+strconv.FormatInt(singleID, 10))
}
coll_meta, ok := mt.collId2Meta[collId]
collMeta, ok := mt.collID2Meta[collID]
if ok {
delete(mt.collId2Meta, collId)
delete(mt.collID2Meta, collID)
}
_, ok = mt.collName2Id[coll_meta.Schema.Name]
_, ok = mt.collName2ID[collMeta.Schema.Name]
if ok {
delete(mt.collName2Id, coll_meta.Schema.Name)
delete(mt.collName2ID, collMeta.Schema.Name)
}
for _, segId := range segIds {
_, ok := mt.segId2Meta[segId]
for _, segID := range segIDs {
_, ok := mt.segID2Meta[segID]
if ok {
delete(mt.segId2Meta, segId)
delete(mt.segID2Meta, segID)
}
}
return mt.client.MultiRemove(totalIdStrs)
return mt.client.MultiRemove(totalIDStrs)
}
func (mt *metaTable) AddCollection(coll *pb.CollectionMeta) error {
@ -236,7 +236,7 @@ func (mt *metaTable) AddCollection(coll *pb.CollectionMeta) error {
if len(coll.PartitionTags) != 0 {
return errors.Errorf("segment should be empty when creating collection")
}
_, ok := mt.collName2Id[coll.Schema.Name]
_, ok := mt.collName2ID[coll.Schema.Name]
if ok {
return errors.Errorf("collection alread exists with name = " + coll.Schema.Name)
}
@ -248,16 +248,16 @@ func (mt *metaTable) AddCollection(coll *pb.CollectionMeta) error {
return nil
}
func (mt *metaTable) DeleteCollection(collId UniqueID) error {
func (mt *metaTable) DeleteCollection(collID UniqueID) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
coll_meta, ok := mt.collId2Meta[collId]
collMeta, ok := mt.collID2Meta[collID]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collId, 10))
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
err := mt.deleteCollectionsAndSegmentsMeta(collId, coll_meta.SegmentIds)
err := mt.deleteCollectionsAndSegmentsMeta(collID, collMeta.SegmentIds)
if err != nil {
_ = mt.reloadFromKV()
return err
@ -265,37 +265,34 @@ func (mt *metaTable) DeleteCollection(collId UniqueID) error {
return nil
}
func (mt *metaTable) HasCollection(collId UniqueID) bool {
func (mt *metaTable) HasCollection(collID UniqueID) bool {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
_, ok := mt.collId2Meta[collId]
if !ok {
return false
}
return true
_, ok := mt.collID2Meta[collID]
return ok
}
func (mt *metaTable) GetCollectionByName(collectionName string) (*pb.CollectionMeta, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
vid, ok := mt.collName2Id[collectionName]
vid, ok := mt.collName2ID[collectionName]
if !ok {
return nil, errors.Errorf("can't find collection: " + collectionName)
}
col, ok := mt.collId2Meta[vid]
col, ok := mt.collID2Meta[vid]
if !ok {
return nil, errors.Errorf("can't find collection: " + collectionName)
}
return &col, nil
}
func (mt *metaTable) AddPartition(collId UniqueID, tag string) error {
func (mt *metaTable) AddPartition(collID UniqueID, tag string) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
coll, ok := mt.collId2Meta[collId]
coll, ok := mt.collID2Meta[collID]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collId, 10))
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
for _, t := range coll.PartitionTags {
@ -313,10 +310,10 @@ func (mt *metaTable) AddPartition(collId UniqueID, tag string) error {
return nil
}
func (mt *metaTable) HasPartition(collId UniqueID, tag string) bool {
func (mt *metaTable) HasPartition(collID UniqueID, tag string) bool {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
col, ok := mt.collId2Meta[collId]
col, ok := mt.collID2Meta[collID]
if !ok {
return false
}
@ -328,42 +325,42 @@ func (mt *metaTable) HasPartition(collId UniqueID, tag string) bool {
return false
}
func (mt *metaTable) DeletePartition(collId UniqueID, tag string) error {
func (mt *metaTable) DeletePartition(collID UniqueID, tag string) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
coll_meta, ok := mt.collId2Meta[collId]
collMeta, ok := mt.collID2Meta[collID]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collId, 10))
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
}
pt := make([]string, 0, len(coll_meta.PartitionTags))
for _, t := range coll_meta.PartitionTags {
pt := make([]string, 0, len(collMeta.PartitionTags))
for _, t := range collMeta.PartitionTags {
if t != tag {
pt = append(pt, t)
}
}
if len(pt) == len(coll_meta.PartitionTags) {
if len(pt) == len(collMeta.PartitionTags) {
return nil
}
to_delete_seg := make([]UniqueID, 0, len(coll_meta.SegmentIds))
seg := make([]UniqueID, 0, len(coll_meta.SegmentIds))
for _, s := range coll_meta.SegmentIds {
sm, ok := mt.segId2Meta[s]
toDeleteSeg := make([]UniqueID, 0, len(collMeta.SegmentIds))
seg := make([]UniqueID, 0, len(collMeta.SegmentIds))
for _, s := range collMeta.SegmentIds {
sm, ok := mt.segID2Meta[s]
if !ok {
return errors.Errorf("can't find segment id = %d", s)
}
if sm.PartitionTag != tag {
seg = append(seg, s)
} else {
to_delete_seg = append(to_delete_seg, s)
toDeleteSeg = append(toDeleteSeg, s)
}
}
coll_meta.PartitionTags = pt
coll_meta.SegmentIds = seg
collMeta.PartitionTags = pt
collMeta.SegmentIds = seg
err := mt.saveCollectionAndDeleteSegmentsMeta(&coll_meta, to_delete_seg)
err := mt.saveCollectionAndDeleteSegmentsMeta(&collMeta, toDeleteSeg)
if err != nil {
_ = mt.reloadFromKV()
return err
@ -374,10 +371,10 @@ func (mt *metaTable) DeletePartition(collId UniqueID, tag string) error {
func (mt *metaTable) AddSegment(seg *pb.SegmentMeta) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
collId := seg.CollectionId
coll_meta := mt.collId2Meta[collId]
coll_meta.SegmentIds = append(coll_meta.SegmentIds, seg.SegmentId)
err := mt.saveCollectionsAndSegmentsMeta(&coll_meta, seg)
collID := seg.CollectionId
collMeta := mt.collID2Meta[collID]
collMeta.SegmentIds = append(collMeta.SegmentIds, seg.SegmentId)
err := mt.saveCollectionsAndSegmentsMeta(&collMeta, seg)
if err != nil {
_ = mt.reloadFromKV()
return err
@ -385,38 +382,38 @@ func (mt *metaTable) AddSegment(seg *pb.SegmentMeta) error {
return nil
}
func (mt *metaTable) GetSegmentById(segId UniqueID) (*pb.SegmentMeta, error) {
func (mt *metaTable) GetSegmentByID(segID UniqueID) (*pb.SegmentMeta, error) {
mt.ddLock.RLock()
defer mt.ddLock.RUnlock()
sm, ok := mt.segId2Meta[segId]
sm, ok := mt.segID2Meta[segID]
if !ok {
return nil, errors.Errorf("can't find segment id = %d", segId)
return nil, errors.Errorf("can't find segment id = %d", segID)
}
return &sm, nil
}
func (mt *metaTable) DeleteSegment(segId UniqueID) error {
func (mt *metaTable) DeleteSegment(segID UniqueID) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
seg_meta, ok := mt.segId2Meta[segId]
segMeta, ok := mt.segID2Meta[segID]
if !ok {
return errors.Errorf("can't find segment. id = " + strconv.FormatInt(segId, 10))
return errors.Errorf("can't find segment. id = " + strconv.FormatInt(segID, 10))
}
coll_meta, ok := mt.collId2Meta[seg_meta.CollectionId]
collMeta, ok := mt.collID2Meta[segMeta.CollectionId]
if !ok {
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(seg_meta.CollectionId, 10))
return errors.Errorf("can't find collection. id = " + strconv.FormatInt(segMeta.CollectionId, 10))
}
for i := 0; i < len(coll_meta.SegmentIds); i++ {
if coll_meta.SegmentIds[i] == segId {
coll_meta.SegmentIds = append(coll_meta.SegmentIds[:i], coll_meta.SegmentIds[i+1:]...)
for i := 0; i < len(collMeta.SegmentIds); i++ {
if collMeta.SegmentIds[i] == segID {
collMeta.SegmentIds = append(collMeta.SegmentIds[:i], collMeta.SegmentIds[i+1:]...)
}
}
err := mt.saveCollectionAndDeleteSegmentsMeta(&coll_meta, []UniqueID{segId})
err := mt.saveCollectionAndDeleteSegmentsMeta(&collMeta, []UniqueID{segID})
if err != nil {
_ = mt.reloadFromKV()
return err
@ -424,19 +421,19 @@ func (mt *metaTable) DeleteSegment(segId UniqueID) error {
return nil
}
func (mt *metaTable) CloseSegment(segId UniqueID, closeTs Timestamp, num_rows int64) error {
func (mt *metaTable) CloseSegment(segID UniqueID, closeTs Timestamp, numRows int64) error {
mt.ddLock.Lock()
defer mt.ddLock.Unlock()
seg_meta, ok := mt.segId2Meta[segId]
segMeta, ok := mt.segID2Meta[segID]
if !ok {
return errors.Errorf("can't find segment id = " + strconv.FormatInt(segId, 10))
return errors.Errorf("can't find segment id = " + strconv.FormatInt(segID, 10))
}
seg_meta.CloseTime = closeTs
seg_meta.NumRows = num_rows
segMeta.CloseTime = closeTs
segMeta.NumRows = numRows
err := mt.saveSegmentMeta(&seg_meta)
err := mt.saveSegmentMeta(&segMeta)
if err != nil {
_ = mt.reloadFromKV()
return err

View File

@ -14,17 +14,17 @@ import (
func TestMetaTable_Collection(t *testing.T) {
cli, err := clientv3.New(clientv3.Config{Endpoints: []string{"127.0.0.1:2379"}})
assert.Nil(t, err)
etcd_kv := kv.NewEtcdKV(cli, "/etcd/test/root")
etcdKV := kv.NewEtcdKV(cli, "/etcd/test/root")
_, err = cli.Delete(context.TODO(), "/etcd/test/root", clientv3.WithPrefix())
assert.Nil(t, err)
meta, err := NewMetaTable(etcd_kv)
meta, err := NewMetaTable(etcdKV)
assert.Nil(t, err)
defer meta.client.Close()
col_meta := pb.CollectionMeta{
Id: 100,
colMeta := pb.CollectionMeta{
ID: 100,
Schema: &schemapb.CollectionSchema{
Name: "coll1",
},
@ -32,8 +32,8 @@ func TestMetaTable_Collection(t *testing.T) {
SegmentIds: []UniqueID{},
PartitionTags: []string{},
}
col_meta_2 := pb.CollectionMeta{
Id: 50,
colMeta2 := pb.CollectionMeta{
ID: 50,
Schema: &schemapb.CollectionSchema{
Name: "coll1",
},
@ -41,8 +41,8 @@ func TestMetaTable_Collection(t *testing.T) {
SegmentIds: []UniqueID{},
PartitionTags: []string{},
}
col_meta_3 := pb.CollectionMeta{
Id: 30,
colMeta3 := pb.CollectionMeta{
ID: 30,
Schema: &schemapb.CollectionSchema{
Name: "coll2",
},
@ -50,8 +50,8 @@ func TestMetaTable_Collection(t *testing.T) {
SegmentIds: []UniqueID{},
PartitionTags: []string{},
}
col_meta_4 := pb.CollectionMeta{
Id: 30,
colMeta4 := pb.CollectionMeta{
ID: 30,
Schema: &schemapb.CollectionSchema{
Name: "coll2",
},
@ -59,8 +59,8 @@ func TestMetaTable_Collection(t *testing.T) {
SegmentIds: []UniqueID{1},
PartitionTags: []string{},
}
col_meta_5 := pb.CollectionMeta{
Id: 30,
colMeta5 := pb.CollectionMeta{
ID: 30,
Schema: &schemapb.CollectionSchema{
Name: "coll2",
},
@ -68,84 +68,84 @@ func TestMetaTable_Collection(t *testing.T) {
SegmentIds: []UniqueID{1},
PartitionTags: []string{"1"},
}
seg_id_1 := pb.SegmentMeta{
segID1 := pb.SegmentMeta{
SegmentId: 200,
CollectionId: 100,
PartitionTag: "p1",
}
seg_id_2 := pb.SegmentMeta{
segID2 := pb.SegmentMeta{
SegmentId: 300,
CollectionId: 100,
PartitionTag: "p1",
}
seg_id_3 := pb.SegmentMeta{
segID3 := pb.SegmentMeta{
SegmentId: 400,
CollectionId: 100,
PartitionTag: "p2",
}
err = meta.AddCollection(&col_meta)
err = meta.AddCollection(&colMeta)
assert.Nil(t, err)
err = meta.AddCollection(&col_meta_2)
err = meta.AddCollection(&colMeta2)
assert.NotNil(t, err)
err = meta.AddCollection(&col_meta_3)
err = meta.AddCollection(&colMeta3)
assert.Nil(t, err)
err = meta.AddCollection(&col_meta_4)
err = meta.AddCollection(&colMeta4)
assert.NotNil(t, err)
err = meta.AddCollection(&col_meta_5)
err = meta.AddCollection(&colMeta5)
assert.NotNil(t, err)
has_collection := meta.HasCollection(col_meta.Id)
assert.True(t, has_collection)
err = meta.AddPartition(col_meta.Id, "p1")
hasCollection := meta.HasCollection(colMeta.ID)
assert.True(t, hasCollection)
err = meta.AddPartition(colMeta.ID, "p1")
assert.Nil(t, err)
err = meta.AddPartition(col_meta.Id, "p2")
err = meta.AddPartition(colMeta.ID, "p2")
assert.Nil(t, err)
err = meta.AddSegment(&seg_id_1)
err = meta.AddSegment(&segID1)
assert.Nil(t, err)
err = meta.AddSegment(&seg_id_2)
err = meta.AddSegment(&segID2)
assert.Nil(t, err)
err = meta.AddSegment(&seg_id_3)
err = meta.AddSegment(&segID3)
assert.Nil(t, err)
get_col_meta, err := meta.GetCollectionByName(col_meta.Schema.Name)
getColMeta, err := meta.GetCollectionByName(colMeta.Schema.Name)
assert.Nil(t, err)
assert.Equal(t, 3, len(get_col_meta.SegmentIds))
err = meta.DeleteCollection(col_meta.Id)
assert.Equal(t, 3, len(getColMeta.SegmentIds))
err = meta.DeleteCollection(colMeta.ID)
assert.Nil(t, err)
has_collection = meta.HasCollection(col_meta.Id)
assert.False(t, has_collection)
_, err = meta.GetSegmentById(seg_id_1.SegmentId)
hasCollection = meta.HasCollection(colMeta.ID)
assert.False(t, hasCollection)
_, err = meta.GetSegmentByID(segID1.SegmentId)
assert.NotNil(t, err)
_, err = meta.GetSegmentById(seg_id_2.SegmentId)
_, err = meta.GetSegmentByID(segID2.SegmentId)
assert.NotNil(t, err)
_, err = meta.GetSegmentById(seg_id_3.SegmentId)
_, err = meta.GetSegmentByID(segID3.SegmentId)
assert.NotNil(t, err)
err = meta.reloadFromKV()
assert.Nil(t, err)
assert.Equal(t, 0, len(meta.proxyId2Meta))
assert.Equal(t, 0, len(meta.tenantId2Meta))
assert.Equal(t, 1, len(meta.collName2Id))
assert.Equal(t, 1, len(meta.collId2Meta))
assert.Equal(t, 0, len(meta.segId2Meta))
assert.Equal(t, 0, len(meta.proxyID2Meta))
assert.Equal(t, 0, len(meta.tenantID2Meta))
assert.Equal(t, 1, len(meta.collName2ID))
assert.Equal(t, 1, len(meta.collID2Meta))
assert.Equal(t, 0, len(meta.segID2Meta))
err = meta.DeleteCollection(col_meta_3.Id)
err = meta.DeleteCollection(colMeta3.ID)
assert.Nil(t, err)
}
func TestMetaTable_DeletePartition(t *testing.T) {
cli, err := clientv3.New(clientv3.Config{Endpoints: []string{"127.0.0.1:2379"}})
assert.Nil(t, err)
etcd_kv := kv.NewEtcdKV(cli, "/etcd/test/root")
etcdKV := kv.NewEtcdKV(cli, "/etcd/test/root")
_, err = cli.Delete(context.TODO(), "/etcd/test/root", clientv3.WithPrefix())
assert.Nil(t, err)
meta, err := NewMetaTable(etcd_kv)
meta, err := NewMetaTable(etcdKV)
assert.Nil(t, err)
defer meta.client.Close()
col_meta := pb.CollectionMeta{
Id: 100,
colMeta := pb.CollectionMeta{
ID: 100,
Schema: &schemapb.CollectionSchema{
Name: "coll1",
},
@ -153,75 +153,75 @@ func TestMetaTable_DeletePartition(t *testing.T) {
SegmentIds: []UniqueID{},
PartitionTags: []string{},
}
seg_id_1 := pb.SegmentMeta{
segID1 := pb.SegmentMeta{
SegmentId: 200,
CollectionId: 100,
PartitionTag: "p1",
}
seg_id_2 := pb.SegmentMeta{
segID2 := pb.SegmentMeta{
SegmentId: 300,
CollectionId: 100,
PartitionTag: "p1",
}
seg_id_3 := pb.SegmentMeta{
segID3 := pb.SegmentMeta{
SegmentId: 400,
CollectionId: 100,
PartitionTag: "p2",
}
err = meta.AddCollection(&col_meta)
err = meta.AddCollection(&colMeta)
assert.Nil(t, err)
err = meta.AddPartition(col_meta.Id, "p1")
err = meta.AddPartition(colMeta.ID, "p1")
assert.Nil(t, err)
err = meta.AddPartition(col_meta.Id, "p2")
err = meta.AddPartition(colMeta.ID, "p2")
assert.Nil(t, err)
err = meta.AddSegment(&seg_id_1)
err = meta.AddSegment(&segID1)
assert.Nil(t, err)
err = meta.AddSegment(&seg_id_2)
err = meta.AddSegment(&segID2)
assert.Nil(t, err)
err = meta.AddSegment(&seg_id_3)
err = meta.AddSegment(&segID3)
assert.Nil(t, err)
after_coll_meta, err := meta.GetCollectionByName("coll1")
afterCollMeta, err := meta.GetCollectionByName("coll1")
assert.Nil(t, err)
assert.Equal(t, 2, len(after_coll_meta.PartitionTags))
assert.Equal(t, 3, len(after_coll_meta.SegmentIds))
assert.Equal(t, 2, len(afterCollMeta.PartitionTags))
assert.Equal(t, 3, len(afterCollMeta.SegmentIds))
err = meta.DeletePartition(100, "p1")
assert.Nil(t, err)
after_coll_meta, err = meta.GetCollectionByName("coll1")
afterCollMeta, err = meta.GetCollectionByName("coll1")
assert.Nil(t, err)
assert.Equal(t, 1, len(after_coll_meta.PartitionTags))
assert.Equal(t, 1, len(after_coll_meta.SegmentIds))
has_partition := meta.HasPartition(col_meta.Id, "p1")
assert.False(t, has_partition)
has_partition = meta.HasPartition(col_meta.Id, "p2")
assert.True(t, has_partition)
_, err = meta.GetSegmentById(seg_id_1.SegmentId)
assert.Equal(t, 1, len(afterCollMeta.PartitionTags))
assert.Equal(t, 1, len(afterCollMeta.SegmentIds))
hasPartition := meta.HasPartition(colMeta.ID, "p1")
assert.False(t, hasPartition)
hasPartition = meta.HasPartition(colMeta.ID, "p2")
assert.True(t, hasPartition)
_, err = meta.GetSegmentByID(segID1.SegmentId)
assert.NotNil(t, err)
_, err = meta.GetSegmentById(seg_id_2.SegmentId)
_, err = meta.GetSegmentByID(segID2.SegmentId)
assert.NotNil(t, err)
_, err = meta.GetSegmentById(seg_id_3.SegmentId)
_, err = meta.GetSegmentByID(segID3.SegmentId)
assert.Nil(t, err)
after_coll_meta, err = meta.GetCollectionByName("coll1")
afterCollMeta, err = meta.GetCollectionByName("coll1")
assert.Nil(t, err)
err = meta.reloadFromKV()
assert.Nil(t, err)
assert.Equal(t, 0, len(meta.proxyId2Meta))
assert.Equal(t, 0, len(meta.tenantId2Meta))
assert.Equal(t, 1, len(meta.collName2Id))
assert.Equal(t, 1, len(meta.collId2Meta))
assert.Equal(t, 1, len(meta.segId2Meta))
assert.Equal(t, 0, len(meta.proxyID2Meta))
assert.Equal(t, 0, len(meta.tenantID2Meta))
assert.Equal(t, 1, len(meta.collName2ID))
assert.Equal(t, 1, len(meta.collID2Meta))
assert.Equal(t, 1, len(meta.segID2Meta))
}
func TestMetaTable_Segment(t *testing.T) {
cli, err := clientv3.New(clientv3.Config{Endpoints: []string{"127.0.0.1:2379"}})
assert.Nil(t, err)
etcd_kv := kv.NewEtcdKV(cli, "/etcd/test/root")
etcdKV := kv.NewEtcdKV(cli, "/etcd/test/root")
_, err = cli.Delete(context.TODO(), "/etcd/test/root", clientv3.WithPrefix())
assert.Nil(t, err)
meta, err := NewMetaTable(etcd_kv)
meta, err := NewMetaTable(etcdKV)
assert.Nil(t, err)
defer meta.client.Close()
@ -230,8 +230,8 @@ func TestMetaTable_Segment(t *testing.T) {
err = meta.client.MultiRemove(keys)
assert.Nil(t, err)
col_meta := pb.CollectionMeta{
Id: 100,
colMeta := pb.CollectionMeta{
ID: 100,
Schema: &schemapb.CollectionSchema{
Name: "coll1",
},
@ -239,48 +239,48 @@ func TestMetaTable_Segment(t *testing.T) {
SegmentIds: []UniqueID{},
PartitionTags: []string{},
}
seg_meta := pb.SegmentMeta{
segMeta := pb.SegmentMeta{
SegmentId: 200,
CollectionId: 100,
PartitionTag: "p1",
}
err = meta.AddCollection(&col_meta)
err = meta.AddCollection(&colMeta)
assert.Nil(t, err)
err = meta.AddPartition(col_meta.Id, "p1")
err = meta.AddPartition(colMeta.ID, "p1")
assert.Nil(t, err)
err = meta.AddSegment(&seg_meta)
err = meta.AddSegment(&segMeta)
assert.Nil(t, err)
get_seg_meta, err := meta.GetSegmentById(seg_meta.SegmentId)
getSegMeta, err := meta.GetSegmentByID(segMeta.SegmentId)
assert.Nil(t, err)
assert.Equal(t, &seg_meta, get_seg_meta)
err = meta.CloseSegment(seg_meta.SegmentId, Timestamp(11), 111)
assert.Equal(t, &segMeta, getSegMeta)
err = meta.CloseSegment(segMeta.SegmentId, Timestamp(11), 111)
assert.Nil(t, err)
get_seg_meta, err = meta.GetSegmentById(seg_meta.SegmentId)
getSegMeta, err = meta.GetSegmentByID(segMeta.SegmentId)
assert.Nil(t, err)
assert.Equal(t, get_seg_meta.NumRows, int64(111))
assert.Equal(t, get_seg_meta.CloseTime, uint64(11))
err = meta.DeleteSegment(seg_meta.SegmentId)
assert.Equal(t, getSegMeta.NumRows, int64(111))
assert.Equal(t, getSegMeta.CloseTime, uint64(11))
err = meta.DeleteSegment(segMeta.SegmentId)
assert.Nil(t, err)
get_seg_meta, err = meta.GetSegmentById(seg_meta.SegmentId)
assert.Nil(t, get_seg_meta)
getSegMeta, err = meta.GetSegmentByID(segMeta.SegmentId)
assert.Nil(t, getSegMeta)
assert.NotNil(t, err)
get_col_meta, err := meta.GetCollectionByName(col_meta.Schema.Name)
getColMeta, err := meta.GetCollectionByName(colMeta.Schema.Name)
assert.Nil(t, err)
assert.Equal(t, 0, len(get_col_meta.SegmentIds))
assert.Equal(t, 0, len(getColMeta.SegmentIds))
meta.tenantId2Meta = make(map[UniqueID]pb.TenantMeta)
meta.proxyId2Meta = make(map[UniqueID]pb.ProxyMeta)
meta.collId2Meta = make(map[UniqueID]pb.CollectionMeta)
meta.collName2Id = make(map[string]UniqueID)
meta.segId2Meta = make(map[UniqueID]pb.SegmentMeta)
meta.tenantID2Meta = make(map[UniqueID]pb.TenantMeta)
meta.proxyID2Meta = make(map[UniqueID]pb.ProxyMeta)
meta.collID2Meta = make(map[UniqueID]pb.CollectionMeta)
meta.collName2ID = make(map[string]UniqueID)
meta.segID2Meta = make(map[UniqueID]pb.SegmentMeta)
err = meta.reloadFromKV()
assert.Nil(t, err)
assert.Equal(t, 0, len(meta.proxyId2Meta))
assert.Equal(t, 0, len(meta.tenantId2Meta))
assert.Equal(t, 1, len(meta.collName2Id))
assert.Equal(t, 1, len(meta.collId2Meta))
assert.Equal(t, 0, len(meta.segId2Meta))
assert.Equal(t, 0, len(meta.proxyID2Meta))
assert.Equal(t, 0, len(meta.tenantID2Meta))
assert.Equal(t, 1, len(meta.collName2ID))
assert.Equal(t, 1, len(meta.collID2Meta))
assert.Equal(t, 0, len(meta.segID2Meta))
}

View File

@ -4,6 +4,7 @@ import (
"context"
"github.com/zilliztech/milvus-distributed/internal/master/id"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
@ -31,13 +32,11 @@ func (s *Master) HasCollection(ctx context.Context, in *internalpb.HasCollection
}
func (s *Master) DescribeCollection(ctx context.Context, in *internalpb.DescribeCollectionRequest) (*servicepb.CollectionDescription, error) {
return &servicepb.CollectionDescription{
}, nil
return &servicepb.CollectionDescription{}, nil
}
func (s *Master) ShowCollections(ctx context.Context, in *internalpb.ShowCollectionRequest) (*servicepb.StringListResponse, error) {
return &servicepb.StringListResponse{
}, nil
return &servicepb.StringListResponse{}, nil
}
//////////////////////////////////////////////////////////////////////////
@ -66,7 +65,7 @@ func (s *Master) DescribePartition(ctx context.Context, in *internalpb.DescribeP
return &servicepb.PartitionDescription{}, nil
}
func (s *Master) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) {
func (s *Master) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) {
return &servicepb.StringListResponse{}, nil
}
@ -92,7 +91,6 @@ func (s *Master) AllocTimestamp(ctx context.Context, request *internalpb.TsoRequ
}
func (s *Master) AllocId(ctx context.Context, request *internalpb.IdRequest) (*internalpb.IdResponse, error) {
panic("implement me")
count := request.GetCount()
ts, err := id.AllocOne()
@ -104,9 +102,9 @@ func (s *Master) AllocId(ctx context.Context, request *internalpb.IdRequest) (*i
response := &internalpb.IdResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
Id: ts,
ID: ts,
Count: count,
}
return response, nil
}
}

View File

@ -3,11 +3,6 @@ package mockmaster
import (
"context"
"fmt"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/kv/mockkv"
"github.com/zilliztech/milvus-distributed/internal/master/id"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
"google.golang.org/grpc"
"log"
"math/rand"
"net"
@ -15,11 +10,17 @@ import (
"sync/atomic"
"time"
"google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/kv/mockkv"
"github.com/zilliztech/milvus-distributed/internal/master/id"
"github.com/zilliztech/milvus-distributed/internal/master/tso"
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
)
const (
MOCK_GRPC_PORT=":0"
MOCKGRPCPORT = ":0"
)
var GrpcServerAddr net.Addr
@ -39,7 +40,7 @@ type Master struct {
// for tso.
tsoAllocator tso.Allocator
kvBase kv.KVBase
kvBase kv.Base
// Add callback functions at different stages
startCallbacks []func()
@ -51,11 +52,11 @@ type Master struct {
// CreateServer creates the UNINITIALIZED pd server with given configuration.
func CreateServer(ctx context.Context) (*Master, error) {
rand.Seed(time.Now().UnixNano())
id.InitGlobalIdAllocator("idTimestamp", mockkv.NewEtcdKV())
id.InitGlobalIDAllocator("idTimestamp", mockkv.NewEtcdKV())
m := &Master{
ctx: ctx,
kvBase: mockkv.NewEtcdKV(),
ctx: ctx,
kvBase: mockkv.NewEtcdKV(),
tsoAllocator: tso.NewGlobalTSOAllocator("timestamp", mockkv.NewEtcdKV()),
}
@ -70,7 +71,7 @@ func (s *Master) AddStartCallback(callbacks ...func()) {
}
// for unittest, get the grpc server addr
func (s *Master) GetGRPCAddr() net.Addr{
func (s *Master) GetGRPCAddr() net.Addr {
return s.grpcAddr
}
@ -150,17 +151,16 @@ func (s *Master) startServerLoop(ctx context.Context) {
}
func (s *Master) stopServerLoop() {
if s.grpcServer != nil{
if s.grpcServer != nil {
s.grpcServer.GracefulStop()
}
s.serverLoopCancel()
s.serverLoopWg.Wait()
}
func (s *Master) grpcLoop() {
defer s.serverLoopWg.Done()
lis, err := net.Listen("tcp", MOCK_GRPC_PORT)
lis, err := net.Listen("tcp", MOCKGRPCPORT)
if err != nil {
log.Printf("failed to listen: %v", err)
return
@ -181,26 +181,18 @@ func (s *Master) pulsarLoop() {
ctx, cancel := context.WithCancel(s.serverLoopCtx)
defer cancel()
for {
select {
case <-ctx.Done():
log.Print("server is closed, exit pulsar loop")
return
}
}
<-ctx.Done()
log.Print("server is closed, exit pulsar loop")
}
func (s *Master) tasksExecutionLoop() {
defer s.serverLoopWg.Done()
ctx, _ := context.WithCancel(s.serverLoopCtx)
ctx, cancel := context.WithCancel(s.serverLoopCtx)
defer cancel()
for {
select {
case <-ctx.Done():
log.Print("server is closed, exit task execution loop")
return
}
}
<-ctx.Done()
log.Print("server is closed, exit task execution loop")
}
func (s *Master) segmentStatisticsLoop() {
@ -209,11 +201,6 @@ func (s *Master) segmentStatisticsLoop() {
ctx, cancel := context.WithCancel(s.serverLoopCtx)
defer cancel()
for {
select {
case <-ctx.Done():
log.Print("server is closed, exit segmentStatistics loop")
return
}
}
<-ctx.Done()
log.Print("server is closed, exit segmentStatistics loop")
}

View File

@ -71,13 +71,13 @@ func (t *createPartitionTask) Execute() error {
collectionMeta.PartitionTags = append(collectionMeta.PartitionTags, partitionName.Tag)
collectionJson, err := json.Marshal(&collectionMeta)
collectionJSON, err := json.Marshal(&collectionMeta)
if err != nil {
return err
}
collectionId := collectionMeta.Id
err = (*t.kvBase).Save(partitionMetaPrefix+strconv.FormatInt(collectionId, 10), string(collectionJson))
collectionID := collectionMeta.ID
err = (*t.kvBase).Save(partitionMetaPrefix+strconv.FormatInt(collectionID, 10), string(collectionJSON))
if err != nil {
return err
}
@ -113,18 +113,18 @@ func (t *dropPartitionTask) Execute() error {
return err
}
err = t.mt.DeletePartition(collectionMeta.Id, partitionName.Tag)
err = t.mt.DeletePartition(collectionMeta.ID, partitionName.Tag)
if err != nil {
return err
}
collectionJson, err := json.Marshal(&collectionMeta)
collectionJSON, err := json.Marshal(&collectionMeta)
if err != nil {
return err
}
collectionId := collectionMeta.Id
err = (*t.kvBase).Save(partitionMetaPrefix+strconv.FormatInt(collectionId, 10), string(collectionJson))
collectionID := collectionMeta.ID
err = (*t.kvBase).Save(partitionMetaPrefix+strconv.FormatInt(collectionID, 10), string(collectionJSON))
if err != nil {
return err
}
@ -160,7 +160,7 @@ func (t *hasPartitionTask) Execute() error {
return err
}
t.hasPartition = t.mt.HasPartition(collectionMeta.Id, partitionName.Tag)
t.hasPartition = t.mt.HasPartition(collectionMeta.ID, partitionName.Tag)
return nil
}
@ -222,10 +222,8 @@ func (t *showPartitionTask) Execute() error {
}
partitions := make([]string, 0)
for _, collection := range t.mt.collId2Meta {
for _, partition := range collection.PartitionTags {
partitions = append(partitions, partition)
}
for _, collection := range t.mt.collID2Meta {
partitions = append(partitions, collection.PartitionTags...)
}
stringListResponse := servicepb.StringListResponse{

View File

@ -42,15 +42,14 @@ func Init() {
InitGlobalTsoAllocator("timestamp", tsoutil.NewTSOKVBase("tso"))
}
func InitGlobalTsoAllocator(key string, base kv.KVBase) {
func InitGlobalTsoAllocator(key string, base kv.Base) {
allocator = NewGlobalTSOAllocator(key, base)
allocator.Initialize()
}
// NewGlobalTSOAllocator creates a new global TSO allocator.
func NewGlobalTSOAllocator(key string, kvBase kv.KVBase) *GlobalTSOAllocator {
var saveInterval time.Duration = 3 * time.Second
func NewGlobalTSOAllocator(key string, kvBase kv.Base) *GlobalTSOAllocator {
var saveInterval = 3 * time.Second
return &GlobalTSOAllocator{
tso: &timestampOracle{
kvBase: kvBase,
@ -79,7 +78,7 @@ func (gta *GlobalTSOAllocator) SetTSO(tso uint64) error {
// GenerateTSO is used to generate a given number of TSOs.
// Make sure you have initialized the TSO allocator before calling.
func (gta *GlobalTSOAllocator) GenerateTSO(count uint32) (uint64, error) {
var physical, logical int64 = 0, 0
var physical, logical int64
if count == 0 {
return 0, errors.New("tso count should be positive")
}

View File

@ -6,6 +6,7 @@ import (
"time"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
)

View File

@ -47,7 +47,7 @@ type atomicObject struct {
// timestampOracle is used to maintain the logic of tso.
type timestampOracle struct {
key string
kvBase kv.KVBase
kvBase kv.Base
// TODO: remove saveInterval
saveInterval time.Duration

View File

@ -2,15 +2,15 @@ package msgstream
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/errors"
"log"
"sync"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/errors"
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/apache/pulsar-client-go/pulsar"
"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
)
@ -37,22 +37,22 @@ type MsgStream interface {
}
type PulsarMsgStream struct {
ctx context.Context
client *pulsar.Client
producers []*pulsar.Producer
consumers []*pulsar.Consumer
repackFunc RepackFunc
unmarshal *UnmarshalDispatcher
receiveBuf chan *MsgPack
wait *sync.WaitGroup
streamCancel func()
ctx context.Context
client *pulsar.Client
producers []*pulsar.Producer
consumers []*pulsar.Consumer
repackFunc RepackFunc
unmarshal *UnmarshalDispatcher
receiveBuf chan *MsgPack
wait *sync.WaitGroup
streamCancel func()
}
func NewPulsarMsgStream(ctx context.Context, receiveBufSize int64) *PulsarMsgStream {
streamCtx, streamCancel := context.WithCancel(ctx)
stream := &PulsarMsgStream{
ctx: streamCtx,
streamCancel: streamCancel,
ctx: streamCtx,
streamCancel: streamCancel,
}
stream.receiveBuf = make(chan *MsgPack, receiveBufSize)
return stream
@ -271,8 +271,8 @@ type PulsarTtMsgStream struct {
func NewPulsarTtMsgStream(ctx context.Context, receiveBufSize int64) *PulsarTtMsgStream {
streamCtx, streamCancel := context.WithCancel(ctx)
pulsarMsgStream := PulsarMsgStream{
ctx: streamCtx,
streamCancel: streamCancel,
ctx: streamCtx,
streamCancel: streamCancel,
}
pulsarMsgStream.receiveBuf = make(chan *MsgPack, receiveBufSize)
return &PulsarTtMsgStream{
@ -410,16 +410,16 @@ func insertRepackFunc(tsMsgs []*TsMsg, hashKeys [][]int32) (map[int32]*MsgPack,
}
sliceRequest := internalPb.InsertRequest{
MsgType: internalPb.MsgType_kInsert,
ReqId: insertRequest.ReqId,
MsgType: internalPb.MsgType_kInsert,
ReqId: insertRequest.ReqId,
CollectionName: insertRequest.CollectionName,
PartitionTag: insertRequest.PartitionTag,
SegmentId: insertRequest.SegmentId,
ChannelId: insertRequest.ChannelId,
ProxyId: insertRequest.ProxyId,
Timestamps: []uint64{insertRequest.Timestamps[index]},
RowIds: []int64{insertRequest.RowIds[index]},
RowData: []*commonPb.Blob{insertRequest.RowData[index]},
PartitionTag: insertRequest.PartitionTag,
SegmentId: insertRequest.SegmentId,
ChannelId: insertRequest.ChannelId,
ProxyId: insertRequest.ProxyId,
Timestamps: []uint64{insertRequest.Timestamps[index]},
RowIds: []int64{insertRequest.RowIds[index]},
RowData: []*commonPb.Blob{insertRequest.RowData[index]},
}
var msg TsMsg = &InsertMsg{
@ -457,13 +457,13 @@ func deleteRepackFunc(tsMsgs []*TsMsg, hashKeys [][]int32) (map[int32]*MsgPack,
}
sliceRequest := internalPb.DeleteRequest{
MsgType: internalPb.MsgType_kDelete,
ReqId: deleteRequest.ReqId,
MsgType: internalPb.MsgType_kDelete,
ReqId: deleteRequest.ReqId,
CollectionName: deleteRequest.CollectionName,
ChannelId: deleteRequest.ChannelId,
ProxyId: deleteRequest.ProxyId,
Timestamps: []uint64{deleteRequest.Timestamps[index]},
PrimaryKeys: []int64{deleteRequest.PrimaryKeys[index]},
ChannelId: deleteRequest.ChannelId,
ProxyId: deleteRequest.ProxyId,
Timestamps: []uint64{deleteRequest.Timestamps[index]},
PrimaryKeys: []int64{deleteRequest.PrimaryKeys[index]},
}
var msg TsMsg = &DeleteMsg{

View File

@ -4,11 +4,12 @@ import (
"context"
"errors"
"fmt"
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"log"
"testing"
"github.com/golang/protobuf/proto"
commonPb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)

View File

@ -5,10 +5,11 @@ package etcdpb
import (
fmt "fmt"
math "math"
proto "github.com/golang/protobuf/proto"
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
schemapb "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
math "math"
)
// Reference imports to suppress errors if they are not otherwise used.
@ -23,7 +24,7 @@ var _ = math.Inf
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
type TenantMeta struct {
Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
ID int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
NumQueryNodes int64 `protobuf:"varint,2,opt,name=num_query_nodes,json=numQueryNodes,proto3" json:"num_query_nodes,omitempty"`
InsertChannelIds []string `protobuf:"bytes,3,rep,name=insert_channel_ids,json=insertChannelIds,proto3" json:"insert_channel_ids,omitempty"`
QueryChannelId string `protobuf:"bytes,4,opt,name=query_channel_id,json=queryChannelId,proto3" json:"query_channel_id,omitempty"`
@ -59,7 +60,7 @@ var xxx_messageInfo_TenantMeta proto.InternalMessageInfo
func (m *TenantMeta) GetId() int64 {
if m != nil {
return m.Id
return m.ID
}
return 0
}
@ -86,7 +87,7 @@ func (m *TenantMeta) GetQueryChannelId() string {
}
type ProxyMeta struct {
Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
ID int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
Address *commonpb.Address `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"`
ResultChannelIds []string `protobuf:"bytes,3,rep,name=result_channel_ids,json=resultChannelIds,proto3" json:"result_channel_ids,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -121,7 +122,7 @@ var xxx_messageInfo_ProxyMeta proto.InternalMessageInfo
func (m *ProxyMeta) GetId() int64 {
if m != nil {
return m.Id
return m.ID
}
return 0
}
@ -141,7 +142,7 @@ func (m *ProxyMeta) GetResultChannelIds() []string {
}
type CollectionMeta struct {
Id int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
ID int64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"`
CreateTime uint64 `protobuf:"varint,3,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"`
SegmentIds []int64 `protobuf:"varint,4,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"`
@ -178,7 +179,7 @@ var xxx_messageInfo_CollectionMeta proto.InternalMessageInfo
func (m *CollectionMeta) GetId() int64 {
if m != nil {
return m.Id
return m.ID
}
return 0
}
@ -213,7 +214,7 @@ func (m *CollectionMeta) GetPartitionTags() []string {
type SegmentMeta struct {
SegmentId int64 `protobuf:"varint,1,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
CollectionId int64 `protobuf:"varint,2,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
CollectionId int64 `protobuf:"varint,2,opt,name=collection_id,json=collectionID,proto3" json:"collection_id,omitempty"`
PartitionTag string `protobuf:"bytes,3,opt,name=partition_tag,json=partitionTag,proto3" json:"partition_tag,omitempty"`
ChannelStart int32 `protobuf:"varint,4,opt,name=channel_start,json=channelStart,proto3" json:"channel_start,omitempty"`
ChannelEnd int32 `protobuf:"varint,5,opt,name=channel_end,json=channelEnd,proto3" json:"channel_end,omitempty"`

View File

@ -5,10 +5,11 @@ package internalpb
import (
fmt "fmt"
math "math"
proto "github.com/golang/protobuf/proto"
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
servicepb "github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
math "math"
)
// Reference imports to suppress errors if they are not otherwise used.
@ -126,7 +127,7 @@ func (PeerRole) EnumDescriptor() ([]byte, []int) {
}
type IdRequest struct {
PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"`
PeerId int64 `protobuf:"varint,1,opt,name=peerID,json=peerId,proto3" json:"peerID,omitempty"`
Role PeerRole `protobuf:"varint,2,opt,name=role,proto3,enum=milvus.proto.internal.PeerRole" json:"role,omitempty"`
Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -182,7 +183,7 @@ func (m *IdRequest) GetCount() uint32 {
type IdResponse struct {
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
Id int64 `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"`
ID int64 `protobuf:"varint,2,opt,name=id,proto3" json:"id,omitempty"`
Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@ -223,7 +224,7 @@ func (m *IdResponse) GetStatus() *commonpb.Status {
func (m *IdResponse) GetId() int64 {
if m != nil {
return m.Id
return m.ID
}
return 0
}
@ -236,7 +237,7 @@ func (m *IdResponse) GetCount() uint32 {
}
type TsoRequest struct {
PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"`
PeerId int64 `protobuf:"varint,1,opt,name=peerID,json=peerId,proto3" json:"peerID,omitempty"`
Role PeerRole `protobuf:"varint,2,opt,name=role,proto3,enum=milvus.proto.internal.PeerRole" json:"role,omitempty"`
Count uint32 `protobuf:"varint,3,opt,name=count,proto3" json:"count,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -347,7 +348,7 @@ func (m *TsoResponse) GetCount() uint32 {
type CreateCollectionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
Schema *commonpb.Blob `protobuf:"bytes,5,opt,name=schema,proto3" json:"schema,omitempty"`
@ -388,7 +389,7 @@ func (m *CreateCollectionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *CreateCollectionRequest) GetReqId() int64 {
func (m *CreateCollectionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -418,7 +419,7 @@ func (m *CreateCollectionRequest) GetSchema() *commonpb.Blob {
type DropCollectionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
CollectionName *servicepb.CollectionName `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
@ -459,7 +460,7 @@ func (m *DropCollectionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *DropCollectionRequest) GetReqId() int64 {
func (m *DropCollectionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -489,7 +490,7 @@ func (m *DropCollectionRequest) GetCollectionName() *servicepb.CollectionName {
type HasCollectionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
CollectionName *servicepb.CollectionName `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
@ -530,7 +531,7 @@ func (m *HasCollectionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *HasCollectionRequest) GetReqId() int64 {
func (m *HasCollectionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -560,7 +561,7 @@ func (m *HasCollectionRequest) GetCollectionName() *servicepb.CollectionName {
type DescribeCollectionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
CollectionName *servicepb.CollectionName `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
@ -601,7 +602,7 @@ func (m *DescribeCollectionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *DescribeCollectionRequest) GetReqId() int64 {
func (m *DescribeCollectionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -631,7 +632,7 @@ func (m *DescribeCollectionRequest) GetCollectionName() *servicepb.CollectionNam
type ShowCollectionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
@ -671,7 +672,7 @@ func (m *ShowCollectionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *ShowCollectionRequest) GetReqId() int64 {
func (m *ShowCollectionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -694,7 +695,7 @@ func (m *ShowCollectionRequest) GetProxyId() int64 {
type CreatePartitionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
@ -735,7 +736,7 @@ func (m *CreatePartitionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *CreatePartitionRequest) GetReqId() int64 {
func (m *CreatePartitionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -765,7 +766,7 @@ func (m *CreatePartitionRequest) GetPartitionName() *servicepb.PartitionName {
type DropPartitionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
@ -806,7 +807,7 @@ func (m *DropPartitionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *DropPartitionRequest) GetReqId() int64 {
func (m *DropPartitionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -836,7 +837,7 @@ func (m *DropPartitionRequest) GetPartitionName() *servicepb.PartitionName {
type HasPartitionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
@ -877,7 +878,7 @@ func (m *HasPartitionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *HasPartitionRequest) GetReqId() int64 {
func (m *HasPartitionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -907,7 +908,7 @@ func (m *HasPartitionRequest) GetPartitionName() *servicepb.PartitionName {
type DescribePartitionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
@ -948,7 +949,7 @@ func (m *DescribePartitionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *DescribePartitionRequest) GetReqId() int64 {
func (m *DescribePartitionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -978,7 +979,7 @@ func (m *DescribePartitionRequest) GetPartitionName() *servicepb.PartitionName {
type ShowPartitionRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
CollectionName *servicepb.CollectionName `protobuf:"bytes,5,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
@ -1019,7 +1020,7 @@ func (m *ShowPartitionRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *ShowPartitionRequest) GetReqId() int64 {
func (m *ShowPartitionRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -1049,7 +1050,7 @@ func (m *ShowPartitionRequest) GetCollectionName() *servicepb.CollectionName {
type InsertRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
PartitionTag string `protobuf:"bytes,4,opt,name=partition_tag,json=partitionTag,proto3" json:"partition_tag,omitempty"`
SegmentId int64 `protobuf:"varint,5,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
@ -1095,7 +1096,7 @@ func (m *InsertRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *InsertRequest) GetReqId() int64 {
func (m *InsertRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -1160,7 +1161,7 @@ func (m *InsertRequest) GetRowData() []*commonpb.Blob {
type DeleteRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
ChannelId int64 `protobuf:"varint,4,opt,name=channel_id,json=channelId,proto3" json:"channel_id,omitempty"`
ProxyId int64 `protobuf:"varint,5,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
@ -1203,7 +1204,7 @@ func (m *DeleteRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *DeleteRequest) GetReqId() int64 {
func (m *DeleteRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -1247,7 +1248,7 @@ func (m *DeleteRequest) GetPrimaryKeys() []int64 {
type SearchRequest struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,2,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
ProxyId int64 `protobuf:"varint,3,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
Timestamp uint64 `protobuf:"varint,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
ResultChannelId int64 `protobuf:"varint,5,opt,name=result_channel_id,json=resultChannelId,proto3" json:"result_channel_id,omitempty"`
@ -1289,7 +1290,7 @@ func (m *SearchRequest) GetMsgType() MsgType {
return MsgType_kNone
}
func (m *SearchRequest) GetReqId() int64 {
func (m *SearchRequest) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -1327,7 +1328,7 @@ func (m *SearchRequest) GetQuery() *commonpb.Blob {
type SearchResult struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
ReqId int64 `protobuf:"varint,3,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,3,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"`
QueryNodeId int64 `protobuf:"varint,5,opt,name=query_node_id,json=queryNodeId,proto3" json:"query_node_id,omitempty"`
Timestamp uint64 `protobuf:"varint,6,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
@ -1377,7 +1378,7 @@ func (m *SearchResult) GetStatus() *commonpb.Status {
return nil
}
func (m *SearchResult) GetReqId() int64 {
func (m *SearchResult) GetReqID() int64 {
if m != nil {
return m.ReqId
}
@ -1421,7 +1422,7 @@ func (m *SearchResult) GetHits() []*servicepb.Hits {
type TimeTickMsg struct {
MsgType MsgType `protobuf:"varint,1,opt,name=msg_type,json=msgType,proto3,enum=milvus.proto.internal.MsgType" json:"msg_type,omitempty"`
PeerId int64 `protobuf:"varint,2,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"`
PeerId int64 `protobuf:"varint,2,opt,name=peerID,json=peerId,proto3" json:"peerID,omitempty"`
Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@ -1546,7 +1547,7 @@ func (m *Key2Seg) GetSegmentIds() []int64 {
}
type Key2SegMsg struct {
ReqId int64 `protobuf:"varint,1,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"`
ReqId int64 `protobuf:"varint,1,opt,name=req_id,json=reqID,proto3" json:"req_id,omitempty"`
Key2Seg []*Key2Seg `protobuf:"bytes,2,rep,name=key2seg,proto3" json:"key2seg,omitempty"`
XXX_NoUnkeyedLiteral struct{} `json:"-"`
XXX_unrecognized []byte `json:"-"`
@ -1578,7 +1579,7 @@ func (m *Key2SegMsg) XXX_DiscardUnknown() {
var xxx_messageInfo_Key2SegMsg proto.InternalMessageInfo
func (m *Key2SegMsg) GetReqId() int64 {
func (m *Key2SegMsg) GetReqID() int64 {
if m != nil {
return m.ReqId
}

View File

@ -3,10 +3,11 @@ package proxy
import (
"context"
"errors"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"log"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
@ -44,7 +45,7 @@ func (p *Proxy) Insert(ctx context.Context, in *servicepb.RowBatch) (*servicepb.
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "insert timeout!",
},
}, errors.New("insert timeout!")
}, errors.New("insert timeout")
case result := <-it.resultChan:
return result, nil
}
@ -73,7 +74,7 @@ func (p *Proxy) CreateCollection(ctx context.Context, req *schemapb.CollectionSc
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "create collection timeout!",
}, errors.New("create collection timeout!")
}, errors.New("create collection timeout")
case result := <-cct.resultChan:
return result, nil
}
@ -105,7 +106,7 @@ func (p *Proxy) Search(ctx context.Context, req *servicepb.Query) (*servicepb.Qu
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "query timeout!",
},
}, errors.New("query timeout!")
}, errors.New("query timeout")
case result := <-qt.resultChan:
return result, nil
}
@ -132,7 +133,7 @@ func (p *Proxy) DropCollection(ctx context.Context, req *servicepb.CollectionNam
return &commonpb.Status{
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "create collection timeout!",
}, errors.New("create collection timeout!")
}, errors.New("create collection timeout")
case result := <-dct.resultChan:
return result, nil
}
@ -162,7 +163,7 @@ func (p *Proxy) HasCollection(ctx context.Context, req *servicepb.CollectionName
Reason: "has collection timeout!",
},
Value: false,
}, errors.New("has collection timeout!")
}, errors.New("has collection timeout")
case result := <-hct.resultChan:
return result, nil
}
@ -191,7 +192,7 @@ func (p *Proxy) DescribeCollection(ctx context.Context, req *servicepb.Collectio
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "describe collection timeout!",
},
}, errors.New("describe collection timeout!")
}, errors.New("describe collection timeout")
case result := <-dct.resultChan:
return result, nil
}
@ -219,7 +220,7 @@ func (p *Proxy) ShowCollections(ctx context.Context, req *commonpb.Empty) (*serv
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
Reason: "show collections timeout!",
},
}, errors.New("show collections timeout!")
}, errors.New("show collections timeout")
case result := <-sct.resultChan:
return result, nil
}

View File

@ -2,13 +2,14 @@ package proxy
import (
"context"
"google.golang.org/grpc"
"log"
"math/rand"
"net"
"sync"
"time"
"google.golang.org/grpc"
"github.com/zilliztech/milvus-distributed/internal/allocator"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
@ -31,7 +32,7 @@ type Proxy struct {
taskSch *TaskScheduler
tick *timeTick
idAllocator *allocator.IdAllocator
idAllocator *allocator.IDAllocator
tsoAllocator *allocator.TimestampAllocator
manipulationMsgStream *msgstream.PulsarMsgStream
@ -55,7 +56,7 @@ func CreateProxy(ctx context.Context) (*Proxy, error) {
p.queryMsgStream = msgstream.NewPulsarMsgStream(p.proxyLoopCtx, bufSize)
p.queryResultMsgStream = msgstream.NewPulsarMsgStream(p.proxyLoopCtx, bufSize)
idAllocator, err := allocator.NewIdAllocator(p.proxyLoopCtx)
idAllocator, err := allocator.NewIDAllocator(p.proxyLoopCtx)
if err != nil {
return nil, err
@ -102,8 +103,8 @@ func (p *Proxy) startProxy() error {
}
// AddCloseCallback adds a callback in the Close phase.
func (s *Proxy) AddCloseCallback(callbacks ...func()) {
s.closeCallbacks = append(s.closeCallbacks, callbacks...)
func (p *Proxy) AddCloseCallback(callbacks ...func()) {
p.closeCallbacks = append(p.closeCallbacks, callbacks...)
}
func (p *Proxy) grpcLoop() {
@ -124,7 +125,9 @@ func (p *Proxy) grpcLoop() {
func (p *Proxy) connectMaster() error {
log.Printf("Connected to master, master_addr=%s", "127.0.0.1:5053")
ctx, _ := context.WithTimeout(context.Background(), 2*time.Second)
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
conn, err := grpc.DialContext(ctx, "127.0.0.1:5053", grpc.WithInsecure(), grpc.WithBlock())
if err != nil {
log.Printf("Connect to master failed, error= %v", err)
@ -148,18 +151,18 @@ func (p *Proxy) queryResultLoop() {
}
tsMsg := msgPack.Msgs[0]
searchResultMsg, _ := (*tsMsg).(*msgstream.SearchResultMsg)
reqId := searchResultMsg.GetReqId()
_, ok := queryResultBuf[reqId]
reqID := searchResultMsg.GetReqID()
_, ok := queryResultBuf[reqID]
if !ok {
queryResultBuf[reqId] = make([]*internalpb.SearchResult, 0)
queryResultBuf[reqID] = make([]*internalpb.SearchResult, 0)
}
queryResultBuf[reqId] = append(queryResultBuf[reqId], &searchResultMsg.SearchResult)
if len(queryResultBuf[reqId]) == 4 {
queryResultBuf[reqID] = append(queryResultBuf[reqID], &searchResultMsg.SearchResult)
if len(queryResultBuf[reqID]) == 4 {
// TODO: use the number of query node instead
t := p.taskSch.getTaskByReqId(reqId)
t := p.taskSch.getTaskByReqID(reqID)
qt := t.(*QueryTask)
qt.resultBuf <- queryResultBuf[reqId]
delete(queryResultBuf, reqId)
qt.resultBuf <- queryResultBuf[reqID]
delete(queryResultBuf, reqID)
}
}
}

View File

@ -13,7 +13,7 @@ import (
)
type task interface {
Id() UniqueID // return ReqId
ID() UniqueID // return ReqId
Type() internalpb.MsgType
BeginTs() Timestamp
EndTs() Timestamp
@ -49,7 +49,7 @@ func (it *InsertTask) EndTs() Timestamp {
return it.ts
}
func (it *InsertTask) Id() UniqueID {
func (it *InsertTask) ID() UniqueID {
return it.ReqId
}
@ -85,7 +85,7 @@ func (it *InsertTask) WaitToFinish() error {
return err
case <-it.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout")
}
}
}
@ -103,7 +103,7 @@ type CreateCollectionTask struct {
cancel context.CancelFunc
}
func (cct *CreateCollectionTask) Id() UniqueID {
func (cct *CreateCollectionTask) ID() UniqueID {
return cct.ReqId
}
@ -153,7 +153,7 @@ func (cct *CreateCollectionTask) WaitToFinish() error {
return err
case <-cct.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout")
}
}
}
@ -171,7 +171,7 @@ type DropCollectionTask struct {
cancel context.CancelFunc
}
func (dct *DropCollectionTask) Id() UniqueID {
func (dct *DropCollectionTask) ID() UniqueID {
return dct.ReqId
}
@ -221,7 +221,7 @@ func (dct *DropCollectionTask) WaitToFinish() error {
return err
case <-dct.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout")
}
}
}
@ -240,7 +240,7 @@ type QueryTask struct {
cancel context.CancelFunc
}
func (qt *QueryTask) Id() UniqueID {
func (qt *QueryTask) ID() UniqueID {
return qt.ReqId
}
@ -294,7 +294,7 @@ func (qt *QueryTask) WaitToFinish() error {
return err
case <-qt.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout")
}
}
}
@ -368,7 +368,7 @@ type HasCollectionTask struct {
cancel context.CancelFunc
}
func (hct *HasCollectionTask) Id() UniqueID {
func (hct *HasCollectionTask) ID() UniqueID {
return hct.ReqId
}
@ -421,7 +421,7 @@ func (hct *HasCollectionTask) WaitToFinish() error {
return err
case <-hct.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout")
}
}
}
@ -439,7 +439,7 @@ type DescribeCollectionTask struct {
cancel context.CancelFunc
}
func (dct *DescribeCollectionTask) Id() UniqueID {
func (dct *DescribeCollectionTask) ID() UniqueID {
return dct.ReqId
}
@ -491,7 +491,7 @@ func (dct *DescribeCollectionTask) WaitToFinish() error {
return err
case <-dct.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout")
}
}
}
@ -509,7 +509,7 @@ type ShowCollectionsTask struct {
cancel context.CancelFunc
}
func (sct *ShowCollectionsTask) Id() UniqueID {
func (sct *ShowCollectionsTask) ID() UniqueID {
return sct.ReqId
}
@ -561,7 +561,7 @@ func (sct *ShowCollectionsTask) WaitToFinish() error {
return err
case <-sct.ctx.Done():
log.Print("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout!")
return errors.New("wait to finish failed, timeout")
}
}
}

View File

@ -74,11 +74,11 @@ func (queue *BaseTaskQueue) PopActiveTask(ts Timestamp) task {
return nil
}
func (queue *BaseTaskQueue) getTaskByReqId(reqId UniqueID) task {
func (queue *BaseTaskQueue) getTaskByReqID(reqId UniqueID) task {
queue.utLock.Lock()
defer queue.utLock.Lock()
for e := queue.unissuedTasks.Front(); e != nil; e = e.Next() {
if e.Value.(task).Id() == reqId {
if e.Value.(task).ID() == reqId {
return e.Value.(task)
}
}
@ -86,7 +86,7 @@ func (queue *BaseTaskQueue) getTaskByReqId(reqId UniqueID) task {
queue.atLock.Lock()
defer queue.atLock.Unlock()
for ats := range queue.activeTasks {
if queue.activeTasks[ats].Id() == reqId {
if queue.activeTasks[ats].ID() == reqId {
return queue.activeTasks[ats]
}
}
@ -179,7 +179,7 @@ type TaskScheduler struct {
DmQueue *DmTaskQueue
DqQueue *DqTaskQueue
idAllocator *allocator.IdAllocator
idAllocator *allocator.IDAllocator
tsoAllocator *allocator.TimestampAllocator
wg sync.WaitGroup
@ -188,7 +188,7 @@ type TaskScheduler struct {
}
func NewTaskScheduler(ctx context.Context,
idAllocator *allocator.IdAllocator,
idAllocator *allocator.IDAllocator,
tsoAllocator *allocator.TimestampAllocator) (*TaskScheduler, error) {
ctx1, cancel := context.WithCancel(ctx)
s := &TaskScheduler{
@ -216,14 +216,14 @@ func (sched *TaskScheduler) scheduleDqTask() task {
return sched.DqQueue.PopUnissuedTask()
}
func (sched *TaskScheduler) getTaskByReqId(reqId UniqueID) task {
if t := sched.DdQueue.getTaskByReqId(reqId); t != nil {
func (sched *TaskScheduler) getTaskByReqID(reqID UniqueID) task {
if t := sched.DdQueue.getTaskByReqID(reqID); t != nil {
return t
}
if t := sched.DmQueue.getTaskByReqId(reqId); t != nil {
if t := sched.DmQueue.getTaskByReqID(reqID); t != nil {
return t
}
if t := sched.DqQueue.getTaskByReqId(reqId); t != nil {
if t := sched.DqQueue.getTaskByReqID(reqID); t != nil {
return t
}
return nil

View File

@ -26,7 +26,8 @@ func TestTimeTick(t *testing.T) {
})
assert.Nil(t, err)
ctx, _ := context.WithTimeout(context.Background(), 4*time.Second)
ctx, cancel := context.WithTimeout(context.Background(), 4*time.Second)
defer cancel()
//var curTs Timestamp
//curTs = 0
@ -39,7 +40,9 @@ func TestTimeTick(t *testing.T) {
}
tt.Start()
ctx2, _ := context.WithTimeout(context.Background(), time.Second*2)
ctx2, cancel2 := context.WithTimeout(context.Background(), time.Second*2)
defer cancel2()
isbreak := false
for {
if isbreak {

View File

@ -12,9 +12,10 @@ package reader
*/
import "C"
import (
"strconv"
"github.com/zilliztech/milvus-distributed/internal/errors"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"strconv"
)
// TODO: rename
@ -120,9 +121,9 @@ func (container *ColSegContainer) removePartition(partition *Partition) error {
if hasPartition && targetCollection != nil {
*targetCollection.Partitions() = tmpPartitions
return nil
} else {
return errors.New("cannot found partition, tag = " + partition.Tag())
}
return errors.New("cannot found partition, tag = " + partition.Tag())
}
func (container *ColSegContainer) getPartitionByTag(partitionTag string) (*Partition, error) {
@ -186,9 +187,9 @@ func (container *ColSegContainer) removeSegment(segment *Segment) error {
if hasSegment && targetPartition != nil {
*targetPartition.Segments() = tmpSegments
return nil
} else {
return errors.New("cannot found segment, id = " + strconv.FormatInt(segment.ID(), 10))
}
return errors.New("cannot found segment, id = " + strconv.FormatInt(segment.ID(), 10))
}
func (container *ColSegContainer) getSegmentByID(segmentID int64) (*Segment, error) {

View File

@ -2,12 +2,13 @@ package reader
import (
"context"
"testing"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"testing"
)
//----------------------------------------------------------------------------------------------------- collection
@ -46,7 +47,7 @@ func TestColSegContainer_addCollection(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -59,7 +60,7 @@ func TestColSegContainer_addCollection(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
}
@ -98,7 +99,7 @@ func TestColSegContainer_removeCollection(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -111,7 +112,7 @@ func TestColSegContainer_removeCollection(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
err := node.container.removeCollection(collection)
@ -154,7 +155,7 @@ func TestColSegContainer_getCollectionByID(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -167,14 +168,14 @@ func TestColSegContainer_getCollectionByID(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
targetCollection, err := node.container.getCollectionByID(UniqueID(0))
assert.NoError(t, err)
assert.NotNil(t, targetCollection)
assert.Equal(t, targetCollection.meta.Schema.Name, "collection0")
assert.Equal(t, targetCollection.meta.Id, UniqueID(0))
assert.Equal(t, targetCollection.meta.ID, UniqueID(0))
}
func TestColSegContainer_getCollectionByName(t *testing.T) {
@ -212,7 +213,7 @@ func TestColSegContainer_getCollectionByName(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -225,14 +226,14 @@ func TestColSegContainer_getCollectionByName(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
targetCollection, err := node.container.getCollectionByName("collection0")
assert.NoError(t, err)
assert.NotNil(t, targetCollection)
assert.Equal(t, targetCollection.meta.Schema.Name, "collection0")
assert.Equal(t, targetCollection.meta.Id, UniqueID(0))
assert.Equal(t, targetCollection.meta.ID, UniqueID(0))
}
//----------------------------------------------------------------------------------------------------- partition
@ -271,7 +272,7 @@ func TestColSegContainer_addPartition(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -284,7 +285,7 @@ func TestColSegContainer_addPartition(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
for _, tag := range collectionMeta.PartitionTags {
@ -329,7 +330,7 @@ func TestColSegContainer_removePartition(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -342,7 +343,7 @@ func TestColSegContainer_removePartition(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
for _, tag := range collectionMeta.PartitionTags {
@ -389,7 +390,7 @@ func TestColSegContainer_getPartitionByTag(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -402,7 +403,7 @@ func TestColSegContainer_getPartitionByTag(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
for _, tag := range collectionMeta.PartitionTags {
@ -452,7 +453,7 @@ func TestColSegContainer_addSegment(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -465,7 +466,7 @@ func TestColSegContainer_addSegment(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0])
@ -514,7 +515,7 @@ func TestColSegContainer_removeSegment(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -527,7 +528,7 @@ func TestColSegContainer_removeSegment(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0])
@ -578,7 +579,7 @@ func TestColSegContainer_getSegmentByID(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -591,7 +592,7 @@ func TestColSegContainer_getSegmentByID(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0])
@ -643,7 +644,7 @@ func TestColSegContainer_hasSegment(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -656,7 +657,7 @@ func TestColSegContainer_hasSegment(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0])

View File

@ -26,7 +26,7 @@ func (c *Collection) Name() string {
}
func (c *Collection) ID() UniqueID {
return (*c.meta).Id
return (*c.meta).ID
}
func (c *Collection) Partitions() *[]*Partition {

View File

@ -2,12 +2,13 @@ package reader
import (
"context"
"testing"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"testing"
)
func TestCollection_Partitions(t *testing.T) {
@ -45,7 +46,7 @@ func TestCollection_Partitions(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -58,7 +59,7 @@ func TestCollection_Partitions(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
for _, tag := range collectionMeta.PartitionTags {
@ -101,7 +102,7 @@ func TestCollection_newCollection(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -113,7 +114,7 @@ func TestCollection_newCollection(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
}
func TestCollection_deleteCollection(t *testing.T) {
@ -147,7 +148,7 @@ func TestCollection_deleteCollection(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -159,7 +160,7 @@ func TestCollection_deleteCollection(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
deleteCollection(collection)
}

View File

@ -39,6 +39,11 @@ func (dsService *dataSyncService) start() {
dsService.fg.Start()
}
func (dsService *dataSyncService) close() {
dsService.fg.Close()
(*dsService.dmStream).Close()
}
func (dsService *dataSyncService) initNodes() {
// TODO: add delete pipeline support

View File

@ -1,782 +1,180 @@
package reader
//import (
// "context"
// "encoding/binary"
// "math"
// "testing"
// "time"
//
// "github.com/zilliztech/milvus-distributed/internal/msgstream"
// "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
// internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
//)
//
//const ctxTimeInMillisecond = 500
//
//func TestManipulationService_Start(t *testing.T) {
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
// pulsarUrl := "pulsar://localhost:6650"
//
// node := NewQueryNode(ctx, 0, pulsarUrl)
// node.manipulationService = newDataSyncService(node.ctx, node, node.pulsarURL)
//
// segmentID := int64(0)
//
// var collection = node.newCollection(0, "collection0", "")
// var partition = collection.newPartition("partition0")
// var segment = partition.newSegment(segmentID)
// node.SegmentsMap[segmentID] = segment
//
// node.manipulationService.initNodes()
// go node.manipulationService.fg.Start()
//
// const msgLength = 10
// const DIM = 16
// const N = 3
//
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records []*commonpb.Blob
// for i := 0; i < N; i++ {
// blob := &commonpb.Blob{
// Value: rawData,
// }
// records = append(records, blob)
// }
//
// timeRange := TimeRange{
// timestampMin: 0,
// timestampMax: math.MaxUint64,
// }
//
// insertMessages := make([]*msgstream.TsMsg, 0)
//
// for i := 0; i < msgLength; i++ {
// var msg msgstream.TsMsg = &msgstream.InsertTask{
// InsertRequest: internalPb.InsertRequest{
// MsgType: internalPb.MsgType_kInsert,
// ReqId: int64(0),
// CollectionName: "collection0",
// PartitionTag: "default",
// SegmentId: int64(0),
// ChannelId: int64(0),
// ProxyId: int64(0),
// Timestamps: []uint64{uint64(i + 1000), uint64(i + 1000)},
// RowIds: []int64{int64(i), int64(i)},
// RowData: []*commonpb.Blob{
// {Value: rawData},
// {Value: rawData},
// },
// },
// }
// insertMessages = append(insertMessages, &msg)
// }
//
// msgPack := msgstream.MsgPack{
// BeginTs: timeRange.timestampMin,
// EndTs: timeRange.timestampMax,
// Msgs: insertMessages,
// }
//
// var msgStreamMsg Msg = &msgStreamMsg{
// tsMessages: msgPack.Msgs,
// timeRange: TimeRange{
// timestampMin: msgPack.BeginTs,
// timestampMax: msgPack.EndTs,
// },
// }
// node.manipulationService.fg.Input(&msgStreamMsg)
//
// node.Close()
//
// for {
// select {
// case <-ctx.Done():
// return
// }
// }
//}
import (
"context"
"encoding/binary"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"math"
"testing"
"time"
//import (
// "context"
// "encoding/binary"
// "math"
// "strconv"
// "sync"
// "testing"
// "time"
//
// "github.com/stretchr/testify/assert"
// "github.com/zilliztech/milvus-distributed/internal/conf"
// "github.com/zilliztech/milvus-distributed/internal/msgclient"
// msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
//)
//
//func TestInsertAndDelete_MessagesPreprocess(t *testing.T) {
// ctx := context.Background()
//
// node := NewQueryNode(ctx, 0, 0)
// var collection = node.newCollection(0, "collection0", "")
// _ = collection.newPartition("partition0")
//
// const msgLength = 10
// const DIM = 16
// const N = 3
//
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records [][]byte
// for i := 0; i < N; i++ {
// records = append(records, rawData)
// }
//
// insertDeleteMessages := make([]*msgPb.InsertOrDeleteMsg, 0)
//
// for i := 0; i < msgLength; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i),
// ChannelId: 0,
// Op: msgPb.OpType_INSERT,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// timeRange := TimeRange{
// timestampMin: 0,
// timestampMax: math.MaxUint64,
// }
//
// node.QueryNodeDataInit()
//
// assert.NotNil(t, node.deletePreprocessData)
// assert.NotNil(t, node.insertData)
// assert.NotNil(t, node.deleteData)
//
// node.MessagesPreprocess(insertDeleteMessages, timeRange)
//
// assert.Equal(t, len(node.insertData.insertIDs), msgLength)
// assert.Equal(t, len(node.insertData.insertTimestamps), msgLength)
// assert.Equal(t, len(node.insertData.insertRecords), msgLength)
// assert.Equal(t, len(node.insertData.insertOffset), 0)
//
// assert.Equal(t, len(node.buffer.InsertDeleteBuffer), 0)
// assert.Equal(t, len(node.buffer.validInsertDeleteBuffer), 0)
//
// assert.Equal(t, len(node.SegmentsMap), 10)
// assert.Equal(t, len(node.Collections[0].Partitions[0].segments), 10)
//
// node.Close()
//}
//
//// NOTE: start pulsar before test
//func TestInsertAndDelete_WriterDelete(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
// mc.InitClient(ctx, pulsarAddr)
//
// mc.ReceiveMessage()
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// var collection = node.newCollection(0, "collection0", "")
// _ = collection.newPartition("partition0")
//
// const msgLength = 10
// const DIM = 16
// const N = 3
//
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records [][]byte
// for i := 0; i < N; i++ {
// records = append(records, rawData)
// }
//
// insertDeleteMessages := make([]*msgPb.InsertOrDeleteMsg, 0)
//
// for i := 0; i < msgLength; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i),
// ChannelId: 0,
// Op: msgPb.OpType_DELETE,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// timeRange := TimeRange{
// timestampMin: 0,
// timestampMax: math.MaxUint64,
// }
//
// node.QueryNodeDataInit()
//
// assert.NotNil(t, node.deletePreprocessData)
// assert.NotNil(t, node.insertData)
// assert.NotNil(t, node.deleteData)
//
// node.MessagesPreprocess(insertDeleteMessages, timeRange)
//
// for i := 0; i < msgLength; i++ {
// key2SegMsg := msgPb.Key2SegMsg{
// Uid: int64(i),
// Timestamp: uint64(i + 1000),
// SegmentId: []int64{int64(i)},
// }
// node.messageClient.Key2SegChan <- &key2SegMsg
// }
//
// assert.Equal(t, len(node.deleteData.deleteIDs), 0)
// assert.Equal(t, len(node.deleteData.deleteTimestamps), 0)
// assert.Equal(t, len(node.deleteData.deleteOffset), 0)
//
// assert.Equal(t, len(node.buffer.InsertDeleteBuffer), 0)
// assert.Equal(t, len(node.buffer.validInsertDeleteBuffer), 0)
//
// assert.Equal(t, len(node.deletePreprocessData.deleteRecords), msgLength)
// assert.Equal(t, node.deletePreprocessData.count, int32(msgLength))
//
// node.WriterDelete()
//
// assert.Equal(t, len(node.deletePreprocessData.deleteRecords), msgLength)
// assert.Equal(t, node.deletePreprocessData.count, int32(0))
//
// assert.Equal(t, len(node.SegmentsMap), 10)
// assert.Equal(t, len(node.Collections[0].Partitions[0].segments), 10)
//
// node.Close()
//}
//
//// NOTE: start pulsar before test
//func TestInsertAndDelete_PreInsertAndDelete(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
// mc.InitClient(ctx, pulsarAddr)
//
// mc.ReceiveMessage()
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// var collection = node.newCollection(0, "collection0", "")
// _ = collection.newPartition("partition0")
//
// const msgLength = 10
// const DIM = 16
// const N = 3
//
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records [][]byte
// for i := 0; i < N; i++ {
// records = append(records, rawData)
// }
//
// insertDeleteMessages := make([]*msgPb.InsertOrDeleteMsg, 0)
//
// for i := 0; i < msgLength/2; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i),
// ChannelId: 0,
// Op: msgPb.OpType_INSERT,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// for i := 0; i < msgLength/2; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i + msgLength/2),
// ChannelId: 0,
// Op: msgPb.OpType_DELETE,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// timeRange := TimeRange{
// timestampMin: 0,
// timestampMax: math.MaxUint64,
// }
//
// node.QueryNodeDataInit()
//
// assert.NotNil(t, node.deletePreprocessData)
// assert.NotNil(t, node.insertData)
// assert.NotNil(t, node.deleteData)
//
// node.MessagesPreprocess(insertDeleteMessages, timeRange)
//
// for i := 0; i < msgLength; i++ {
// key2SegMsg := msgPb.Key2SegMsg{
// Uid: int64(i),
// Timestamp: uint64(i + 1000),
// SegmentId: []int64{int64(i)},
// }
// node.messageClient.Key2SegChan <- &key2SegMsg
// }
//
// assert.Equal(t, len(node.insertData.insertIDs), msgLength/2)
// assert.Equal(t, len(node.insertData.insertTimestamps), msgLength/2)
// assert.Equal(t, len(node.insertData.insertRecords), msgLength/2)
// assert.Equal(t, len(node.insertData.insertOffset), 0)
//
// assert.Equal(t, len(node.deleteData.deleteIDs), 0)
// assert.Equal(t, len(node.deleteData.deleteTimestamps), 0)
// assert.Equal(t, len(node.deleteData.deleteOffset), 0)
//
// assert.Equal(t, len(node.buffer.InsertDeleteBuffer), 0)
// assert.Equal(t, len(node.buffer.validInsertDeleteBuffer), 0)
//
// assert.Equal(t, len(node.deletePreprocessData.deleteRecords), msgLength/2)
// assert.Equal(t, node.deletePreprocessData.count, int32(msgLength/2))
//
// assert.Equal(t, len(node.SegmentsMap), 10)
// assert.Equal(t, len(node.Collections[0].Partitions[0].segments), 10)
//
// node.WriterDelete()
//
// assert.Equal(t, len(node.deletePreprocessData.deleteRecords), msgLength/2)
// assert.Equal(t, node.deletePreprocessData.count, int32(0))
//
// node.PreInsertAndDelete()
//
// assert.Equal(t, len(node.insertData.insertOffset), msgLength/2)
//
// assert.Equal(t, len(node.deleteData.deleteIDs), msgLength/2)
// assert.Equal(t, len(node.deleteData.deleteTimestamps), msgLength/2)
// assert.Equal(t, len(node.deleteData.deleteOffset), msgLength/2)
//
// node.Close()
//}
//
//func TestInsertAndDelete_DoInsert(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// var collection = node.newCollection(0, "collection0", "")
// _ = collection.newPartition("partition0")
//
// const msgLength = 10
// const DIM = 16
// const N = 3
//
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records [][]byte
// for i := 0; i < N; i++ {
// records = append(records, rawData)
// }
//
// insertDeleteMessages := make([]*msgPb.InsertOrDeleteMsg, 0)
//
// for i := 0; i < msgLength; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i),
// ChannelId: 0,
// Op: msgPb.OpType_INSERT,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// timeRange := TimeRange{
// timestampMin: 0,
// timestampMax: math.MaxUint64,
// }
//
// node.QueryNodeDataInit()
//
// assert.NotNil(t, node.deletePreprocessData)
// assert.NotNil(t, node.insertData)
// assert.NotNil(t, node.deleteData)
//
// node.MessagesPreprocess(insertDeleteMessages, timeRange)
//
// assert.Equal(t, len(node.insertData.insertIDs), msgLength)
// assert.Equal(t, len(node.insertData.insertTimestamps), msgLength)
// assert.Equal(t, len(node.insertData.insertRecords), msgLength)
// assert.Equal(t, len(node.insertData.insertOffset), 0)
//
// assert.Equal(t, len(node.buffer.InsertDeleteBuffer), 0)
// assert.Equal(t, len(node.buffer.validInsertDeleteBuffer), 0)
//
// assert.Equal(t, len(node.SegmentsMap), 10)
// assert.Equal(t, len(node.Collections[0].Partitions[0].segments), 10)
//
// node.PreInsertAndDelete()
//
// assert.Equal(t, len(node.insertData.insertOffset), msgLength)
//
// wg := sync.WaitGroup{}
// for segmentID := range node.insertData.insertRecords {
// wg.Add(1)
// go node.DoInsert(segmentID, &wg)
// }
// wg.Wait()
//
// node.Close()
//}
//
//// NOTE: start pulsar before test
//func TestInsertAndDelete_DoDelete(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
// mc.InitClient(ctx, pulsarAddr)
//
// mc.ReceiveMessage()
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// var collection = node.newCollection(0, "collection0", "")
// _ = collection.newPartition("partition0")
//
// const msgLength = 10
// const DIM = 16
// const N = 3
//
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records [][]byte
// for i := 0; i < N; i++ {
// records = append(records, rawData)
// }
//
// insertDeleteMessages := make([]*msgPb.InsertOrDeleteMsg, 0)
//
// for i := 0; i < msgLength; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i),
// ChannelId: 0,
// Op: msgPb.OpType_DELETE,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// timeRange := TimeRange{
// timestampMin: 0,
// timestampMax: math.MaxUint64,
// }
//
// node.QueryNodeDataInit()
//
// assert.NotNil(t, node.deletePreprocessData)
// assert.NotNil(t, node.insertData)
// assert.NotNil(t, node.deleteData)
//
// node.MessagesPreprocess(insertDeleteMessages, timeRange)
//
// for i := 0; i < msgLength; i++ {
// key2SegMsg := msgPb.Key2SegMsg{
// Uid: int64(i),
// Timestamp: uint64(i + 1000),
// SegmentId: []int64{int64(i)},
// }
// node.messageClient.Key2SegChan <- &key2SegMsg
// }
//
// assert.Equal(t, len(node.deleteData.deleteIDs), 0)
// assert.Equal(t, len(node.deleteData.deleteTimestamps), 0)
// assert.Equal(t, len(node.deleteData.deleteOffset), 0)
//
// assert.Equal(t, len(node.buffer.InsertDeleteBuffer), 0)
// assert.Equal(t, len(node.buffer.validInsertDeleteBuffer), 0)
//
// assert.Equal(t, len(node.deletePreprocessData.deleteRecords), msgLength)
// assert.Equal(t, node.deletePreprocessData.count, int32(msgLength))
//
// assert.Equal(t, len(node.SegmentsMap), 10)
// assert.Equal(t, len(node.Collections[0].Partitions[0].segments), 10)
//
// node.WriterDelete()
//
// assert.Equal(t, len(node.deletePreprocessData.deleteRecords), msgLength)
// assert.Equal(t, node.deletePreprocessData.count, int32(0))
//
// node.PreInsertAndDelete()
//
// assert.Equal(t, len(node.deleteData.deleteIDs), msgLength)
// assert.Equal(t, len(node.deleteData.deleteTimestamps), msgLength)
// assert.Equal(t, len(node.deleteData.deleteOffset), msgLength)
//
// wg := sync.WaitGroup{}
// for segmentID, deleteIDs := range node.deleteData.deleteIDs {
// if segmentID < 0 {
// continue
// }
// wg.Add(1)
// var deleteTimestamps = node.deleteData.deleteTimestamps[segmentID]
// go node.DoDelete(segmentID, &deleteIDs, &deleteTimestamps, &wg)
// }
// wg.Wait()
//
// node.Close()
//}
//
//// NOTE: start pulsar before test
//func TestInsertAndDelete_DoInsertAndDelete(t *testing.T) {
// conf.LoadConfig("config.yaml")
//
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
// ctx, cancel := context.WithDeadline(context.Background(), d)
// defer cancel()
//
// mc := msgclient.ReaderMessageClient{}
// pulsarAddr := "pulsar://"
// pulsarAddr += conf.Config.Pulsar.Address
// pulsarAddr += ":"
// pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
// mc.InitClient(ctx, pulsarAddr)
//
// mc.ReceiveMessage()
// node := CreateQueryNode(ctx, 0, 0, &mc)
//
// var collection = node.newCollection(0, "collection0", "")
// _ = collection.newPartition("partition0")
//
// const msgLength = 10
// const DIM = 16
// const N = 3
//
// var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
// var rawData []byte
// for _, ele := range vec {
// buf := make([]byte, 4)
// binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
// rawData = append(rawData, buf...)
// }
// bs := make([]byte, 4)
// binary.LittleEndian.PutUint32(bs, 1)
// rawData = append(rawData, bs...)
// var records [][]byte
// for i := 0; i < N; i++ {
// records = append(records, rawData)
// }
//
// insertDeleteMessages := make([]*msgPb.InsertOrDeleteMsg, 0)
//
// for i := 0; i < msgLength/2; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i),
// ChannelId: 0,
// Op: msgPb.OpType_INSERT,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// for i := 0; i < msgLength/2; i++ {
// msg := msgPb.InsertOrDeleteMsg{
// CollectionName: "collection0",
// RowsData: &msgPb.RowData{
// Blob: rawData,
// },
// Uid: int64(i),
// PartitionTag: "partition0",
// Timestamp: uint64(i + 1000),
// SegmentId: int64(i + msgLength/2),
// ChannelId: 0,
// Op: msgPb.OpType_DELETE,
// ClientId: 0,
// ExtraParams: nil,
// }
// insertDeleteMessages = append(insertDeleteMessages, &msg)
// }
//
// timeRange := TimeRange{
// timestampMin: 0,
// timestampMax: math.MaxUint64,
// }
//
// node.QueryNodeDataInit()
//
// assert.NotNil(t, node.deletePreprocessData)
// assert.NotNil(t, node.insertData)
// assert.NotNil(t, node.deleteData)
//
// node.MessagesPreprocess(insertDeleteMessages, timeRange)
//
// for i := 0; i < msgLength; i++ {
// key2SegMsg := msgPb.Key2SegMsg{
// Uid: int64(i),
// Timestamp: uint64(i + 1000),
// SegmentId: []int64{int64(i)},
// }
// node.messageClient.Key2SegChan <- &key2SegMsg
// }
//
// assert.Equal(t, len(node.insertData.insertIDs), msgLength/2)
// assert.Equal(t, len(node.insertData.insertTimestamps), msgLength/2)
// assert.Equal(t, len(node.insertData.insertRecords), msgLength/2)
// assert.Equal(t, len(node.insertData.insertOffset), 0)
//
// assert.Equal(t, len(node.deleteData.deleteIDs), 0)
// assert.Equal(t, len(node.deleteData.deleteTimestamps), 0)
// assert.Equal(t, len(node.deleteData.deleteOffset), 0)
//
// assert.Equal(t, len(node.buffer.InsertDeleteBuffer), 0)
// assert.Equal(t, len(node.buffer.validInsertDeleteBuffer), 0)
//
// assert.Equal(t, len(node.deletePreprocessData.deleteRecords), msgLength/2)
// assert.Equal(t, node.deletePreprocessData.count, int32(msgLength/2))
//
// assert.Equal(t, len(node.SegmentsMap), 10)
// assert.Equal(t, len(node.Collections[0].Partitions[0].segments), 10)
//
// node.WriterDelete()
//
// assert.Equal(t, len(node.deletePreprocessData.deleteRecords), msgLength/2)
// assert.Equal(t, node.deletePreprocessData.count, int32(0))
//
// node.PreInsertAndDelete()
//
// assert.Equal(t, len(node.insertData.insertOffset), msgLength/2)
//
// assert.Equal(t, len(node.deleteData.deleteIDs), msgLength/2)
// assert.Equal(t, len(node.deleteData.deleteTimestamps), msgLength/2)
// assert.Equal(t, len(node.deleteData.deleteOffset), msgLength/2)
//
// status := node.DoInsertAndDelete()
//
// assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
//
// node.Close()
//}
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
)
const ctxTimeInMillisecond = 2000
const closeWithDeadline = true
// NOTE: start pulsar before test
func TestManipulationService_Start(t *testing.T) {
var ctx context.Context
if closeWithDeadline {
var cancel context.CancelFunc
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, cancel = context.WithDeadline(context.Background(), d)
defer cancel()
} else {
ctx = context.Background()
}
// init query node
pulsarUrl := "pulsar://localhost:6650"
node := NewQueryNode(ctx, 0, pulsarUrl)
// init meta
fieldVec := schemapb.FieldSchema{
Name: "vec",
DataType: schemapb.DataType_VECTOR_FLOAT,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "16",
},
},
}
fieldInt := schemapb.FieldSchema{
Name: "age",
DataType: schemapb.DataType_INT32,
TypeParams: []*commonpb.KeyValuePair{
{
Key: "dim",
Value: "1",
},
},
}
schema := schemapb.CollectionSchema{
Name: "collection0",
Fields: []*schemapb.FieldSchema{
&fieldVec, &fieldInt,
},
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
PartitionTags: []string{"default"},
}
collectionMetaBlob := proto.MarshalTextString(&collectionMeta)
assert.NotEqual(t, "", collectionMetaBlob)
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
partition, err := node.container.addPartition(collection, collectionMeta.PartitionTags[0])
assert.NoError(t, err)
segmentID := UniqueID(0)
targetSeg, err := node.container.addSegment(collection, partition, segmentID)
assert.NoError(t, err)
assert.Equal(t, targetSeg.segmentID, segmentID)
// test data generate
const msgLength = 10
const DIM = 16
const N = 10
var vec = [DIM]float32{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}
var rawData []byte
for _, ele := range vec {
buf := make([]byte, 4)
binary.LittleEndian.PutUint32(buf, math.Float32bits(ele))
rawData = append(rawData, buf...)
}
bs := make([]byte, 4)
binary.LittleEndian.PutUint32(bs, 1)
rawData = append(rawData, bs...)
var records []*commonpb.Blob
for i := 0; i < N; i++ {
blob := &commonpb.Blob{
Value: rawData,
}
records = append(records, blob)
}
timeRange := TimeRange{
timestampMin: 0,
timestampMax: math.MaxUint64,
}
// messages generate
insertMessages := make([]*msgstream.TsMsg, 0)
for i := 0; i < msgLength; i++ {
var msg msgstream.TsMsg = &msgstream.InsertMsg{
BaseMsg: msgstream.BaseMsg{
HashValues: []int32{
int32(i), int32(i),
},
},
InsertRequest: internalPb.InsertRequest{
MsgType: internalPb.MsgType_kInsert,
ReqId: int64(0),
CollectionName: "collection0",
PartitionTag: "default",
SegmentId: int64(0),
ChannelId: int64(0),
ProxyId: int64(0),
Timestamps: []uint64{uint64(i + 1000), uint64(i + 1000)},
RowIds: []int64{int64(i), int64(i)},
RowData: []*commonpb.Blob{
{Value: rawData},
{Value: rawData},
},
},
}
insertMessages = append(insertMessages, &msg)
}
msgPack := msgstream.MsgPack{
BeginTs: timeRange.timestampMin,
EndTs: timeRange.timestampMax,
Msgs: insertMessages,
}
// pulsar produce
const receiveBufSize = 1024
producerChannels := []string{"insert"}
insertStream := msgstream.NewPulsarMsgStream(ctx, receiveBufSize)
insertStream.SetPulsarCient(pulsarUrl)
insertStream.CreatePulsarProducers(producerChannels)
var insertMsgStream msgstream.MsgStream = insertStream
insertMsgStream.Start()
err = insertMsgStream.Produce(&msgPack)
assert.NoError(t, err)
// dataSync
node.dataSyncService = newDataSyncService(node.ctx, node, node.pulsarURL)
go node.dataSyncService.start()
node.Close()
for {
select {
case <-ctx.Done():
return
}
}
}

View File

@ -1,9 +1,10 @@
package reader
import (
"log"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
"log"
)
type filterDmNode struct {
@ -15,10 +16,10 @@ func (fdmNode *filterDmNode) Name() string {
}
func (fdmNode *filterDmNode) Operate(in []*Msg) []*Msg {
// fmt.Println("Do filterDmNode operation")
//fmt.Println("Do filterDmNode operation")
if len(in) != 1 {
log.Println("Invalid operate message input in filterDmNode")
log.Println("Invalid operate message input in filterDmNode, input length = ", len(in))
// TODO: add error handling
}

View File

@ -3,10 +3,11 @@ package reader
import (
"errors"
"fmt"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"log"
"strconv"
"sync"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
)
type insertNode struct {
@ -29,7 +30,7 @@ func (iNode *insertNode) Operate(in []*Msg) []*Msg {
// fmt.Println("Do insertNode operation")
if len(in) != 1 {
log.Println("Invalid operate message input in insertNode")
log.Println("Invalid operate message input in insertNode, input length = ", len(in))
// TODO: add error handling
}

View File

@ -19,12 +19,12 @@ type schemaUpdateMsg struct {
type insertMsg struct {
insertMessages []*msgstream.InsertMsg
timeRange TimeRange
timeRange TimeRange
}
type deleteMsg struct {
deleteMessages []*msgstream.DeleteMsg
timeRange TimeRange
timeRange TimeRange
}
type serviceTimeMsg struct {

View File

@ -2,9 +2,9 @@ package reader
import (
"context"
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"github.com/zilliztech/milvus-distributed/internal/util/flowgraph"
)
func newDmInputNode(ctx context.Context, pulsarURL string) *flowgraph.InputNode {

View File

@ -7,4 +7,4 @@ const maxParallelism = flowgraph.MaxQueueLength
type BaseNode = flowgraph.BaseNode
type Node = flowgraph.Node
type InputNode = flowgraph.InputNode
type InputNode = flowgraph.InputNode

View File

@ -6,7 +6,7 @@ import (
type serviceTimeNode struct {
BaseNode
node *QueryNode
node *QueryNode
}
func (stNode *serviceTimeNode) Name() string {
@ -17,7 +17,7 @@ func (stNode *serviceTimeNode) Operate(in []*Msg) []*Msg {
// fmt.Println("Do serviceTimeNode operation")
if len(in) != 1 {
log.Println("Invalid operate message input in serviceTimeNode")
log.Println("Invalid operate message input in serviceTimeNode, input length = ", len(in))
// TODO: add error handling
}

View File

@ -3,7 +3,6 @@ package reader
import (
"context"
"fmt"
"github.com/golang/protobuf/proto"
"log"
"path"
"reflect"
@ -11,6 +10,8 @@ import (
"strings"
"time"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/kv"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"

View File

@ -2,12 +2,13 @@ package reader
import (
"context"
"testing"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"testing"
)
func TestPartition_Segments(t *testing.T) {
@ -45,7 +46,7 @@ func TestPartition_Segments(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -58,7 +59,7 @@ func TestPartition_Segments(t *testing.T) {
var collection = node.container.addCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
assert.Equal(t, len(node.container.collections), 1)
for _, tag := range collectionMeta.PartitionTags {
@ -72,7 +73,7 @@ func TestPartition_Segments(t *testing.T) {
targetPartition := (*partitions)[0]
const segmentNum = 3
for i:= 0; i < segmentNum; i++ {
for i := 0; i < segmentNum; i++ {
_, err := node.container.addSegment(collection, targetPartition, UniqueID(i))
assert.NoError(t, err)
}

View File

@ -20,12 +20,11 @@ func (ss *searchService) Plan(queryBlob string) *plan {
CPlaceholderGroup* ParserPlaceholderGroup(const char* placeholders_blob)
*/
/*
long int GetNumOfQuery(CPlaceholderGroup* placeholder_group)
long int GetNumOfQuery(CPlaceholderGroup* placeholder_group)
long int GetTopK(CPlan* plan)
*/
long int GetTopK(CPlan* plan)
*/
return nil
}

View File

@ -4,9 +4,10 @@ import (
"context"
"encoding/json"
"fmt"
"log"
"github.com/zilliztech/milvus-distributed/internal/msgstream"
servicePb "github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
"log"
)
type searchService struct {
@ -61,7 +62,7 @@ func (ss *searchService) start() {
producerChannels := []string{"searchResult"}
searchResultStream := msgstream.NewPulsarMsgStream(context.Background(), receiveBufSize)
searchResultStream := msgstream.NewPulsarMsgStream(ss.ctx, receiveBufSize)
searchResultStream.SetPulsarCient(ss.pulsarURL)
searchResultStream.CreatePulsarProducers(producerChannels)

View File

@ -179,16 +179,14 @@ func (s *Segment) segmentDelete(offset int64, entityIDs *[]UniqueID, timestamps
func (s *Segment) segmentSearch(query *queryInfo, timestamp Timestamp, vectorRecord *servicePb.PlaceholderValue) (*SearchResult, error) {
/*
*/
*/
//type CQueryInfo C.CQueryInfo
/*
void* Search(void* plan, void* placeholder_groups, uint64_t* timestamps, int num_groups, long int* result_ids,
float* result_distances)
*/
cQuery := C.CQueryInfo{
num_queries: C.long(query.NumQueries),
topK: C.int(query.TopK),

View File

@ -2,11 +2,12 @@ package reader
import (
"encoding/binary"
"math"
"testing"
"github.com/golang/protobuf/proto"
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
"math"
"testing"
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
@ -45,7 +46,7 @@ func TestSegment_newSegment(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -57,7 +58,7 @@ func TestSegment_newSegment(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
@ -95,7 +96,7 @@ func TestSegment_deleteSegment(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -107,7 +108,7 @@ func TestSegment_deleteSegment(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
@ -148,7 +149,7 @@ func TestSegment_getRowCount(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -160,7 +161,7 @@ func TestSegment_getRowCount(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
@ -230,7 +231,7 @@ func TestSegment_getDeletedCount(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -242,7 +243,7 @@ func TestSegment_getDeletedCount(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
@ -319,7 +320,7 @@ func TestSegment_getMemSize(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -331,7 +332,7 @@ func TestSegment_getMemSize(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
@ -402,7 +403,7 @@ func TestSegment_segmentInsert(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -414,7 +415,7 @@ func TestSegment_segmentInsert(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
@ -481,7 +482,7 @@ func TestSegment_segmentDelete(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -493,7 +494,7 @@ func TestSegment_segmentDelete(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
@ -647,7 +648,7 @@ func TestSegment_segmentPreInsert(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -659,7 +660,7 @@ func TestSegment_segmentPreInsert(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)
@ -720,7 +721,7 @@ func TestSegment_segmentPreDelete(t *testing.T) {
}
collectionMeta := etcdpb.CollectionMeta{
Id: UniqueID(0),
ID: UniqueID(0),
Schema: &schema,
CreateTime: Timestamp(0),
SegmentIds: []UniqueID{0},
@ -732,7 +733,7 @@ func TestSegment_segmentPreDelete(t *testing.T) {
collection := newCollection(&collectionMeta, collectionMetaBlob)
assert.Equal(t, collection.meta.Schema.Name, "collection0")
assert.Equal(t, collection.meta.Id, UniqueID(0))
assert.Equal(t, collection.meta.ID, UniqueID(0))
segmentID := UniqueID(0)
segment := newSegment(collection, segmentID)

View File

@ -113,6 +113,7 @@ func TestS3Driver_Segment(t *testing.T) {
segmentIndex, err := client.GetSegmentIndex(ctx, "segmentA")
assert.Equal(t, "This is segmentA's index!", string(segmentIndex))
assert.Nil(t, err)
err = client.DeleteSegmentIndex(ctx, "segmentA")
assert.Nil(t, err)

View File

@ -93,6 +93,9 @@ func (s *S3Driver) deleteGE(ctx context.Context, key Key, timestamp Timestamp) e
return err
}
keyStart, err := codec.MvccEncode(key, timestamp, "")
if err != nil {
panic(err)
}
err = s.driver.DeleteRange(ctx, []byte(keyStart), keys[len(keys)-1])
return err
}
@ -213,7 +216,7 @@ func (s *S3Driver) GetSegments(ctx context.Context, key Key, timestamp Timestamp
var segments []string
for k, v := range segmentsSet {
if v == true {
if v {
segments = append(segments, k)
}
}

View File

@ -6,15 +6,14 @@ import (
"github.com/minio/minio-go/v7"
"github.com/minio/minio-go/v7/pkg/credentials"
"github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
. "github.com/zilliztech/milvus-distributed/internal/storage/internal/minio/codec"
. "github.com/zilliztech/milvus-distributed/internal/storage/type"
storageType "github.com/zilliztech/milvus-distributed/internal/storage/type"
)
type minioDriver struct {
type MinioDriver struct {
driver *minioStore
}
func NewMinioDriver(ctx context.Context) (*minioDriver, error) {
func NewMinioDriver(ctx context.Context) (*MinioDriver, error) {
// to-do read conf
var endPoint = "localhost:9000"
var accessKeyID = "testminio"
@ -41,15 +40,15 @@ func NewMinioDriver(ctx context.Context) (*minioDriver, error) {
return nil, err
}
}
return &minioDriver{
return &MinioDriver{
&minioStore{
client: minioClient,
},
}, nil
}
func (s *minioDriver) put(ctx context.Context, key Key, value Value, timestamp Timestamp, suffix string) error {
minioKey, err := MvccEncode(key, timestamp, suffix)
func (s *MinioDriver) put(ctx context.Context, key storageType.Key, value storageType.Value, timestamp storageType.Timestamp, suffix string) error {
minioKey, err := codec.MvccEncode(key, timestamp, suffix)
if err != nil {
return err
}
@ -58,8 +57,8 @@ func (s *minioDriver) put(ctx context.Context, key Key, value Value, timestamp T
return err
}
func (s *minioDriver) scanLE(ctx context.Context, key Key, timestamp Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) {
keyEnd, err := MvccEncode(key, timestamp, "")
func (s *MinioDriver) scanLE(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp, keyOnly bool) ([]storageType.Timestamp, []storageType.Key, []storageType.Value, error) {
keyEnd, err := codec.MvccEncode(key, timestamp, "")
if err != nil {
return nil, nil, nil, err
}
@ -69,7 +68,7 @@ func (s *minioDriver) scanLE(ctx context.Context, key Key, timestamp Timestamp,
return nil, nil, nil, err
}
var timestamps []Timestamp
var timestamps []storageType.Timestamp
for _, key := range keys {
_, timestamp, _, _ := codec.MvccDecode(key)
timestamps = append(timestamps, timestamp)
@ -78,8 +77,8 @@ func (s *minioDriver) scanLE(ctx context.Context, key Key, timestamp Timestamp,
return timestamps, keys, values, nil
}
func (s *minioDriver) scanGE(ctx context.Context, key Key, timestamp Timestamp, keyOnly bool) ([]Timestamp, []Key, []Value, error) {
keyStart, err := MvccEncode(key, timestamp, "")
func (s *MinioDriver) scanGE(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp, keyOnly bool) ([]storageType.Timestamp, []storageType.Key, []storageType.Value, error) {
keyStart, err := codec.MvccEncode(key, timestamp, "")
if err != nil {
return nil, nil, nil, err
}
@ -89,7 +88,7 @@ func (s *minioDriver) scanGE(ctx context.Context, key Key, timestamp Timestamp,
return nil, nil, nil, err
}
var timestamps []Timestamp
var timestamps []storageType.Timestamp
for _, key := range keys {
_, timestamp, _, _ := codec.MvccDecode(key)
timestamps = append(timestamps, timestamp)
@ -98,30 +97,36 @@ func (s *minioDriver) scanGE(ctx context.Context, key Key, timestamp Timestamp,
return timestamps, keys, values, nil
}
//scan(ctx context.Context, key Key, start Timestamp, end Timestamp, withValue bool) ([]Timestamp, []Key, []Value, error)
func (s *minioDriver) deleteLE(ctx context.Context, key Key, timestamp Timestamp) error {
keyEnd, err := MvccEncode(key, timestamp, "delete")
//scan(ctx context.Context, key storageType.Key, start storageType.Timestamp, end storageType.Timestamp, withValue bool) ([]storageType.Timestamp, []storageType.Key, []storageType.Value, error)
func (s *MinioDriver) deleteLE(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) error {
keyEnd, err := codec.MvccEncode(key, timestamp, "delete")
if err != nil {
return err
}
err = s.driver.DeleteRange(ctx, key, keyEnd)
return err
}
func (s *minioDriver) deleteGE(ctx context.Context, key Key, timestamp Timestamp) error {
func (s *MinioDriver) deleteGE(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) error {
keys, _, err := s.driver.GetByPrefix(ctx, key, true)
if err != nil {
return err
}
keyStart, err := MvccEncode(key, timestamp, "")
err = s.driver.DeleteRange(ctx, []byte(keyStart), keys[len(keys)-1])
return err
keyStart, err := codec.MvccEncode(key, timestamp, "")
if err != nil {
panic(err)
}
err = s.driver.DeleteRange(ctx, keyStart, keys[len(keys)-1])
if err != nil {
panic(err)
}
return nil
}
func (s *minioDriver) deleteRange(ctx context.Context, key Key, start Timestamp, end Timestamp) error {
keyStart, err := MvccEncode(key, start, "")
func (s *MinioDriver) deleteRange(ctx context.Context, key storageType.Key, start storageType.Timestamp, end storageType.Timestamp) error {
keyStart, err := codec.MvccEncode(key, start, "")
if err != nil {
return err
}
keyEnd, err := MvccEncode(key, end, "")
keyEnd, err := codec.MvccEncode(key, end, "")
if err != nil {
return err
}
@ -129,8 +134,8 @@ func (s *minioDriver) deleteRange(ctx context.Context, key Key, start Timestamp,
return err
}
func (s *minioDriver) GetRow(ctx context.Context, key Key, timestamp Timestamp) (Value, error) {
minioKey, err := MvccEncode(key, timestamp, "")
func (s *MinioDriver) GetRow(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) (storageType.Value, error) {
minioKey, err := codec.MvccEncode(key, timestamp, "")
if err != nil {
return nil, err
}
@ -140,7 +145,7 @@ func (s *minioDriver) GetRow(ctx context.Context, key Key, timestamp Timestamp)
return nil, err
}
_, _, suffix, err := MvccDecode(keys[0])
_, _, suffix, err := codec.MvccDecode(keys[0])
if err != nil {
return nil, err
}
@ -150,8 +155,8 @@ func (s *minioDriver) GetRow(ctx context.Context, key Key, timestamp Timestamp)
return values[0], err
}
func (s *minioDriver) GetRows(ctx context.Context, keys []Key, timestamps []Timestamp) ([]Value, error) {
var values []Value
func (s *MinioDriver) GetRows(ctx context.Context, keys []storageType.Key, timestamps []storageType.Timestamp) ([]storageType.Value, error) {
var values []storageType.Value
for i, key := range keys {
value, err := s.GetRow(ctx, key, timestamps[i])
if err != nil {
@ -162,15 +167,15 @@ func (s *minioDriver) GetRows(ctx context.Context, keys []Key, timestamps []Time
return values, nil
}
func (s *minioDriver) PutRow(ctx context.Context, key Key, value Value, segment string, timestamp Timestamp) error {
minioKey, err := MvccEncode(key, timestamp, segment)
func (s *MinioDriver) PutRow(ctx context.Context, key storageType.Key, value storageType.Value, segment string, timestamp storageType.Timestamp) error {
minioKey, err := codec.MvccEncode(key, timestamp, segment)
if err != nil {
return err
}
err = s.driver.Put(ctx, minioKey, value)
return err
}
func (s *minioDriver) PutRows(ctx context.Context, keys []Key, values []Value, segments []string, timestamps []Timestamp) error {
func (s *MinioDriver) PutRows(ctx context.Context, keys []storageType.Key, values []storageType.Value, segments []string, timestamps []storageType.Timestamp) error {
maxThread := 100
batchSize := 1
keysLength := len(keys)
@ -186,7 +191,7 @@ func (s *minioDriver) PutRows(ctx context.Context, keys []Key, values []Value, s
}
errCh := make(chan error)
f := func(ctx2 context.Context, keys2 []Key, values2 []Value, segments2 []string, timestamps2 []Timestamp) {
f := func(ctx2 context.Context, keys2 []storageType.Key, values2 []storageType.Value, segments2 []string, timestamps2 []storageType.Timestamp) {
for i := 0; i < len(keys2); i++ {
err := s.PutRow(ctx2, keys2[i], values2[i], segments2[i], timestamps2[i])
errCh <- err
@ -211,8 +216,8 @@ func (s *minioDriver) PutRows(ctx context.Context, keys []Key, values []Value, s
return nil
}
func (s *minioDriver) GetSegments(ctx context.Context, key Key, timestamp Timestamp) ([]string, error) {
keyEnd, err := MvccEncode(key, timestamp, "")
func (s *MinioDriver) GetSegments(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) ([]string, error) {
keyEnd, err := codec.MvccEncode(key, timestamp, "")
if err != nil {
return nil, err
}
@ -222,7 +227,7 @@ func (s *minioDriver) GetSegments(ctx context.Context, key Key, timestamp Timest
}
segmentsSet := map[string]bool{}
for _, key := range keys {
_, _, segment, err := MvccDecode(key)
_, _, segment, err := codec.MvccDecode(key)
if err != nil {
panic("must no error")
}
@ -233,15 +238,15 @@ func (s *minioDriver) GetSegments(ctx context.Context, key Key, timestamp Timest
var segments []string
for k, v := range segmentsSet {
if v == true {
if v {
segments = append(segments, k)
}
}
return segments, err
}
func (s *minioDriver) DeleteRow(ctx context.Context, key Key, timestamp Timestamp) error {
minioKey, err := MvccEncode(key, timestamp, "delete")
func (s *MinioDriver) DeleteRow(ctx context.Context, key storageType.Key, timestamp storageType.Timestamp) error {
minioKey, err := codec.MvccEncode(key, timestamp, "delete")
if err != nil {
return err
}
@ -250,7 +255,7 @@ func (s *minioDriver) DeleteRow(ctx context.Context, key Key, timestamp Timestam
return err
}
func (s *minioDriver) DeleteRows(ctx context.Context, keys []Key, timestamps []Timestamp) error {
func (s *MinioDriver) DeleteRows(ctx context.Context, keys []storageType.Key, timestamps []storageType.Timestamp) error {
maxThread := 100
batchSize := 1
keysLength := len(keys)
@ -266,7 +271,7 @@ func (s *minioDriver) DeleteRows(ctx context.Context, keys []Key, timestamps []T
}
errCh := make(chan error)
f := func(ctx2 context.Context, keys2 []Key, timestamps2 []Timestamp) {
f := func(ctx2 context.Context, keys2 []storageType.Key, timestamps2 []storageType.Timestamp) {
for i := 0; i < len(keys2); i++ {
err := s.DeleteRow(ctx2, keys2[i], timestamps2[i])
errCh <- err
@ -291,21 +296,21 @@ func (s *minioDriver) DeleteRows(ctx context.Context, keys []Key, timestamps []T
return nil
}
func (s *minioDriver) PutLog(ctx context.Context, key Key, value Value, timestamp Timestamp, channel int) error {
logKey := LogEncode(key, timestamp, channel)
func (s *MinioDriver) PutLog(ctx context.Context, key storageType.Key, value storageType.Value, timestamp storageType.Timestamp, channel int) error {
logKey := codec.LogEncode(key, timestamp, channel)
err := s.driver.Put(ctx, logKey, value)
return err
}
func (s *minioDriver) GetLog(ctx context.Context, start Timestamp, end Timestamp, channels []int) ([]Value, error) {
func (s *MinioDriver) GetLog(ctx context.Context, start storageType.Timestamp, end storageType.Timestamp, channels []int) ([]storageType.Value, error) {
keys, values, err := s.driver.GetByPrefix(ctx, []byte("log_"), false)
if err != nil {
return nil, err
}
var resultValues []Value
var resultValues []storageType.Value
for i, key := range keys {
_, ts, channel, err := LogDecode(string(key))
_, ts, channel, err := codec.LogDecode(string(key))
if err != nil {
return nil, err
}
@ -321,32 +326,32 @@ func (s *minioDriver) GetLog(ctx context.Context, start Timestamp, end Timestamp
return resultValues, nil
}
func (s *minioDriver) GetSegmentIndex(ctx context.Context, segment string) (SegmentIndex, error) {
func (s *MinioDriver) GetSegmentIndex(ctx context.Context, segment string) (storageType.SegmentIndex, error) {
return s.driver.Get(ctx, SegmentEncode(segment, "index"))
return s.driver.Get(ctx, codec.SegmentEncode(segment, "index"))
}
func (s *minioDriver) PutSegmentIndex(ctx context.Context, segment string, index SegmentIndex) error {
func (s *MinioDriver) PutSegmentIndex(ctx context.Context, segment string, index storageType.SegmentIndex) error {
return s.driver.Put(ctx, SegmentEncode(segment, "index"), index)
return s.driver.Put(ctx, codec.SegmentEncode(segment, "index"), index)
}
func (s *minioDriver) DeleteSegmentIndex(ctx context.Context, segment string) error {
func (s *MinioDriver) DeleteSegmentIndex(ctx context.Context, segment string) error {
return s.driver.Delete(ctx, SegmentEncode(segment, "index"))
return s.driver.Delete(ctx, codec.SegmentEncode(segment, "index"))
}
func (s *minioDriver) GetSegmentDL(ctx context.Context, segment string) (SegmentDL, error) {
func (s *MinioDriver) GetSegmentDL(ctx context.Context, segment string) (storageType.SegmentDL, error) {
return s.driver.Get(ctx, SegmentEncode(segment, "DL"))
return s.driver.Get(ctx, codec.SegmentEncode(segment, "DL"))
}
func (s *minioDriver) PutSegmentDL(ctx context.Context, segment string, log SegmentDL) error {
func (s *MinioDriver) PutSegmentDL(ctx context.Context, segment string, log storageType.SegmentDL) error {
return s.driver.Put(ctx, SegmentEncode(segment, "DL"), log)
return s.driver.Put(ctx, codec.SegmentEncode(segment, "DL"), log)
}
func (s *minioDriver) DeleteSegmentDL(ctx context.Context, segment string) error {
func (s *MinioDriver) DeleteSegmentDL(ctx context.Context, segment string) error {
return s.driver.Delete(ctx, SegmentEncode(segment, "DL"))
return s.driver.Delete(ctx, codec.SegmentEncode(segment, "DL"))
}

View File

@ -113,6 +113,7 @@ func TestMinioDriver_Segment(t *testing.T) {
segmentIndex, err := client.GetSegmentIndex(ctx, "segmentA")
assert.Equal(t, "This is segmentA's index!", string(segmentIndex))
assert.Nil(t, err)
err = client.DeleteSegmentIndex(ctx, "segmentA")
assert.Nil(t, err)

View File

@ -1,4 +1,4 @@
package tikv_driver
package tikvdriver
import (
"context"
@ -191,6 +191,8 @@ func (s *TikvStore) GetRows(ctx context.Context, keys []Key, timestamps []Timest
batches := batchKeys(keys)
ch := make(chan kvPair, len(keys))
ctx, cancel := context.WithCancel(ctx)
defer cancel()
for n, b := range batches {
batch := b
numBatch := n
@ -322,6 +324,9 @@ func (s *TikvStore) GetLog(ctx context.Context, start Timestamp, end Timestamp,
}
slice := strings.Split(suffix, string(DelimiterPlusOne))
channel, err := strconv.Atoi(slice[len(slice)-1])
if err != nil {
panic(err)
}
for _, item := range channels {
if item == channel {
logs = append(logs, log)
@ -374,7 +379,7 @@ func (s *TikvStore) GetSegments(ctx context.Context, key Key, timestamp Timestam
var segments []string
for k, v := range segmentsSet {
if v == true {
if v {
segments = append(segments, k)
}
}

View File

@ -1,4 +1,4 @@
package tikv_driver
package tikvdriver
import (
"bytes"
@ -42,12 +42,14 @@ func TestTikvEngine_Prefix(t *testing.T) {
// Get by prefix
ks, _, err := engine.GetByPrefix(ctx, prefix, true)
assert.Equal(t, 2, len(ks))
assert.Nil(t, err)
// Delete by prefix
err = engine.DeleteByPrefix(ctx, prefix)
assert.Nil(t, err)
ks, _, err = engine.GetByPrefix(ctx, prefix, true)
assert.Equal(t, 0, len(ks))
assert.Nil(t, err)
//Test large amount keys
num := engine.conf.Raw.MaxScanLimit + 1
@ -104,14 +106,16 @@ func TestTikvStore_Row(t *testing.T) {
// Delete a row
err = store.DeleteRow(ctx, key, 4)
assert.Nil(t, nil)
assert.Nil(t, err)
v, err = store.GetRow(ctx, key, 5)
assert.Nil(t, err)
assert.Nil(t, v)
// Clear test data
err = store.engine.DeleteByPrefix(ctx, key)
assert.Nil(t, err)
k, va, err := store.engine.GetByPrefix(ctx, key, false)
assert.Nil(t, err)
assert.Nil(t, k)
assert.Nil(t, va)
}
@ -146,7 +150,7 @@ func TestTikvStore_BatchRow(t *testing.T) {
assert.Nil(t, err)
// Batch get rows
for i, _ := range timestamps {
for i := range timestamps {
timestamps[i] = 2
}
checkValues, err := store.GetRows(ctx, testKeys, timestamps)
@ -158,13 +162,13 @@ func TestTikvStore_BatchRow(t *testing.T) {
}
// Delete all test rows
for i, _ := range timestamps {
for i := range timestamps {
timestamps[i] = math.MaxUint64
}
err = store.DeleteRows(ctx, testKeys, timestamps)
assert.Nil(t, err)
// Ensure all test row is deleted
for i, _ := range timestamps {
for i := range timestamps {
timestamps[i] = math.MaxUint64
}
checkValues, err = store.GetRows(ctx, testKeys, timestamps)
@ -218,6 +222,9 @@ func TestTikvStore_Log(t *testing.T) {
// Check log
log, err := store.GetLog(ctx, 0, 2, []int{1, 2})
if err != nil {
panic(err)
}
sort.Slice(log, func(i, j int) bool {
return bytes.Compare(log[i], log[j]) == -1
})

View File

@ -66,11 +66,10 @@ func (fg *TimeTickedFlowGraph) Start() {
wg.Wait()
}
func (fg *TimeTickedFlowGraph) Close() error {
func (fg *TimeTickedFlowGraph) Close() {
for _, v := range fg.nodeCtx {
v.Close()
}
return nil
}
func NewTimeTickedFlowGraph(ctx context.Context) *TimeTickedFlowGraph {

View File

@ -174,7 +174,9 @@ func receiveResult(ctx context.Context, fg *TimeTickedFlowGraph) (float64, bool)
func TestTimeTickedFlowGraph_Start(t *testing.T) {
duration := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), duration)
ctx, cancel := context.WithDeadline(context.Background(), duration)
defer cancel()
fg := NewTimeTickedFlowGraph(ctx)
var a Node = &nodeA{

View File

@ -2,6 +2,7 @@ package flowgraph
import (
"github.com/zilliztech/milvus-distributed/internal/msgstream"
"log"
)
type InputNode struct {
@ -24,8 +25,16 @@ func (inNode *InputNode) InStream() *msgstream.MsgStream {
// empty input and return one *Msg
func (inNode *InputNode) Operate(in []*Msg) []*Msg {
//fmt.Println("Do InputNode operation")
msgPack := (*inNode.inStream).Consume()
// TODO: add status
if msgPack == nil {
log.Println("null msg pack")
return nil
}
var msgStreamMsg Msg = &MsgStreamMsg{
tsMessages: msgPack.Msgs,
timestampMin: msgPack.BeginTs,

View File

@ -2,6 +2,7 @@ package flowgraph
import (
"context"
"fmt"
"log"
"sync"
)
@ -32,17 +33,19 @@ type nodeCtx struct {
func (nodeCtx *nodeCtx) Start(ctx context.Context, wg *sync.WaitGroup) {
if (*nodeCtx.node).IsInputNode() {
fmt.Println("start InputNode.inStream")
inStream, ok := (*nodeCtx.node).(*InputNode)
if !ok {
log.Fatal("Invalid inputNode")
}
go (*inStream.inStream).Start()
(*inStream.inStream).Start()
}
for {
select {
case <-ctx.Done():
wg.Done()
fmt.Println((*nodeCtx.node).Name(), "closed")
return
default:
// inputs from inputsMessages for Operate
@ -52,21 +55,25 @@ func (nodeCtx *nodeCtx) Start(ctx context.Context, wg *sync.WaitGroup) {
nodeCtx.collectInputMessages()
inputs = nodeCtx.inputMessages
}
n := *nodeCtx.node
res := n.Operate(inputs)
wg := sync.WaitGroup{}
downstreamLength := len(nodeCtx.downstreamInputChanIdx)
if len(nodeCtx.downstream) < downstreamLength {
log.Fatal("nodeCtx.downstream length = ", len(nodeCtx.downstream))
log.Println("nodeCtx.downstream length = ", len(nodeCtx.downstream))
}
if len(res) < downstreamLength {
log.Fatal("node result length = ", len(res))
log.Println("node result length = ", len(res))
break
}
w := sync.WaitGroup{}
for i := 0; i < downstreamLength; i++ {
wg.Add(1)
go nodeCtx.downstream[i].ReceiveMsg(&wg, res[i], nodeCtx.downstreamInputChanIdx[(*nodeCtx.downstream[i].node).Name()])
w.Add(1)
go nodeCtx.downstream[i].ReceiveMsg(&w, res[i], nodeCtx.downstreamInputChanIdx[(*nodeCtx.downstream[i].node).Name()])
}
wg.Wait()
w.Wait()
}
}
}
@ -74,12 +81,13 @@ func (nodeCtx *nodeCtx) Start(ctx context.Context, wg *sync.WaitGroup) {
func (nodeCtx *nodeCtx) Close() {
for _, channel := range nodeCtx.inputChannels {
close(channel)
fmt.Println("close inputChannel")
}
}
func (nodeCtx *nodeCtx) ReceiveMsg(wg *sync.WaitGroup, msg *Msg, inputChanIdx int) {
nodeCtx.inputChannels[inputChanIdx] <- msg
// fmt.Println((*nodeCtx.node).Name(), "receive to input channel ", inputChanIdx)
//fmt.Println((*nodeCtx.node).Name(), "receive to input channel ", inputChanIdx)
wg.Done()
}
@ -93,8 +101,13 @@ func (nodeCtx *nodeCtx) collectInputMessages() {
// and move them to inputMessages.
for i := 0; i < inputsNum; i++ {
channel := nodeCtx.inputChannels[i]
msg := <-channel
nodeCtx.inputMessages = append(nodeCtx.inputMessages, msg)
msg, ok := <-channel
if !ok {
// TODO: add status
log.Println("input channel closed")
return
}
nodeCtx.inputMessages[i] = msg
}
}