Remove segment status

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
pull/4973/head^2
bigsheeper 2020-10-24 10:45:57 +08:00 committed by yefu.chen
parent e84b0180c9
commit 52932da551
32 changed files with 3467 additions and 744 deletions

View File

@ -0,0 +1,592 @@
# Milvus Developer Guides
by Rentong Guo, Sep 15, 2020
## 1. System Overview
In this section, we sketch the system design of Milvus , including data model, data organization, architecture, and state synchronization.
#### 1.1 Data Model
Milvus exposes the following set of data features to applications:
* a data model based on schematized relational tables, in that rows must have primary-keys,
* a query language specifies data definition, data manipulation, and data query, where data definition includes create, drop, and data manipulation includes insert, upsert, delete, and data query falls into three types, primary key search, approximate nearest neighbor search (ANNS), ANNS with predicates.
The requests' execution order is strictly in accordance with their issue-time order. We take proxy's issue time as a requst's issue time. For a batch request, all its sub-requests share a same issue time. In cases there are multiple proxies, issue time from different proxies are regarded as coming from a central clock.
Transaction is currently not supported by Milvus. Only batch requests such as batch insert/delete/query are supported. A batch insert/delete is guaranteed to become visible atomically.
#### 1.2 Data Organization
<img src="./figs/data_organization.pdf" width=550>
In Milvus, 'collection' refers to the concept of table. A collection can be optionally divided into several 'partitions'. Both collection and partition are the basic execution scopes of queries. When use parition, users should clearly know how a collection should be partitioned. In most cases, parition leads to more flexible data management and more efficient quering. For a partitioned collection, queries can be executed both on the collection or a set of specified partitions.
Each collection or parition contains a set of 'segment groups'. Segment group is the basic unit of data-to-node mapping. It's also the basic unit of replica. For instance, if a query node failed, its segment groups will be redistributed accross other nodes. If a query node is overloaded, part of its segment groups will be migrated to underloaded ones. If a hot collection/partition is detected, its segment groups will be replicated to smooth the system load skewness.
'Segment' is the finest unit of data organization. It is where the data and indexes are actually kept. Each segment contains a set of rows. In order to reduce the memory footprint during a query execution and to fully utilize SIMD, the physical data layout within segments is organized in a column-based manner.
#### 1.3 Architecture Overview
<img src="./figs/system_framework.pdf" width=800>
The main components, proxy, WAL, query node and write node can scale to multiple instances. These components scale seperately for better tradeoff between availability and cost.
The WAL forms a hash ring. Requests (i.e. inserts and deletes) from clients will be repacked by proxy. Operations shared identical hash value (the hash value of primary key) will be routed to the same hash bucket. In addtion, some preprocessing work will be done by proxy, such as static validity checking, primary key assignment (if not given by user), timestamp assignment.
The query/write nodes are linked to the hash ring, with each node covers some portion of the buckets. Once the hash function and bucket coverage are settled, the chain 'proxy -> WAL -> query/write node' will act as a producer-consumer pipeline. Logs in each bucket is a determined operation stream. Via performing the operation stream in order, the query nodes keep themselves up to date.
The query nodes hold all the indexes in memory. Since building index is time-consuming, the query nodes will dump their index to disk (store engine) for fast failure recovery and cross node index copy.
The write nodes are stateless. They simply transforms the newly arrived WALs to binlog format, then append the binlog to store enginey.
Note that not all the components are necessarily replicated. The system provides failure tolerance by maintaining multiple copies of WAL and binlog. When there is no in-memory index replica and there occurs a query node failure, other query nodes will take over its indexes by loading the dumped index files, or rebuilding them from binlog and WALs. The links from query nodes to the hash ring will also be adjusted such that the failure node's input WAL stream can be properly handled by its neighbors.
#### 1.4 State Synchronization
<img src="./figs/state_sync.pdf" width=800>
Data in Milvus have three different forms, namely WAL, binlog, and index. As mentioned in the previous section, WAL can be viewed as a determined operation stream. Other two data forms keep themselves up to date by performing the operation stream in time order.
Each of the WAL is attached with a timestamp, which is the time when the log is sent to the hash bucket. Binlog records, table rows, index cells will also keep that timestamp. In this way, different data forms can offer consistent snapshot for a given time T. For example, requests such as "fetch binlogs before T for point-in-time recovery", "get the row with primary key K at time T", "launch a similarity search at time T for vector V" perform on binlog, index respectively. Though different data forms these three requests are performed, they observe identical snapshot, namely all the state changes before T.
For better throughput, Milvus allows asynchronous state synchronization between WAL and index/binlog/table. Whenever the data is not fresh enough to satisfiy a query, the query will be suspended until the data is up-to-date, or timeout will be returned.
## 2. Schema
#### 2.1 Collection Schema
``` go
type CollectionSchema struct {
Name string
Fields []FieldSchema
}
```
#### 2.2 Field Schema
``` go
type FieldSchema struct {
Id uint64
Name string
Description string
DataType DataType
TypeParams map[string]string
IndexParams map[string]string
}
```
#### 2.3 Type Params
#### 2.4 Index Params
## 3. Request
#### 3.1 Base Request
``` go
type BaseRequest interface {
Type() ReqType
PreExecute() Status
Execute() Status
PostExecute() Status
WaitToFinish() Status
}
```
#### 3.2 Definition Requests
###### 3.2.1 Collection
* CreateCollection
* DropCollection
* HasCollection
* DescribeCollection
* ShowCollections
###### 3.2.2 Partition
* CreatePartition
* DropPartition
* HasPartition
* DescribePartition
* ShowPartitions
###### 3.2.3 Index
* CreateIndex
* DropIndex
* DescribeIndex
###### 3.2.4 Definition Request & Task
```go
type DDRequest struct {
CollectionName string
PartitionName string
SegmentId uint64
ChannelId uint64
PrimaryKeys []uint64
RowData []*RowDataBlob
reqType ReqType
ts Timestamp
}
type DDTask struct {
DDRequest
}
// TsMsg interfaces
func (req *DDTask) Ts() Timestamp
func (req *DDTask) SetTs(ts Timestamp)
// BaseRequest interfaces
func (req *DDTask) Type() ReqType
func (req *DDTask) PreExecute() Status
func (req *DDTask) Execute() Status
func (req *DDTask) PostExecute() Status
func (req *DDTask) WaitToFinish() Status
```
#### 3.3 Manipulation Requsts
###### 3.3.1 Insert
* Insert
###### 3.3.2 Delete
* DeleteByID
###### 3.3.3 Manipulation Requst
```go
type DMRequest struct {
CollectionName string
PartitionTag string
SegmentId uint64
ChannelId uint64
PrimaryKeys []uint64
RowData []*RowDataBlob
reqType ReqType
ts Timestamp
}
type DMTask struct {
DMRequest
}
// TsMsg interfaces
func (req *DMTask) Ts() Timestamp
func (req *DMTask) SetTs(ts Timestamp)
// BaseRequest interfaces
func (req *DMTask) Type() ReqType
func (req *DMTask) PreExecute() Status
func (req *DMTask) Execute() Status
func (req *DMTask) PostExecute() Status
func (req *DMTask) WaitToFinish() Status
```
#### 3.5 Query
## 4. Time
#### 4.1 Timestamp
Before we discuss timestamp, let's take a brief review of Hybrid Logical Clock (HLC). HLC uses 64bits timestamps which are composed of a 46-bits physical component (thought of as and always close to local wall time) and a 18-bits logical component (used to distinguish between events with the same physical component).
<img src="./figs/hlc.pdf" width=450>
HLC's logical part is advanced on each request. The phsical part can be increased in two cases:
A. when the local wall time is greater than HLC's physical part,
B. or the logical part overflows.
In either cases, the physical part will be updated, and the logical part will be set to 0.
Keep the physical part close to local wall time may face non-monotonic problems such as updates to POSIX time that could turn time backward. HLC avoids such problems, since if 'local wall time < HLC's physical part' holds, only case B is satisfied, thus montonicity is guaranteed.
Milvus does not support transaction, but it should gurantee the deterministic execution of the multi-way WAL. The timestamp attached to each request should
- have its physical part close to wall time (has an acceptable bounded error, a.k.a. uncertainty interval in transaction senarios),
- and be globally unique.
HLC leverages on physical clocks at nodes that are synchronized using the NTP. NTP usually maintain time to within tens of milliseconds over local networks in datacenter. Asymmetric routes and network congestion occasionally cause errors of hundreds of milliseconds. Both the normal time error and the spike are acceptable for Milvus use cases.
The interface of Timestamp is as follows.
```
type timestamp struct {
physical uint64 // 18-63 bits
logical uint64 // 0-17 bits
}
type Timestamp uint64
```
#### 4.2 Timestamp Oracle
```
type timestampOracle struct {
client *etcd.Client // client of a reliable meta service, i.e. etcd client
rootPath string // this timestampOracle's working root path on the reliable kv service
saveInterval uint64
lastSavedTime uint64
tso Timestamp // monotonically increasing timestamp
}
func (tso *timestampOracle) GetTimestamp(count uint32) ([]Timestamp, Status)
func (tso *timestampOracle) saveTimestamp() Status
func (tso *timestampOracle) loadTimestamp() Status
```
#### 4.3 Batch Allocation of Timestamps
#### 4.4 Expiration of Timestamps
#### 4.5 T_safe
## 5. Basic Components
#### 5.1 Watchdog
``` go
type ActiveComponent interface {
Id() string
Status() Status
Clean() Status
Restart() Status
}
type ComponentHeartbeat interface {
Id() string
Status() Status
Serialize() string
}
type Watchdog struct {
targets [] *ActiveComponent
heartbeats ComponentHeartbeat chan
}
// register ActiveComponent
func (dog *Watchdog) Register(target *ActiveComponent)
// called by ActiveComponents
func (dog *Watchdog) PutHeartbeat(heartbeat *ComponentHeartbeat)
// dump heatbeats as log stream
func (dog *Watchdog) dumpHeartbeat(heartbeat *ComponentHeartbeat)
```
#### 5.2 Global Parameter Table
``` go
type GlobalParamsTable struct {
params memoryKV
}
func (gparams *GlobalParamsTable) Save(key, value string) Status
func (gparams *GlobalParamsTable) Load(key string) (string, Status)
func (gparams *GlobalParamsTable) LoadRange(key, endKey string, limit int) ([]string, []string, Status)
func (gparams *GlobalParamsTable) Remove(key string) Status
```
## 6. Proxy
#### 6.1 Overview
#### 6.2 Request Scheduler
``` go
type requestScheduler struct {
definitions requestQueue
manipulations requestQueue
queries requestQueue
}
func (rs *requestScheduler) ExecuteRequest(req *Request) Status
func (rs *requestScheduler) staticValidityCheck(req *Request) Status
func (rs *requestScheduler) setTimestamp(req *Request)
func (rs *requestScheduler) setPrimaryKey(req *Request)
func (rs *requestScheduler) setSegmentId(req *Request)
func (rs *requestScheduler) setProxyId(req *Request)
// @param selection
// bit_0 = 1: select definition queue
// bit_1 = 1: select manipulation queue
// bit_2 = 1: select query queue
// example: if mode = 3, then both definition and manipulation queues are selected
func (rs *requestScheduler) AreRequestsDelivered(ts Timestamp, selection uint32) bool
// ActiveComponent interfaces
func (rs *requestScheduler) Id() String
func (rs *requestScheduler) Status() Status
func (rs *requestScheduler) Clean() Status
func (rs *requestScheduler) Restart() Status
func (rs *requestScheduler) heartbeat()
// protobuf
message ReqSchedulerHeartbeat {
string id
uint64 definition_queue_length
uint64 manipulation_queue_length
uint64 query_queue_length
uint64 num_delivered_definitions
uint64 num_delivered_manipulations
uint64 num_delivered_queries
}
```
#### 6.3 Time Tick
``` go
type timeTick struct {
lastTick Timestamp
currentTick Timestamp
}
func (tt *timeTick) tick() Status
// ActiveComponent interfaces
func (tt *timeTick) ID() String
func (tt *timeTick) Status() Status
func (tt *timeTick) Clean() Status
func (tt *timeTick) Restart() Status
func (tt *timeTick) heartbeat()
// protobuf
message TimeTickHeartbeat {
string id
uint64 last_tick
}
```
## 7. Message Stream
#### 7.1 Overview
#### 7.2 Message Stream
``` go
type TsMsg interface {
SetTs(ts Timestamp)
Ts() Timestamp
}
type TsMsgMarshaler interface {
Marshal(input *TsMsg) ([]byte, Status)
Unmarshal(input []byte) (*TsMsg, Status)
}
type MsgPack struct {
BeginTs Timestamp
EndTs Timestamp
Msgs []*TsMsg
}
type MsgStream interface {
SetMsgMarshaler(marshal *TsMsgMarshaler, unmarshal *TsMsgMarshaler)
Produce(*MsgPack) Status
Consume() *MsgPack // message can be consumed exactly once
}
type PulsarMsgStream struct {
client *pulsar.Client
produceChannels []string
consumeChannels []string
msgMarshaler *TsMsgMarshaler
msgUnmarshaler *TsMsgMarshaler
}
func (ms *PulsarMsgStream) SetMsgMarshaler(marshal *TsMsgMarshaler, unmarshal *TsMsgMarshaler)
func (ms *PulsarMsgStream) Produce(*MsgPack) Status
func (ms *PulsarMsgStream) Consume() *MsgPack //return messages in one time tick
type PulsarTtMsgStream struct {
client *pulsar.Client
produceChannels []string
consumeChannels []string
msgMarshaler *TsMsgMarshaler
msgUnmarshaler *TsMsgMarshaler
inputBuf []*TsMsg
unsolvedBuf []*TsMsg
msgPacks []*MsgPack
}
func (ms *PulsarTtMsgStream) SetMsgMarshaler(marshal *TsMsgMarshaler, unmarshal *TsMsgMarshaler)
func (ms *PulsarTtMsgStream) Produce(*MsgPack) Status
func (ms *PulsarTtMsgStream) Consume() *MsgPack //return messages in one time tick
```
## 8. Query Node
#### 8.1 Collection and Segment Meta
###### 8.1.1 Collection
``` go
type Collection struct {
Name string
Id uint64
Fields map[string]FieldMeta
SegmentsId []uint64
cCollectionSchema C.CCollectionSchema
}
```
###### 8.1.2 Field Meta
```go
type FieldMeta struct {
Name string
Id uint64
IsPrimaryKey bool
TypeParams map[string]string
IndexParams map[string]string
}
```
###### 8.1.3 Segment
``` go
type Segment struct {
Id uint64
ParitionName string
CollectionId uint64
OpenTime Timestamp
CloseTime Timestamp
NumRows uint64
cSegment C.CSegmentBase
}
```
#### 8.2 Message Streams
```go
type ManipulationReqUnmarshaler struct {}
// implementations of MsgUnmarshaler interfaces
func (unmarshaler *InsertMsgUnmarshaler) Unmarshal(input *pulsar.Message) (*TsMsg, Status)
type QueryReqUnmarshaler struct {}
// implementations of MsgUnmarshaler interfaces
func (unmarshaler *QueryReqUnmarshaler) Unmarshal(input *pulsar.Message) (*TsMsg, Status)
```
#### 8.3 Query Node
## 4. Storage Engine
#### 4.X Interfaces
## 5. Master
#### 5.X Interfaces

Binary file not shown.

Binary file not shown.

Binary file not shown.

After

Width:  |  Height:  |  Size: 10 KiB

Binary file not shown.

Binary file not shown.

View File

@ -34,7 +34,7 @@ type ReaderMessageClient struct {
// message channel // message channel
searchChan chan *msgpb.SearchMsg searchChan chan *msgpb.SearchMsg
key2SegChan chan *msgpb.Key2SegMsg Key2SegChan chan *msgpb.Key2SegMsg
// pulsar // pulsar
client pulsar.Client client pulsar.Client
@ -130,7 +130,7 @@ func (mc *ReaderMessageClient) receiveKey2SegMsg() {
if err != nil { if err != nil {
log.Fatal(err) log.Fatal(err)
} }
mc.key2SegChan <- &key2SegMsg mc.Key2SegChan <- &key2SegMsg
mc.key2segConsumer.Ack(msg) mc.key2segConsumer.Ack(msg)
} }
} }
@ -235,7 +235,7 @@ func (mc *ReaderMessageClient) InitClient(ctx context.Context, url string) {
// init channel // init channel
mc.searchChan = make(chan *msgpb.SearchMsg, conf.Config.Reader.SearchChanSize) mc.searchChan = make(chan *msgpb.SearchMsg, conf.Config.Reader.SearchChanSize)
mc.key2SegChan = make(chan *msgpb.Key2SegMsg, conf.Config.Reader.Key2SegChanSize) mc.Key2SegChan = make(chan *msgpb.Key2SegMsg, conf.Config.Reader.Key2SegChanSize)
mc.InsertOrDeleteMsg = make([]*msgpb.InsertOrDeleteMsg, 0) mc.InsertOrDeleteMsg = make([]*msgpb.InsertOrDeleteMsg, 0)
mc.Key2SegMsg = make([]*msgpb.Key2SegMsg, 0) mc.Key2SegMsg = make([]*msgpb.Key2SegMsg, 0)
@ -335,9 +335,9 @@ func (mc *ReaderMessageClient) PrepareBatchMsg() []int {
func (mc *ReaderMessageClient) PrepareKey2SegmentMsg() { func (mc *ReaderMessageClient) PrepareKey2SegmentMsg() {
mc.Key2SegMsg = mc.Key2SegMsg[:0] mc.Key2SegMsg = mc.Key2SegMsg[:0]
msgLen := len(mc.key2SegChan) msgLen := len(mc.Key2SegChan)
for i := 0; i < msgLen; i++ { for i := 0; i < msgLen; i++ {
msg := <-mc.key2SegChan msg := <-mc.Key2SegChan
mc.Key2SegMsg = append(mc.Key2SegMsg, msg) mc.Key2SegMsg = append(mc.Key2SegMsg, msg)
} }
} }

View File

@ -22,9 +22,9 @@ type Collection struct {
func (c *Collection) NewPartition(partitionName string) *Partition { func (c *Collection) NewPartition(partitionName string) *Partition {
/* /*
CPartition CPartition
NewPartition(CCollection collection, const char* partition_name); NewPartition(CCollection collection, const char* partition_name);
*/ */
cName := C.CString(partitionName) cName := C.CString(partitionName)
partitionPtr := C.NewPartition(c.CollectionPtr, cName) partitionPtr := C.NewPartition(c.CollectionPtr, cName)
@ -33,13 +33,25 @@ func (c *Collection) NewPartition(partitionName string) *Partition {
return newPartition return newPartition
} }
func (c *Collection) DeletePartition(partition *Partition) { func (c *Collection) DeletePartition(node *QueryNode, partition *Partition) {
/* /*
void void
DeletePartition(CPartition partition); DeletePartition(CPartition partition);
*/ */
cPtr := partition.PartitionPtr cPtr := partition.PartitionPtr
C.DeletePartition(cPtr) C.DeletePartition(cPtr)
// TODO: remove from c.Partitions tmpPartitions := make([]*Partition, 0)
for _, p := range c.Partitions {
if p.PartitionName == partition.PartitionName {
for _, s := range p.Segments {
delete(node.SegmentsMap, s.SegmentId)
}
} else {
tmpPartitions = append(tmpPartitions, p)
}
}
c.Partitions = tmpPartitions
} }

View File

@ -0,0 +1,37 @@
package reader
import (
"context"
"github.com/stretchr/testify/assert"
"testing"
)
func TestCollection_NewPartition(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, collection.CollectionID, uint64(0))
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, len(collection.Partitions), 1)
}
func TestCollection_DeletePartition(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, collection.CollectionID, uint64(0))
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, len(collection.Partitions), 1)
collection.DeletePartition(node, partition)
assert.Equal(t, len(collection.Partitions), 0)
}

View File

@ -18,10 +18,10 @@ import (
type IndexConfig struct{} type IndexConfig struct{}
func (s *Segment) buildIndex(collection* Collection) msgPb.Status { func (s *Segment) BuildIndex(collection *Collection) msgPb.Status {
/* /*
int int
BuildIndex(CCollection c_collection, CSegmentBase c_segment); BuildIndex(CCollection c_collection, CSegmentBase c_segment);
*/ */
var status = C.BuildIndex(collection.CollectionPtr, s.SegmentPtr) var status = C.BuildIndex(collection.CollectionPtr, s.SegmentPtr)
if status != 0 { if status != 0 {
@ -30,8 +30,18 @@ func (s *Segment) buildIndex(collection* Collection) msgPb.Status {
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS} return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
} }
func (s *Segment) dropIndex(fieldName string) msgPb.Status { func (s *Segment) DropIndex(fieldName string) msgPb.Status {
// WARN: Not support yet // WARN: Not support yet
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS} return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
} }
func (node *QueryNode) UpdateIndexes(collection *Collection, indexConfig *string) {
/*
void
UpdateIndexes(CCollection c_collection, const char *index_string);
*/
cCollectionPtr := collection.CollectionPtr
cIndexConfig := C.CString(*indexConfig)
C.UpdateIndexes(cCollectionPtr, cIndexConfig)
}

View File

@ -53,8 +53,148 @@ func TestIndex_BuildIndex(t *testing.T) {
var err = segment.SegmentInsert(offset, &ids, &timestamps, &records) var err = segment.SegmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err) assert.NoError(t, err)
// 6. Close segment, and build index // 6. Build index
err = segment.CloseSegment(collection) segment.BuildIndex(collection)
//assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
// 7. Do search
var queryJson = "{\"field_name\":\"fakevec\",\"num_queries\":1,\"topK\":10}"
var queryRawData = make([]float32, 0)
for i := 0; i < DIM; i++ {
queryRawData = append(queryRawData, float32(i))
}
var vectorRecord = msgPb.VectorRowRecord{
FloatData: queryRawData,
}
query := node.QueryJson2Info(&queryJson)
var searchRes, searchErr = segment.SegmentSearch(query, timestamps[N/2], &vectorRecord)
assert.NoError(t, searchErr)
fmt.Println(searchRes)
// 8. Destruct node, collection, and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
node.Close()
}
func TestIndex_DropIndex(t *testing.T) {
// 1. Construct node, collection, partition and segment
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
// 2. Create ids and timestamps
ids := make([]int64, 0)
timestamps := make([]uint64, 0)
// 3. Create records, use schema below:
// schema_tmp->AddField("fakeVec", DataType::VECTOR_FLOAT, 16);
// schema_tmp->AddField("age", DataType::INT32);
const DIM = 16
const N = 100
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++ {
ids = append(ids, int64(i))
timestamps = append(timestamps, uint64(i))
records = append(records, rawData)
}
// 4. Do PreInsert
var offset = segment.SegmentPreInsert(N)
assert.GreaterOrEqual(t, offset, int64(0))
// 5. Do Insert
var err = segment.SegmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
// 6. Build index
var status = segment.BuildIndex(collection)
//assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
// 7. Do search
var queryJson = "{\"field_name\":\"fakevec\",\"num_queries\":1,\"topK\":10}"
var queryRawData = make([]float32, 0)
for i := 0; i < DIM; i++ {
queryRawData = append(queryRawData, float32(i))
}
var vectorRecord = msgPb.VectorRowRecord{
FloatData: queryRawData,
}
query := node.QueryJson2Info(&queryJson)
var searchRes, searchErr = segment.SegmentSearch(query, timestamps[N/2], &vectorRecord)
assert.NoError(t, searchErr)
fmt.Println(searchRes)
// 8. Drop index
status = segment.DropIndex("fakevec")
assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
// 9. Destruct node, collection, and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
node.Close()
}
func TestIndex_UpdateIndex(t *testing.T) {
// 1. Construct node, collection, partition and segment
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
// 2. Create ids and timestamps
ids := make([]int64, 0)
timestamps := make([]uint64, 0)
// 3. Create records, use schema below:
// schema_tmp->AddField("fakeVec", DataType::VECTOR_FLOAT, 16);
// schema_tmp->AddField("age", DataType::INT32);
const DIM = 16
const N = 100
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++ {
ids = append(ids, int64(i))
timestamps = append(timestamps, uint64(i))
records = append(records, rawData)
}
// 4. Do PreInsert
var offset = segment.SegmentPreInsert(N)
assert.GreaterOrEqual(t, offset, int64(0))
// 5. Do Insert
var err = segment.SegmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
// 6. Build index
segment.BuildIndex(collection)
assert.NoError(t, err) assert.NoError(t, err)
// 7. Do search // 7. Do search
@ -72,9 +212,13 @@ func TestIndex_BuildIndex(t *testing.T) {
assert.NoError(t, searchErr) assert.NoError(t, searchErr)
fmt.Println(searchRes) fmt.Println(searchRes)
// 8. Destruct node, collection, and segment // 8. UpdateIndex
partition.DeleteSegment(segment) newIndex := ""
collection.DeletePartition(partition) node.UpdateIndexes(collection, &newIndex)
// 9. Destruct node, collection, and segment
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
node.Close() node.Close()
} }

View File

@ -0,0 +1,238 @@
package reader
import (
"fmt"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"log"
"sync"
)
func (node *QueryNode) MessagesPreprocess(insertDeleteMessages []*msgPb.InsertOrDeleteMsg, timeRange TimeRange) msgPb.Status {
var tMax = timeRange.timestampMax
// 1. Extract messages before readTimeSync from QueryNodeDataBuffer.
// Set valid bitmap to false.
// If segmentId dose not exist in segments map, creating an new segment.
for i, msg := range node.buffer.InsertDeleteBuffer {
if msg.Timestamp < tMax {
if !node.FoundSegmentBySegmentID(msg.SegmentId) {
collection, _ := node.GetCollectionByCollectionName(msg.CollectionName)
if collection != nil {
partition := collection.GetPartitionByName(msg.PartitionTag)
if partition != nil {
newSegment := partition.NewSegment(msg.SegmentId)
node.SegmentsMap[msg.SegmentId] = newSegment
} else {
log.Fatal("Cannot find partition:", msg.PartitionTag)
}
} else {
log.Fatal("Cannot find collection:", msg.CollectionName)
}
}
if msg.Op == msgPb.OpType_INSERT {
if msg.RowsData == nil {
continue
}
node.insertData.insertIDs[msg.SegmentId] = append(node.insertData.insertIDs[msg.SegmentId], msg.Uid)
node.insertData.insertTimestamps[msg.SegmentId] = append(node.insertData.insertTimestamps[msg.SegmentId], msg.Timestamp)
node.insertData.insertRecords[msg.SegmentId] = append(node.insertData.insertRecords[msg.SegmentId], msg.RowsData.Blob)
} else if msg.Op == msgPb.OpType_DELETE {
var r = DeleteRecord{
entityID: msg.Uid,
timestamp: msg.Timestamp,
}
node.deletePreprocessData.deleteRecords = append(node.deletePreprocessData.deleteRecords, &r)
node.deletePreprocessData.count++
}
node.buffer.validInsertDeleteBuffer[i] = false
}
}
// 2. Remove invalid messages from buffer.
tmpInsertOrDeleteBuffer := make([]*msgPb.InsertOrDeleteMsg, 0)
for i, isValid := range node.buffer.validInsertDeleteBuffer {
if isValid {
tmpInsertOrDeleteBuffer = append(tmpInsertOrDeleteBuffer, node.buffer.InsertDeleteBuffer[i])
}
}
node.buffer.InsertDeleteBuffer = tmpInsertOrDeleteBuffer
// 3. Resize the valid bitmap and set all bits to true.
node.buffer.validInsertDeleteBuffer = node.buffer.validInsertDeleteBuffer[:len(node.buffer.InsertDeleteBuffer)]
for i := range node.buffer.validInsertDeleteBuffer {
node.buffer.validInsertDeleteBuffer[i] = true
}
// 4. Extract messages before readTimeSync from current messageClient.
// Move massages after readTimeSync to QueryNodeDataBuffer.
// Set valid bitmap to true.
// If segmentId dose not exist in segments map, creating an new segment.
for _, msg := range insertDeleteMessages {
if msg.Timestamp < tMax {
if !node.FoundSegmentBySegmentID(msg.SegmentId) {
collection, _ := node.GetCollectionByCollectionName(msg.CollectionName)
if collection != nil {
partition := collection.GetPartitionByName(msg.PartitionTag)
if partition != nil {
newSegment := partition.NewSegment(msg.SegmentId)
node.SegmentsMap[msg.SegmentId] = newSegment
} else {
log.Fatal("Cannot find partition:", msg.PartitionTag)
}
} else {
log.Fatal("Cannot find collection:", msg.CollectionName)
}
}
if msg.Op == msgPb.OpType_INSERT {
if msg.RowsData == nil {
continue
}
node.insertData.insertIDs[msg.SegmentId] = append(node.insertData.insertIDs[msg.SegmentId], msg.Uid)
node.insertData.insertTimestamps[msg.SegmentId] = append(node.insertData.insertTimestamps[msg.SegmentId], msg.Timestamp)
node.insertData.insertRecords[msg.SegmentId] = append(node.insertData.insertRecords[msg.SegmentId], msg.RowsData.Blob)
} else if msg.Op == msgPb.OpType_DELETE {
var r = DeleteRecord{
entityID: msg.Uid,
timestamp: msg.Timestamp,
}
node.deletePreprocessData.deleteRecords = append(node.deletePreprocessData.deleteRecords, &r)
node.deletePreprocessData.count++
}
} else {
node.buffer.InsertDeleteBuffer = append(node.buffer.InsertDeleteBuffer, msg)
node.buffer.validInsertDeleteBuffer = append(node.buffer.validInsertDeleteBuffer, true)
}
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) WriterDelete() msgPb.Status {
// TODO: set timeout
for {
if node.deletePreprocessData.count == 0 {
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
node.messageClient.PrepareKey2SegmentMsg()
var ids, timestamps, segmentIDs = node.GetKey2Segments()
for i := 0; i < len(*ids); i++ {
id := (*ids)[i]
timestamp := (*timestamps)[i]
segmentID := (*segmentIDs)[i]
for _, r := range node.deletePreprocessData.deleteRecords {
if r.timestamp == timestamp && r.entityID == id {
r.segmentID = segmentID
node.deletePreprocessData.count--
}
}
}
}
}
func (node *QueryNode) PreInsertAndDelete() msgPb.Status {
// 1. Do PreInsert
for segmentID := range node.insertData.insertRecords {
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
var numOfRecords = len(node.insertData.insertRecords[segmentID])
var offset = targetSegment.SegmentPreInsert(numOfRecords)
node.insertData.insertOffset[segmentID] = offset
}
// 2. Sort delete preprocess data by segment id
for _, r := range node.deletePreprocessData.deleteRecords {
node.deleteData.deleteIDs[r.segmentID] = append(node.deleteData.deleteIDs[r.segmentID], r.entityID)
node.deleteData.deleteTimestamps[r.segmentID] = append(node.deleteData.deleteTimestamps[r.segmentID], r.timestamp)
}
// 3. Do PreDelete
for segmentID := range node.deleteData.deleteIDs {
if segmentID < 0 {
continue
}
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
var numOfRecords = len(node.deleteData.deleteIDs[segmentID])
var offset = targetSegment.SegmentPreDelete(numOfRecords)
node.deleteData.deleteOffset[segmentID] = offset
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoInsertAndDelete() msgPb.Status {
var wg sync.WaitGroup
// Do insert
for segmentID := range node.insertData.insertRecords {
wg.Add(1)
go node.DoInsert(segmentID, &wg)
}
// Do delete
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)
fmt.Println("Do delete done")
}
wg.Wait()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoInsert(segmentID int64, wg *sync.WaitGroup) msgPb.Status {
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
ids := node.insertData.insertIDs[segmentID]
timestamps := node.insertData.insertTimestamps[segmentID]
records := node.insertData.insertRecords[segmentID]
offsets := node.insertData.insertOffset[segmentID]
err = targetSegment.SegmentInsert(offsets, &ids, &timestamps, &records)
fmt.Println("Do insert done, len = ", len(node.insertData.insertIDs[segmentID]))
node.QueryLog(len(ids))
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
wg.Done()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoDelete(segmentID int64, deleteIDs *[]int64, deleteTimestamps *[]uint64, wg *sync.WaitGroup) msgPb.Status {
var segment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
offset := node.deleteData.deleteOffset[segmentID]
node.msgCounter.DeleteCounter += int64(len(*deleteIDs))
err = segment.SegmentDelete(offset, deleteIDs, deleteTimestamps)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
wg.Done()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}

View File

@ -0,0 +1,668 @@
package reader
import (
"context"
"encoding/binary"
"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"
"math"
"strconv"
"sync"
"testing"
"time"
)
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, _ := context.WithDeadline(context.Background(), d)
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, _ := context.WithDeadline(context.Background(), d)
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, _ := context.WithDeadline(context.Background(), d)
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, _ := context.WithDeadline(context.Background(), d)
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, _ := context.WithDeadline(context.Background(), d)
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()
}

View File

@ -11,20 +11,20 @@ import (
"time" "time"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/master/kv" "github.com/zilliztech/milvus-distributed/internal/master/kv"
"github.com/zilliztech/milvus-distributed/internal/master/segment" "github.com/zilliztech/milvus-distributed/internal/master/segment"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3"
"go.etcd.io/etcd/mvcc/mvccpb" "go.etcd.io/etcd/mvcc/mvccpb"
) )
const ( const (
CollectonPrefix = "/collection/" CollectionPrefix = "/collection/"
SegmentPrefix = "/segment/" SegmentPrefix = "/segment/"
) )
func GetCollectionObjId(key string) string { func GetCollectionObjId(key string) string {
prefix := path.Join(conf.Config.Etcd.Rootpath, CollectonPrefix) + "/" prefix := path.Join(conf.Config.Etcd.Rootpath, CollectionPrefix) + "/"
return strings.TrimPrefix(key, prefix) return strings.TrimPrefix(key, prefix)
} }
@ -34,7 +34,7 @@ func GetSegmentObjId(key string) string {
} }
func isCollectionObj(key string) bool { func isCollectionObj(key string) bool {
prefix := path.Join(conf.Config.Etcd.Rootpath, CollectonPrefix) + "/" prefix := path.Join(conf.Config.Etcd.Rootpath, CollectionPrefix) + "/"
prefix = strings.TrimSpace(prefix) prefix = strings.TrimSpace(prefix)
// println("prefix is :$", prefix) // println("prefix is :$", prefix)
index := strings.Index(key, prefix) index := strings.Index(key, prefix)
@ -122,7 +122,7 @@ func (node *QueryNode) processSegmentCreate(id string, value string) {
newSegmentID := int64(segment.SegmentID) // todo change all to uint64 newSegmentID := int64(segment.SegmentID) // todo change all to uint64
// start new segment and add it into partition.OpenedSegments // start new segment and add it into partition.OpenedSegments
newSegment := partition.NewSegment(newSegmentID) newSegment := partition.NewSegment(newSegmentID)
newSegment.SegmentStatus = SegmentOpened // newSegment.SegmentStatus = SegmentOpened
newSegment.SegmentCloseTime = segment.CloseTimeStamp newSegment.SegmentCloseTime = segment.CloseTimeStamp
node.SegmentsMap[newSegmentID] = newSegment node.SegmentsMap[newSegmentID] = newSegment
} }
@ -195,10 +195,32 @@ func (node *QueryNode) processModify(key string, msg string) {
func (node *QueryNode) processSegmentDelete(id string) { func (node *QueryNode) processSegmentDelete(id string) {
println("Delete segment: ", id) println("Delete segment: ", id)
segmentId, err := strconv.ParseInt(id, 10, 64)
if err != nil {
log.Println("Cannot parse segment id:" + id)
}
for _, col := range node.Collections {
for _, p := range col.Partitions {
for _, s := range p.Segments {
if s.SegmentId == segmentId {
p.DeleteSegment(node, s)
}
}
}
}
} }
func (node *QueryNode) processCollectionDelete(id string) { func (node *QueryNode) processCollectionDelete(id string) {
println("Delete collection: ", id) println("Delete collection: ", id)
collectionId, err := strconv.ParseInt(id, 10, 64)
if err != nil {
log.Println("Cannot parse collection id:" + id)
}
targetCollection := node.GetCollectionByID(uint64(collectionId))
node.DeleteCollection(targetCollection)
} }
func (node *QueryNode) processDelete(key string) { func (node *QueryNode) processDelete(key string) {
@ -241,7 +263,7 @@ func (node *QueryNode) processResp(resp clientv3.WatchResponse) error {
} }
func (node *QueryNode) loadCollections() error { func (node *QueryNode) loadCollections() error {
keys, values := node.kvBase.LoadWithPrefix(CollectonPrefix) keys, values := node.kvBase.LoadWithPrefix(CollectionPrefix)
for i := range keys { for i := range keys {
objID := GetCollectionObjId(keys[i]) objID := GetCollectionObjId(keys[i])
node.processCollectionCreate(objID, values[i]) node.processCollectionCreate(objID, values[i])

View File

@ -0,0 +1,606 @@
package reader
import (
"context"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/master/collection"
"github.com/zilliztech/milvus-distributed/internal/master/segment"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
"github.com/zilliztech/milvus-distributed/internal/proto/master"
messagePb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"log"
"math"
"sync"
"testing"
"time"
)
func TestMeta_GetCollectionObjId(t *testing.T) {
conf.LoadConfig("config.yaml")
var key = "/collection/collection0"
var collectionObjId1 = GetCollectionObjId(key)
assert.Equal(t, collectionObjId1, "/collection/collection0")
key = "fakeKey"
var collectionObjId2 = GetCollectionObjId(key)
assert.Equal(t, collectionObjId2, "fakeKey")
}
func TestMeta_GetSegmentObjId(t *testing.T) {
conf.LoadConfig("config.yaml")
var key = "/segment/segment0"
var segmentObjId1 = GetSegmentObjId(key)
assert.Equal(t, segmentObjId1, "/segment/segment0")
key = "fakeKey"
var segmentObjId2 = GetSegmentObjId(key)
assert.Equal(t, segmentObjId2, "fakeKey")
}
func TestMeta_isCollectionObj(t *testing.T) {
conf.LoadConfig("config.yaml")
var key = "by-dev/collection/collection0"
var b1 = isCollectionObj(key)
assert.Equal(t, b1, true)
key = "by-dev/segment/segment0"
var b2 = isCollectionObj(key)
assert.Equal(t, b2, false)
}
func TestMeta_isSegmentObj(t *testing.T) {
conf.LoadConfig("config.yaml")
var key = "by-dev/segment/segment0"
var b1 = isSegmentObj(key)
assert.Equal(t, b1, true)
key = "by-dev/collection/collection0"
var b2 = isSegmentObj(key)
assert.Equal(t, b2, false)
}
func TestMeta_isSegmentChannelRangeInQueryNodeChannelRange(t *testing.T) {
conf.LoadConfig("config.yaml")
var s = segment.Segment{
SegmentID: uint64(0),
CollectionID: uint64(0),
PartitionTag: "partition0",
ChannelStart: 0,
ChannelEnd: 128,
OpenTimeStamp: uint64(0),
CloseTimeStamp: uint64(math.MaxUint64),
CollectionName: "collection0",
Status: master.SegmentStatus_OPENED,
Rows: int64(0),
}
var b = isSegmentChannelRangeInQueryNodeChannelRange(&s)
assert.Equal(t, b, true)
s = segment.Segment{
SegmentID: uint64(0),
CollectionID: uint64(0),
PartitionTag: "partition0",
ChannelStart: 128,
ChannelEnd: 256,
OpenTimeStamp: uint64(0),
CloseTimeStamp: uint64(math.MaxUint64),
CollectionName: "collection0",
Status: master.SegmentStatus_OPENED,
Rows: int64(0),
}
b = isSegmentChannelRangeInQueryNodeChannelRange(&s)
assert.Equal(t, b, false)
}
func TestMeta_PrintCollectionStruct(t *testing.T) {
var age = collection.FieldMeta{
FieldName: "age",
Type: messagePb.DataType_INT32,
DIM: int64(1),
}
var vec = collection.FieldMeta{
FieldName: "vec",
Type: messagePb.DataType_VECTOR_FLOAT,
DIM: int64(16),
}
var fieldMetas = []collection.FieldMeta{age, vec}
var c = collection.Collection{
ID: uint64(0),
Name: "collection0",
CreateTime: uint64(0),
Schema: fieldMetas,
SegmentIDs: []uint64{
0, 1, 2,
},
PartitionTags: []string{
"partition0",
},
GrpcMarshalString: "",
IndexParam: nil,
}
printCollectionStruct(&c)
}
func TestMeta_PrintSegmentStruct(t *testing.T) {
var s = segment.Segment{
SegmentID: uint64(0),
CollectionID: uint64(0),
PartitionTag: "partition0",
ChannelStart: 128,
ChannelEnd: 256,
OpenTimeStamp: uint64(0),
CloseTimeStamp: uint64(math.MaxUint64),
CollectionName: "collection0",
Status: master.SegmentStatus_OPENED,
Rows: int64(0),
}
printSegmentStruct(&s)
}
func TestMeta_ProcessCollectionCreate(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCollectionCreate(id, value)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
}
func TestMeta_ProcessSegmentCreate(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
c := node.NewCollection(uint64(0), "test", "")
c.NewPartition("default")
node.processSegmentCreate(id, value)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentId, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
}
func TestMeta_ProcessCreate(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
key1 := "by-dev/collection/0"
msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCreate(key1, msg1)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
key2 := "by-dev/segment/0"
msg2 := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processCreate(key2, msg2)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentId, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
}
func TestMeta_ProcessSegmentModify(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
var c = node.NewCollection(uint64(0), "test", "")
c.NewPartition("default")
node.processSegmentCreate(id, value)
var s = node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentId, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
newValue := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177888," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processSegmentModify(id, newValue)
s = node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentId, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177888))
assert.Equal(t, s.SegmentStatus, 0)
}
func TestMeta_ProcessCollectionModify(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCollectionCreate(id, value)
var c = node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
// TODO: use different index for testing processCollectionModify
newValue := "{\"id\":0,\"name\":\"test_new\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCollectionModify(id, newValue)
c = node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
}
func TestMeta_ProcessModify(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
key1 := "by-dev/collection/0"
msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCreate(key1, msg1)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
key2 := "by-dev/segment/0"
msg2 := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processCreate(key2, msg2)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentId, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
// modify
// TODO: use different index for testing processCollectionModify
msg3 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processModify(key1, msg3)
c = node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
msg4 := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177888," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processModify(key2, msg4)
s = node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentId, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177888))
assert.Equal(t, s.SegmentStatus, 0)
}
func TestMeta_ProcessSegmentDelete(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
c := node.NewCollection(uint64(0), "test", "")
c.NewPartition("default")
node.processSegmentCreate(id, value)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentId, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
node.processSegmentDelete("0")
mapSize := len(node.SegmentsMap)
assert.Equal(t, mapSize, 0)
}
func TestMeta_ProcessCollectionDelete(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
id := "0"
value := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCollectionCreate(id, value)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
node.processCollectionDelete(id)
collectionsSize := len(node.Collections)
assert.Equal(t, collectionsSize, 0)
}
func TestMeta_ProcessDelete(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
key1 := "by-dev/collection/0"
msg1 := "{\"id\":0,\"name\":\"test\",\"creat_time\":1603359905,\"schema\":" +
"[{\"field_name\":\"age\",\"type\":4,\"dimension\":1}," +
"{\"field_name\":\"field_vec\",\"type\":101,\"dimension\":512}]," +
"\"segment_ids\":[6886378356295345384],\"partition_tags\":[\"default\"]," +
"\"grpc_marshal_string\":\"id: 6886378356295345384\\nname: \\\"test\\\"\\nschema: \\u003c\\n " +
"field_metas: \\u003c\\n field_name: \\\"age\\\"\\n type: INT32\\n dim: 1\\n \\u003e\\n " +
"field_metas: \\u003c\\n field_name: \\\"field_vec\\\"\\n type: VECTOR_FLOAT\\n " +
"dim: 512\\n \\u003e\\n\\u003e\\ncreate_time: 1603359905\\nsegment_ids: " +
"6886378356295345384\\npartition_tags: \\\"default\\\"\\n\",\"index_param\":null}"
node.processCreate(key1, msg1)
c := node.Collections[0]
assert.Equal(t, c.CollectionName, "test")
assert.Equal(t, c.CollectionID, uint64(0))
key2 := "by-dev/segment/0"
msg2 := "{\"segment_id\":0,\"collection_id\":0," +
"\"partition_tag\":\"default\",\"channel_start\":0,\"channel_end\":128," +
"\"open_timestamp\":1603360439,\"close_timestamp\":70368744177663," +
"\"collection_name\":\"test\",\"segment_status\":0,\"rows\":0}"
node.processCreate(key2, msg2)
s := node.SegmentsMap[int64(0)]
assert.Equal(t, s.SegmentId, int64(0))
assert.Equal(t, s.SegmentCloseTime, uint64(70368744177663))
assert.Equal(t, s.SegmentStatus, 0)
node.processDelete(key1)
collectionsSize := len(node.Collections)
assert.Equal(t, collectionsSize, 0)
mapSize := len(node.SegmentsMap)
assert.Equal(t, mapSize, 0)
}
func TestMeta_ProcessResp(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
err := node.InitFromMeta()
assert.Nil(t, err)
metaChan := node.kvBase.WatchWithPrefix("")
select {
case <-node.ctx.Done():
return
case resp := <-metaChan:
_ = node.processResp(resp)
}
}
func TestMeta_LoadCollections(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
err := node.InitFromMeta()
assert.Nil(t, err)
err2 := node.loadCollections()
assert.Nil(t, err2)
}
func TestMeta_LoadSegments(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
err := node.InitFromMeta()
assert.Nil(t, err)
err2 := node.loadSegments()
assert.Nil(t, err2)
}
func TestMeta_InitFromMeta(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
mc := msgclient.ReaderMessageClient{}
node := CreateQueryNode(ctx, 0, 0, &mc)
err := node.InitFromMeta()
assert.Nil(t, err)
}
func TestMeta_RunMetaService(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
node := CreateQueryNode(ctx, 0, 0, nil)
wg := sync.WaitGroup{}
err := node.InitFromMeta()
if err != nil {
log.Printf("Init query node from meta failed")
return
}
wg.Add(1)
go node.RunMetaService(&wg)
wg.Wait()
node.Close()
}

View File

@ -31,7 +31,7 @@ func (p *Partition) NewSegment(segmentId int64) *Segment {
return newSegment return newSegment
} }
func (p *Partition) DeleteSegment(segment *Segment) { func (p *Partition) DeleteSegment(node *QueryNode, segment *Segment) {
/* /*
void void
DeleteSegment(CSegmentBase segment); DeleteSegment(CSegmentBase segment);
@ -39,5 +39,15 @@ func (p *Partition) DeleteSegment(segment *Segment) {
cPtr := segment.SegmentPtr cPtr := segment.SegmentPtr
C.DeleteSegment(cPtr) C.DeleteSegment(cPtr)
// TODO: remove from p.Segments tmpSegments := make([]*Segment, 0)
for _, s := range p.Segments {
if s.SegmentId == segment.SegmentId {
delete(node.SegmentsMap, s.SegmentId)
} else {
tmpSegments = append(tmpSegments, s)
}
}
p.Segments = tmpSegments
} }

View File

@ -0,0 +1,61 @@
package reader
import (
"context"
"github.com/stretchr/testify/assert"
"testing"
)
func TestPartition_NewSegment(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, collection.CollectionID, uint64(0))
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, node.Collections[0].Partitions[0].Segments[0].SegmentId, int64(0))
assert.Equal(t, len(collection.Partitions), 1)
assert.Equal(t, len(node.Collections), 1)
assert.Equal(t, len(node.Collections[0].Partitions[0].Segments), 1)
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, node.FoundSegmentBySegmentID(int64(0)), true)
}
func TestPartition_DeleteSegment(t *testing.T) {
// 1. Construct node, collection, partition and segment
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, collection.CollectionID, uint64(0))
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, node.Collections[0].Partitions[0].Segments[0].SegmentId, int64(0))
assert.Equal(t, len(collection.Partitions), 1)
assert.Equal(t, len(node.Collections), 1)
assert.Equal(t, len(node.Collections[0].Partitions[0].Segments), 1)
assert.Equal(t, segment.SegmentId, int64(0))
// 2. Destruct collection, partition and segment
partition.DeleteSegment(node, segment)
assert.Equal(t, len(collection.Partitions), 1)
assert.Equal(t, len(node.Collections), 1)
assert.Equal(t, len(node.Collections[0].Partitions[0].Segments), 0)
assert.Equal(t, node.FoundSegmentBySegmentID(int64(0)), false)
}

View File

@ -15,20 +15,10 @@ import "C"
import ( import (
"context" "context"
"encoding/json"
"fmt"
"log"
"sort"
"sync"
"sync/atomic"
"time"
"github.com/zilliztech/milvus-distributed/internal/conf"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/zilliztech/milvus-distributed/internal/master/kv" "github.com/zilliztech/milvus-distributed/internal/master/kv"
"github.com/zilliztech/milvus-distributed/internal/msgclient" "github.com/zilliztech/milvus-distributed/internal/msgclient"
"github.com/stretchr/testify/assert" msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
//"github.com/stretchr/testify/assert" "time"
) )
type InsertData struct { type InsertData struct {
@ -91,10 +81,10 @@ type QueryNode struct {
// context // context
ctx context.Context ctx context.Context
QueryNodeId uint64 QueryNodeId uint64
Collections []*Collection Collections []*Collection
SegmentsMap map[int64]*Segment SegmentsMap map[int64]*Segment
messageClient *msgclient.ReaderMessageClient messageClient *msgclient.ReaderMessageClient
queryNodeTimeSync *QueryNodeTime queryNodeTimeSync *QueryNodeTime
buffer QueryNodeDataBuffer buffer QueryNodeDataBuffer
deletePreprocessData DeletePreprocessData deletePreprocessData DeletePreprocessData
@ -239,469 +229,19 @@ func (node *QueryNode) DeleteCollection(collection *Collection) {
cPtr := collection.CollectionPtr cPtr := collection.CollectionPtr
C.DeleteCollection(cPtr) C.DeleteCollection(cPtr)
// TODO: remove from node.Collections collectionID := collection.CollectionID
} tmpCollections := make([]*Collection, 0)
for _, col := range node.Collections {
func (node *QueryNode) UpdateIndexes(collection *Collection, indexConfig *string) { if col.CollectionID == collectionID {
/* for _, p := range collection.Partitions {
void for _, s := range p.Segments {
UpdateIndexes(CCollection c_collection, const char *index_string); delete(node.SegmentsMap, s.SegmentId)
*/
cCollectionPtr := collection.CollectionPtr
cIndexConfig := C.CString(*indexConfig)
C.UpdateIndexes(cCollectionPtr, cIndexConfig)
}
////////////////////////////////////////////////////////////////////////////////////////////////////
func (node *QueryNode) PrepareBatchMsg() []int {
var msgLen = node.messageClient.PrepareBatchMsg()
return msgLen
}
func (node *QueryNode) InitQueryNodeCollection() {
// TODO: remove hard code, add collection creation request
// TODO: error handle
var newCollection = node.NewCollection(0, "collection1", "fakeSchema")
var newPartition = newCollection.NewPartition("partition1")
// TODO: add segment id
var segment = newPartition.NewSegment(0)
node.SegmentsMap[0] = segment
}
////////////////////////////////////////////////////////////////////////////////////////////////////
func (node *QueryNode) RunInsertDelete(wg *sync.WaitGroup) {
const Debug = true
const CountInsertMsgBaseline = 1000 * 1000
var BaselineCounter int64 = 0
if Debug {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
default:
var msgLen = node.PrepareBatchMsg()
var timeRange = TimeRange{node.messageClient.TimeSyncStart(), node.messageClient.TimeSyncEnd()}
assert.NotEqual(nil, 0, timeRange.timestampMin)
assert.NotEqual(nil, 0, timeRange.timestampMax)
if node.msgCounter.InsertCounter/CountInsertMsgBaseline != BaselineCounter {
node.WriteQueryLog()
BaselineCounter = node.msgCounter.InsertCounter / CountInsertMsgBaseline
} }
if msgLen[0] == 0 && len(node.buffer.InsertDeleteBuffer) <= 0 {
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
continue
}
node.QueryNodeDataInit()
node.MessagesPreprocess(node.messageClient.InsertOrDeleteMsg, timeRange)
//fmt.Println("MessagesPreprocess Done")
node.WriterDelete()
node.PreInsertAndDelete()
//fmt.Println("PreInsertAndDelete Done")
node.DoInsertAndDelete()
//fmt.Println("DoInsertAndDelete Done")
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
}
}
} else {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
default:
var msgLen = node.PrepareBatchMsg()
var timeRange = TimeRange{node.messageClient.TimeSyncStart(), node.messageClient.TimeSyncEnd()}
assert.NotEqual(nil, 0, timeRange.timestampMin)
assert.NotEqual(nil, 0, timeRange.timestampMax)
if msgLen[0] == 0 && len(node.buffer.InsertDeleteBuffer) <= 0 {
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
continue
}
node.QueryNodeDataInit()
node.MessagesPreprocess(node.messageClient.InsertOrDeleteMsg, timeRange)
//fmt.Println("MessagesPreprocess Done")
node.WriterDelete()
node.PreInsertAndDelete()
//fmt.Println("PreInsertAndDelete Done")
node.DoInsertAndDelete()
//fmt.Println("DoInsertAndDelete Done")
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
}
}
}
wg.Done()
}
func (node *QueryNode) TestInsertDelete(timeRange TimeRange) {
node.QueryNodeDataInit()
node.MessagesPreprocess(node.messageClient.InsertOrDeleteMsg, timeRange)
fmt.Println("MessagesPreprocess Done")
node.WriterDelete()
node.PreInsertAndDelete()
fmt.Println("PreInsertAndDelete Done")
node.DoInsertAndDelete()
fmt.Println("DoInsertAndDelete Done")
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
fmt.Print("UpdateSearchTimeSync Done\n\n\n")
}
func (node *QueryNode) RunSearch(wg *sync.WaitGroup) {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
case msg := <-node.messageClient.GetSearchChan():
node.messageClient.SearchMsg = node.messageClient.SearchMsg[:0]
node.messageClient.SearchMsg = append(node.messageClient.SearchMsg, msg)
//for {
//if node.messageClient.SearchMsg[0].Timestamp < node.queryNodeTimeSync.ServiceTimeSync {
var status = node.Search(node.messageClient.SearchMsg)
fmt.Println("Do Search done")
if status.ErrorCode != 0 {
fmt.Println("Search Failed")
node.PublishFailedSearchResult()
}
//break
//}
//}
default:
}
}
wg.Done()
}
////////////////////////////////////////////////////////////////////////////////////////////////////
func (node *QueryNode) MessagesPreprocess(insertDeleteMessages []*msgPb.InsertOrDeleteMsg, timeRange TimeRange) msgPb.Status {
var tMax = timeRange.timestampMax
// 1. Extract messages before readTimeSync from QueryNodeDataBuffer.
// Set valid bitmap to false.
for i, msg := range node.buffer.InsertDeleteBuffer {
if msg.Timestamp < tMax {
if msg.Op == msgPb.OpType_INSERT {
if msg.RowsData == nil {
continue
}
node.insertData.insertIDs[msg.SegmentId] = append(node.insertData.insertIDs[msg.SegmentId], msg.Uid)
node.insertData.insertTimestamps[msg.SegmentId] = append(node.insertData.insertTimestamps[msg.SegmentId], msg.Timestamp)
node.insertData.insertRecords[msg.SegmentId] = append(node.insertData.insertRecords[msg.SegmentId], msg.RowsData.Blob)
} else if msg.Op == msgPb.OpType_DELETE {
var r = DeleteRecord{
entityID: msg.Uid,
timestamp: msg.Timestamp,
}
node.deletePreprocessData.deleteRecords = append(node.deletePreprocessData.deleteRecords, &r)
atomic.AddInt32(&node.deletePreprocessData.count, 1)
}
node.buffer.validInsertDeleteBuffer[i] = false
}
}
// 2. Remove invalid messages from buffer.
tmpInsertOrDeleteBuffer := make([]*msgPb.InsertOrDeleteMsg, 0)
for i, isValid := range node.buffer.validInsertDeleteBuffer {
if isValid {
tmpInsertOrDeleteBuffer = append(tmpInsertOrDeleteBuffer, node.buffer.InsertDeleteBuffer[i])
}
}
node.buffer.InsertDeleteBuffer = tmpInsertOrDeleteBuffer
// 3. Resize the valid bitmap and set all bits to true.
node.buffer.validInsertDeleteBuffer = node.buffer.validInsertDeleteBuffer[:len(node.buffer.InsertDeleteBuffer)]
for i := range node.buffer.validInsertDeleteBuffer {
node.buffer.validInsertDeleteBuffer[i] = true
}
// 4. Extract messages before readTimeSync from current messageClient.
// Move massages after readTimeSync to QueryNodeDataBuffer.
// Set valid bitmap to true.
for _, msg := range insertDeleteMessages {
if msg.Timestamp < tMax {
if msg.Op == msgPb.OpType_INSERT {
if msg.RowsData == nil {
continue
}
node.insertData.insertIDs[msg.SegmentId] = append(node.insertData.insertIDs[msg.SegmentId], msg.Uid)
node.insertData.insertTimestamps[msg.SegmentId] = append(node.insertData.insertTimestamps[msg.SegmentId], msg.Timestamp)
node.insertData.insertRecords[msg.SegmentId] = append(node.insertData.insertRecords[msg.SegmentId], msg.RowsData.Blob)
} else if msg.Op == msgPb.OpType_DELETE {
var r = DeleteRecord{
entityID: msg.Uid,
timestamp: msg.Timestamp,
}
node.deletePreprocessData.deleteRecords = append(node.deletePreprocessData.deleteRecords, &r)
atomic.AddInt32(&node.deletePreprocessData.count, 1)
} }
} else { } else {
node.buffer.InsertDeleteBuffer = append(node.buffer.InsertDeleteBuffer, msg) tmpCollections = append(tmpCollections, col)
node.buffer.validInsertDeleteBuffer = append(node.buffer.validInsertDeleteBuffer, true)
} }
} }
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS} node.Collections = tmpCollections
}
func (node *QueryNode) WriterDelete() msgPb.Status {
// TODO: set timeout
for {
if node.deletePreprocessData.count == 0 {
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
node.messageClient.PrepareKey2SegmentMsg()
var ids, timestamps, segmentIDs = node.GetKey2Segments()
for i := 0; i < len(*ids); i++ {
id := (*ids)[i]
timestamp := (*timestamps)[i]
segmentID := (*segmentIDs)[i]
for _, r := range node.deletePreprocessData.deleteRecords {
if r.timestamp == timestamp && r.entityID == id {
r.segmentID = segmentID
atomic.AddInt32(&node.deletePreprocessData.count, -1)
}
}
}
}
}
func (node *QueryNode) PreInsertAndDelete() msgPb.Status {
// 1. Do PreInsert
for segmentID := range node.insertData.insertRecords {
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
var numOfRecords = len(node.insertData.insertRecords[segmentID])
var offset = targetSegment.SegmentPreInsert(numOfRecords)
node.insertData.insertOffset[segmentID] = offset
}
// 2. Sort delete preprocess data by segment id
for _, r := range node.deletePreprocessData.deleteRecords {
node.deleteData.deleteIDs[r.segmentID] = append(node.deleteData.deleteIDs[r.segmentID], r.entityID)
node.deleteData.deleteTimestamps[r.segmentID] = append(node.deleteData.deleteTimestamps[r.segmentID], r.timestamp)
}
// 3. Do PreDelete
for segmentID := range node.deleteData.deleteIDs {
if segmentID < 0 {
continue
}
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
var numOfRecords = len(node.deleteData.deleteIDs[segmentID])
var offset = targetSegment.SegmentPreDelete(numOfRecords)
node.deleteData.deleteOffset[segmentID] = offset
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoInsertAndDelete() msgPb.Status {
var wg sync.WaitGroup
// Do insert
for segmentID := range node.insertData.insertRecords {
wg.Add(1)
go node.DoInsert(segmentID, &wg)
}
// Do delete
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)
fmt.Println("Do delete done")
}
wg.Wait()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoInsert(segmentID int64, wg *sync.WaitGroup) msgPb.Status {
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
ids := node.insertData.insertIDs[segmentID]
timestamps := node.insertData.insertTimestamps[segmentID]
records := node.insertData.insertRecords[segmentID]
offsets := node.insertData.insertOffset[segmentID]
err = targetSegment.SegmentInsert(offsets, &ids, &timestamps, &records)
fmt.Println("Do insert done, len = ", len(node.insertData.insertIDs[segmentID]))
node.QueryLog(len(ids))
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
wg.Done()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) DoDelete(segmentID int64, deleteIDs *[]int64, deleteTimestamps *[]uint64, wg *sync.WaitGroup) msgPb.Status {
var segment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
offset := node.deleteData.deleteOffset[segmentID]
node.msgCounter.DeleteCounter += int64(len(*deleteIDs))
err = segment.SegmentDelete(offset, deleteIDs, deleteTimestamps)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
wg.Done()
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func (node *QueryNode) QueryJson2Info(queryJson *string) *QueryInfo {
var query QueryInfo
var err = json.Unmarshal([]byte(*queryJson), &query)
if err != nil {
log.Printf("Unmarshal query json failed")
return nil
}
//fmt.Println(query)
return &query
}
func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg) msgPb.Status {
type SearchResultTmp struct {
ResultId int64
ResultDistance float32
}
node.msgCounter.SearchCounter += int64(len(searchMessages))
// Traverse all messages in the current messageClient.
// TODO: Do not receive batched search requests
for _, msg := range searchMessages {
var clientId = msg.ClientId
var searchTimestamp = msg.Timestamp
// ServiceTimeSync update by TimeSync, which is get from proxy.
// Proxy send this timestamp per `conf.Config.Timesync.Interval` milliseconds.
// However, timestamp of search request (searchTimestamp) is precision time.
// So the ServiceTimeSync is always less than searchTimestamp.
// Here, we manually make searchTimestamp's logic time minus `conf.Config.Timesync.Interval` milliseconds.
// Which means `searchTimestamp.logicTime = searchTimestamp.logicTime - conf.Config.Timesync.Interval`.
var logicTimestamp = searchTimestamp << 46 >> 46
searchTimestamp = (searchTimestamp>>18-uint64(conf.Config.Timesync.Interval+600))<<18 + logicTimestamp
var vector = msg.Records
// We now only the first Json is valid.
var queryJson = msg.Json[0]
// 1. Timestamp check
// TODO: return or wait? Or adding graceful time
if searchTimestamp > node.queryNodeTimeSync.ServiceTimeSync {
fmt.Println("Invalid query time, timestamp = ", searchTimestamp>>18, ", SearchTimeSync = ", node.queryNodeTimeSync.ServiceTimeSync>>18)
return msgPb.Status{ErrorCode: 1}
}
// 2. Get query information from query json
query := node.QueryJson2Info(&queryJson)
// 2d slice for receiving multiple queries's results
var resultsTmp = make([][]SearchResultTmp, query.NumQueries)
for i := 0; i < int(query.NumQueries); i++ {
resultsTmp[i] = make([]SearchResultTmp, 0)
}
// 3. Do search in all segments
for _, segment := range node.SegmentsMap {
if segment.GetRowCount() <= 0 {
// Skip empty segment
continue
}
//fmt.Println("Search in segment:", segment.SegmentId, ",segment rows:", segment.GetRowCount())
var res, err = segment.SegmentSearch(query, searchTimestamp, vector)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
for i := 0; i < int(query.NumQueries); i++ {
for j := i * query.TopK; j < (i+1)*query.TopK; j++ {
resultsTmp[i] = append(resultsTmp[i], SearchResultTmp{
ResultId: res.ResultIds[j],
ResultDistance: res.ResultDistances[j],
})
}
}
}
// 4. Reduce results
for _, rTmp := range resultsTmp {
sort.Slice(rTmp, func(i, j int) bool {
return rTmp[i].ResultDistance < rTmp[j].ResultDistance
})
}
for _, rTmp := range resultsTmp {
if len(rTmp) > query.TopK {
rTmp = rTmp[:query.TopK]
}
}
var entities = msgPb.Entities{
Ids: make([]int64, 0),
}
var results = msgPb.QueryResult{
Status: &msgPb.Status{
ErrorCode: 0,
},
Entities: &entities,
Distances: make([]float32, 0),
QueryId: uint64(msg.Uid),
ProxyId: clientId,
}
for _, rTmp := range resultsTmp {
for _, res := range rTmp {
results.Entities.Ids = append(results.Entities.Ids, res.ResultId)
results.Distances = append(results.Distances, res.ResultDistance)
results.Scores = append(results.Distances, float32(0))
}
}
// Send numQueries to RowNum.
results.RowNum = query.NumQueries
// 5. publish result to pulsar
//fmt.Println(results.Entities.Ids)
//fmt.Println(results.Distances)
node.PublishSearchResult(&results)
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
} }

View File

@ -4,13 +4,13 @@ type QueryNodeTime struct {
ReadTimeSyncMin uint64 ReadTimeSyncMin uint64
ReadTimeSyncMax uint64 ReadTimeSyncMax uint64
WriteTimeSync uint64 WriteTimeSync uint64
ServiceTimeSync uint64 ServiceTimeSync uint64
TSOTimeSync uint64 TSOTimeSync uint64
} }
type TimeRange struct { type TimeRange struct {
timestampMin uint64 timestampMin uint64
timestampMax uint64 timestampMax uint64
} }
func (t *QueryNodeTime) UpdateReadTimeSync() { func (t *QueryNodeTime) UpdateReadTimeSync() {

View File

@ -0,0 +1,46 @@
package reader
import (
"github.com/stretchr/testify/assert"
"testing"
)
func TestQueryNodeTime_UpdateReadTimeSync(t *testing.T) {
queryNodeTimeSync := &QueryNodeTime{
ReadTimeSyncMin: uint64(0),
ReadTimeSyncMax: uint64(1),
WriteTimeSync: uint64(2),
ServiceTimeSync: uint64(3),
TSOTimeSync: uint64(4),
}
queryNodeTimeSync.UpdateReadTimeSync()
assert.Equal(t, queryNodeTimeSync.ReadTimeSyncMin, uint64(1))
}
func TestQueryNodeTime_UpdateSearchTimeSync(t *testing.T) {
queryNodeTimeSync := &QueryNodeTime{
ReadTimeSyncMin: uint64(0),
ReadTimeSyncMax: uint64(1),
WriteTimeSync: uint64(2),
ServiceTimeSync: uint64(3),
TSOTimeSync: uint64(4),
}
timeRange := TimeRange{
timestampMin: 0,
timestampMax: 1,
}
queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
assert.Equal(t, queryNodeTimeSync.ServiceTimeSync, uint64(1))
}
func TestQueryNodeTime_UpdateTSOTimeSync(t *testing.T) {
// TODO: add UpdateTSOTimeSync test
}
func TestQueryNodeTime_UpdateWriteTimeSync(t *testing.T) {
// TODO: add UpdateWriteTimeSync test
}

View File

@ -1,12 +1,78 @@
package reader package reader
import ( import (
"context"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf"
"testing" "testing"
) )
// TODO: add query node test func TestQueryNode_CreateQueryNode(t *testing.T) {
conf.LoadConfig("config.yaml")
func TestQueryNode_RunInsertDelete(t *testing.T) { ctx, _ := context.WithCancel(context.Background())
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
} }
func TestQueryNode_NewQueryNode(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, _ := context.WithCancel(context.Background())
node := NewQueryNode(ctx, 0, 0)
assert.NotNil(t, node)
}
func TestQueryNode_Close(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, _ := context.WithCancel(context.Background())
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
node.Close()
}
func TestQueryNode_QueryNodeDataInit(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, _ := context.WithCancel(context.Background())
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
node.QueryNodeDataInit()
assert.NotNil(t, node.deletePreprocessData)
assert.NotNil(t, node.insertData)
assert.NotNil(t, node.deleteData)
}
func TestQueryNode_NewCollection(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, _ := context.WithCancel(context.Background())
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
var collection = node.NewCollection(0, "collection0", "")
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, len(node.Collections), 1)
}
func TestQueryNode_DeleteCollection(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, _ := context.WithCancel(context.Background())
node := CreateQueryNode(ctx, 0, 0, nil)
assert.NotNil(t, node)
var collection = node.NewCollection(0, "collection0", "")
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, len(node.Collections), 1)
node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
}

View File

@ -2,6 +2,8 @@ package reader
import ( import (
"context" "context"
"fmt"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/msgclient" "github.com/zilliztech/milvus-distributed/internal/msgclient"
"log" "log"
"sync" "sync"
@ -15,7 +17,6 @@ func StartQueryNode(ctx context.Context, pulsarURL string) {
qn := CreateQueryNode(ctx, 0, 0, &mc) qn := CreateQueryNode(ctx, 0, 0, &mc)
// Segments Services // Segments Services
go qn.SegmentManagementService()
go qn.SegmentStatisticService() go qn.SegmentStatisticService()
wg := sync.WaitGroup{} wg := sync.WaitGroup{}
@ -33,3 +34,99 @@ func StartQueryNode(ctx context.Context, pulsarURL string) {
wg.Wait() wg.Wait()
qn.Close() qn.Close()
} }
func (node *QueryNode) RunInsertDelete(wg *sync.WaitGroup) {
const Debug = true
const CountInsertMsgBaseline = 1000 * 1000
var BaselineCounter int64 = 0
if Debug {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
default:
var msgLen = node.PrepareBatchMsg()
var timeRange = TimeRange{node.messageClient.TimeSyncStart(), node.messageClient.TimeSyncEnd()}
assert.NotEqual(nil, 0, timeRange.timestampMin)
assert.NotEqual(nil, 0, timeRange.timestampMax)
if node.msgCounter.InsertCounter/CountInsertMsgBaseline != BaselineCounter {
node.WriteQueryLog()
BaselineCounter = node.msgCounter.InsertCounter / CountInsertMsgBaseline
}
if msgLen[0] == 0 && len(node.buffer.InsertDeleteBuffer) <= 0 {
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
continue
}
node.QueryNodeDataInit()
node.MessagesPreprocess(node.messageClient.InsertOrDeleteMsg, timeRange)
//fmt.Println("MessagesPreprocess Done")
node.WriterDelete()
node.PreInsertAndDelete()
//fmt.Println("PreInsertAndDelete Done")
node.DoInsertAndDelete()
//fmt.Println("DoInsertAndDelete Done")
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
}
}
} else {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
default:
var msgLen = node.PrepareBatchMsg()
var timeRange = TimeRange{node.messageClient.TimeSyncStart(), node.messageClient.TimeSyncEnd()}
assert.NotEqual(nil, 0, timeRange.timestampMin)
assert.NotEqual(nil, 0, timeRange.timestampMax)
if msgLen[0] == 0 && len(node.buffer.InsertDeleteBuffer) <= 0 {
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
continue
}
node.QueryNodeDataInit()
node.MessagesPreprocess(node.messageClient.InsertOrDeleteMsg, timeRange)
//fmt.Println("MessagesPreprocess Done")
node.WriterDelete()
node.PreInsertAndDelete()
//fmt.Println("PreInsertAndDelete Done")
node.DoInsertAndDelete()
//fmt.Println("DoInsertAndDelete Done")
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
}
}
}
wg.Done()
}
func (node *QueryNode) RunSearch(wg *sync.WaitGroup) {
for {
select {
case <-node.ctx.Done():
wg.Done()
return
case msg := <-node.messageClient.GetSearchChan():
node.messageClient.SearchMsg = node.messageClient.SearchMsg[:0]
node.messageClient.SearchMsg = append(node.messageClient.SearchMsg, msg)
//for {
//if node.messageClient.SearchMsg[0].Timestamp < node.queryNodeTimeSync.ServiceTimeSync {
var status = node.Search(node.messageClient.SearchMsg)
fmt.Println("Do Search done")
if status.ErrorCode != 0 {
fmt.Println("Search Failed")
node.PublishFailedSearchResult()
}
//break
//}
//}
default:
}
}
wg.Done()
}

View File

@ -2,15 +2,18 @@ package reader
import ( import (
"context" "context"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient"
"strconv" "strconv"
"sync"
"testing" "testing"
"time" "time"
) )
const ctxTimeInMillisecond = 10 const ctxTimeInMillisecond = 10
// NOTE: start pulsar before test // NOTE: start pulsar and etcd before test
func TestReader_startQueryNode(t *testing.T) { func TestReader_startQueryNode(t *testing.T) {
conf.LoadConfig("config.yaml") conf.LoadConfig("config.yaml")
@ -21,5 +24,69 @@ func TestReader_startQueryNode(t *testing.T) {
pulsarAddr += conf.Config.Pulsar.Address pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":" pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
StartQueryNode(ctx, pulsarAddr) StartQueryNode(ctx, pulsarAddr)
// To make sure to get here
assert.Equal(t, 0, 0)
}
// NOTE: start pulsar before test
func TestReader_RunInsertDelete(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
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)
wg := sync.WaitGroup{}
wg.Add(1)
go node.RunInsertDelete(&wg)
wg.Wait()
node.Close()
// To make sure to get here
assert.Equal(t, 0, 0)
}
// NOTE: start pulsar before test
func TestReader_RunSearch(t *testing.T) {
conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d)
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)
wg := sync.WaitGroup{}
wg.Add(1)
go node.RunSearch(&wg)
wg.Wait()
node.Close()
// To make sure to get here
assert.Equal(t, 0, 0)
} }

View File

@ -2,6 +2,7 @@ package reader
import ( import (
"context" "context"
"github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient" "github.com/zilliztech/milvus-distributed/internal/msgclient"
"strconv" "strconv"
@ -24,8 +25,8 @@ func TestResult_PublishSearchResult(t *testing.T) {
pulsarAddr += conf.Config.Pulsar.Address pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":" pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.InitClient(ctx, pulsarAddr)
node := CreateQueryNode(ctx, 0, 0, &mc) node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment // Construct node, collection, partition and segment
@ -46,7 +47,10 @@ func TestResult_PublishSearchResult(t *testing.T) {
result.Entities.Ids = append(result.Entities.Ids, int64(i)) result.Entities.Ids = append(result.Entities.Ids, int64(i))
result.Distances = append(result.Distances, float32(i)) result.Distances = append(result.Distances, float32(i))
} }
node.PublishSearchResult(&result)
status := node.PublishSearchResult(&result)
assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
node.Close() node.Close()
} }
@ -62,8 +66,8 @@ func TestResult_PublishFailedSearchResult(t *testing.T) {
pulsarAddr += conf.Config.Pulsar.Address pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":" pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.InitClient(ctx, pulsarAddr)
node := CreateQueryNode(ctx, 0, 0, &mc) node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment // Construct node, collection, partition and segment
@ -72,8 +76,8 @@ func TestResult_PublishFailedSearchResult(t *testing.T) {
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[0] = segment node.SegmentsMap[0] = segment
// TODO: start pulsar server status := node.PublishFailedSearchResult()
node.PublishFailedSearchResult() assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
node.Close() node.Close()
} }
@ -90,8 +94,8 @@ func TestResult_PublicStatistic(t *testing.T) {
pulsarAddr += conf.Config.Pulsar.Address pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":" pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.InitClient(ctx, pulsarAddr)
node := CreateQueryNode(ctx, 0, 0, &mc) node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment // Construct node, collection, partition and segment
@ -113,8 +117,8 @@ func TestResult_PublicStatistic(t *testing.T) {
statisticData = append(statisticData, stat) statisticData = append(statisticData, stat)
} }
// TODO: start pulsar server status := node.PublicStatistic(&statisticData)
node.PublicStatistic(&statisticData) assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
node.Close() node.Close()
} }

119
internal/reader/search.go Normal file
View File

@ -0,0 +1,119 @@
package reader
import (
"fmt"
"github.com/zilliztech/milvus-distributed/internal/conf"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"sort"
)
func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg) msgPb.Status {
type SearchResultTmp struct {
ResultId int64
ResultDistance float32
}
node.msgCounter.SearchCounter += int64(len(searchMessages))
// Traverse all messages in the current messageClient.
// TODO: Do not receive batched search requests
for _, msg := range searchMessages {
var clientId = msg.ClientId
var searchTimestamp = msg.Timestamp
// ServiceTimeSync update by TimeSync, which is get from proxy.
// Proxy send this timestamp per `conf.Config.Timesync.Interval` milliseconds.
// However, timestamp of search request (searchTimestamp) is precision time.
// So the ServiceTimeSync is always less than searchTimestamp.
// Here, we manually make searchTimestamp's logic time minus `conf.Config.Timesync.Interval` milliseconds.
// Which means `searchTimestamp.logicTime = searchTimestamp.logicTime - conf.Config.Timesync.Interval`.
var logicTimestamp = searchTimestamp << 46 >> 46
searchTimestamp = (searchTimestamp>>18-uint64(conf.Config.Timesync.Interval+600))<<18 + logicTimestamp
var vector = msg.Records
// We now only the first Json is valid.
var queryJson = msg.Json[0]
// 1. Timestamp check
// TODO: return or wait? Or adding graceful time
if searchTimestamp > node.queryNodeTimeSync.ServiceTimeSync {
fmt.Println("Invalid query time, timestamp = ", searchTimestamp>>18, ", SearchTimeSync = ", node.queryNodeTimeSync.ServiceTimeSync>>18)
return msgPb.Status{ErrorCode: 1}
}
// 2. Get query information from query json
query := node.QueryJson2Info(&queryJson)
// 2d slice for receiving multiple queries's results
var resultsTmp = make([][]SearchResultTmp, query.NumQueries)
for i := 0; i < int(query.NumQueries); i++ {
resultsTmp[i] = make([]SearchResultTmp, 0)
}
// 3. Do search in all segments
for _, segment := range node.SegmentsMap {
if segment.GetRowCount() <= 0 {
// Skip empty segment
continue
}
//fmt.Println("Search in segment:", segment.SegmentId, ",segment rows:", segment.GetRowCount())
var res, err = segment.SegmentSearch(query, searchTimestamp, vector)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
for i := 0; i < int(query.NumQueries); i++ {
for j := i * query.TopK; j < (i+1)*query.TopK; j++ {
resultsTmp[i] = append(resultsTmp[i], SearchResultTmp{
ResultId: res.ResultIds[j],
ResultDistance: res.ResultDistances[j],
})
}
}
}
// 4. Reduce results
for _, rTmp := range resultsTmp {
sort.Slice(rTmp, func(i, j int) bool {
return rTmp[i].ResultDistance < rTmp[j].ResultDistance
})
}
for _, rTmp := range resultsTmp {
if len(rTmp) > query.TopK {
rTmp = rTmp[:query.TopK]
}
}
var entities = msgPb.Entities{
Ids: make([]int64, 0),
}
var results = msgPb.QueryResult{
Status: &msgPb.Status{
ErrorCode: 0,
},
Entities: &entities,
Distances: make([]float32, 0),
QueryId: uint64(msg.Uid),
ProxyId: clientId,
}
for _, rTmp := range resultsTmp {
for _, res := range rTmp {
results.Entities.Ids = append(results.Entities.Ids, res.ResultId)
results.Distances = append(results.Distances, res.ResultDistance)
results.Scores = append(results.Distances, float32(0))
}
}
// Send numQueries to RowNum.
results.RowNum = query.NumQueries
// 5. publish result to pulsar
//fmt.Println(results.Entities.Ids)
//fmt.Println(results.Distances)
node.PublishSearchResult(&results)
}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}

View File

@ -0,0 +1,138 @@
package reader
import (
"context"
"encoding/binary"
"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"
"math"
"strconv"
"sync"
"testing"
)
// NOTE: start pulsar before test
func TestSearch_Search(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, _ := context.WithCancel(context.Background())
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_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()
var queryRawData = make([]float32, 0)
for i := 0; i < DIM; i++ {
queryRawData = append(queryRawData, float32(i))
}
var queryJson = "{\"field_name\":\"fakevec\",\"num_queries\":1,\"topK\":10}"
searchMsg1 := msgPb.SearchMsg{
CollectionName: "collection0",
Records: &msgPb.VectorRowRecord{
FloatData: queryRawData,
},
PartitionTag: []string{"partition0"},
Uid: int64(0),
Timestamp: uint64(0),
ClientId: int64(0),
ExtraParams: nil,
Json: []string{queryJson},
}
searchMessages := []*msgPb.SearchMsg{&searchMsg1}
node.queryNodeTimeSync.UpdateSearchTimeSync(timeRange)
assert.Equal(t, node.queryNodeTimeSync.ServiceTimeSync, timeRange.timestampMax)
status := node.Search(searchMessages)
assert.Equal(t, status.ErrorCode, msgPb.ErrorCode_SUCCESS)
node.Close()
}

View File

@ -13,10 +13,9 @@ package reader
*/ */
import "C" import "C"
import ( import (
"fmt" "github.com/stretchr/testify/assert"
"github.com/zilliztech/milvus-distributed/internal/errors" "github.com/zilliztech/milvus-distributed/internal/errors"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message" msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/stretchr/testify/assert"
"strconv" "strconv"
"unsafe" "unsafe"
) )
@ -24,10 +23,10 @@ import (
const SegmentLifetime = 20000 const SegmentLifetime = 20000
const ( const (
SegmentOpened = 0 //SegmentOpened = 0
SegmentClosed = 1 //SegmentClosed = 1
SegmentIndexing = 2 //SegmentIndexing = 2
SegmentIndexed = 3 //SegmentIndexed = 3
) )
type Segment struct { type Segment struct {
@ -38,23 +37,23 @@ type Segment struct {
SegmentStatus int SegmentStatus int
} }
func (s *Segment) GetStatus() int { //func (s *Segment) GetStatus() int {
/* // /*
bool // bool
IsOpened(CSegmentBase c_segment); // IsOpened(CSegmentBase c_segment);
*/ // */
var isOpened = C.IsOpened(s.SegmentPtr) // var isOpened = C.IsOpened(s.SegmentPtr)
if isOpened { // if isOpened {
return SegmentOpened // return SegmentOpened
} else { // } else {
return SegmentClosed // return SegmentClosed
} // }
} //}
func (s *Segment) GetRowCount() int64 { func (s *Segment) GetRowCount() int64 {
/* /*
long int long int
GetRowCount(CSegmentBase c_segment); GetRowCount(CSegmentBase c_segment);
*/ */
var rowCount = C.GetRowCount(s.SegmentPtr) var rowCount = C.GetRowCount(s.SegmentPtr)
return int64(rowCount) return int64(rowCount)
@ -62,45 +61,45 @@ func (s *Segment) GetRowCount() int64 {
func (s *Segment) GetDeletedCount() int64 { func (s *Segment) GetDeletedCount() int64 {
/* /*
long int long int
GetDeletedCount(CSegmentBase c_segment); GetDeletedCount(CSegmentBase c_segment);
*/ */
var deletedCount = C.GetDeletedCount(s.SegmentPtr) var deletedCount = C.GetDeletedCount(s.SegmentPtr)
return int64(deletedCount) return int64(deletedCount)
} }
func (s *Segment) CloseSegment(collection* Collection) error { //func (s *Segment) CloseSegment(collection* Collection) error {
/* // /*
int // int
Close(CSegmentBase c_segment); // Close(CSegmentBase c_segment);
*/ // */
fmt.Println("Closing segment :", s.SegmentId) // fmt.Println("Closing segment :", s.SegmentId)
//
var status = C.Close(s.SegmentPtr) // var status = C.Close(s.SegmentPtr)
s.SegmentStatus = SegmentClosed // s.SegmentStatus = SegmentClosed
//
if status != 0 { // if status != 0 {
return errors.New("Close segment failed, error code = " + strconv.Itoa(int(status))) // return errors.New("Close segment failed, error code = " + strconv.Itoa(int(status)))
} // }
//
// Build index after closing segment // // Build index after closing segment
//s.SegmentStatus = SegmentIndexing // //s.SegmentStatus = SegmentIndexing
//fmt.Println("Building index...") // //fmt.Println("Building index...")
//s.buildIndex(collection) // //s.BuildIndex(collection)
//
// TODO: remove redundant segment indexed status // // TODO: remove redundant segment indexed status
// Change segment status to indexed // // Change segment status to indexed
//s.SegmentStatus = SegmentIndexed // //s.SegmentStatus = SegmentIndexed
//fmt.Println("Segment closed and indexed") // //fmt.Println("Segment closed and indexed")
//
fmt.Println("Segment closed") // fmt.Println("Segment closed")
return nil // return nil
} //}
func (s *Segment) GetMemSize() uint64 { func (s *Segment) GetMemSize() uint64 {
/* /*
long int long int
GetMemoryUsageInBytes(CSegmentBase c_segment); GetMemoryUsageInBytes(CSegmentBase c_segment);
*/ */
var memoryUsageInBytes = C.GetMemoryUsageInBytes(s.SegmentPtr) var memoryUsageInBytes = C.GetMemoryUsageInBytes(s.SegmentPtr)
@ -110,8 +109,8 @@ func (s *Segment) GetMemSize() uint64 {
//////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////
func (s *Segment) SegmentPreInsert(numOfRecords int) int64 { func (s *Segment) SegmentPreInsert(numOfRecords int) int64 {
/* /*
long int long int
PreInsert(CSegmentBase c_segment, long int size); PreInsert(CSegmentBase c_segment, long int size);
*/ */
var offset = C.PreInsert(s.SegmentPtr, C.long(int64(numOfRecords))) var offset = C.PreInsert(s.SegmentPtr, C.long(int64(numOfRecords)))
@ -120,8 +119,8 @@ func (s *Segment) SegmentPreInsert(numOfRecords int) int64 {
func (s *Segment) SegmentPreDelete(numOfRecords int) int64 { func (s *Segment) SegmentPreDelete(numOfRecords int) int64 {
/* /*
long int long int
PreDelete(CSegmentBase c_segment, long int size); PreDelete(CSegmentBase c_segment, long int size);
*/ */
var offset = C.PreDelete(s.SegmentPtr, C.long(int64(numOfRecords))) var offset = C.PreDelete(s.SegmentPtr, C.long(int64(numOfRecords)))
@ -130,15 +129,15 @@ func (s *Segment) SegmentPreDelete(numOfRecords int) int64 {
func (s *Segment) SegmentInsert(offset int64, entityIDs *[]int64, timestamps *[]uint64, records *[][]byte) error { func (s *Segment) SegmentInsert(offset int64, entityIDs *[]int64, timestamps *[]uint64, records *[][]byte) error {
/* /*
int int
Insert(CSegmentBase c_segment, Insert(CSegmentBase c_segment,
long int reserved_offset, long int reserved_offset,
signed long int size, signed long int size,
const long* primary_keys, const long* primary_keys,
const unsigned long* timestamps, const unsigned long* timestamps,
void* raw_data, void* raw_data,
int sizeof_per_row, int sizeof_per_row,
signed long int count); signed long int count);
*/ */
// Blobs to one big blob // Blobs to one big blob
var numOfRow = len(*entityIDs) var numOfRow = len(*entityIDs)
@ -146,7 +145,7 @@ func (s *Segment) SegmentInsert(offset int64, entityIDs *[]int64, timestamps *[]
assert.Equal(nil, numOfRow, len(*records)) assert.Equal(nil, numOfRow, len(*records))
var rawData = make([]byte, numOfRow * sizeofPerRow) var rawData = make([]byte, numOfRow*sizeofPerRow)
var copyOffset = 0 var copyOffset = 0
for i := 0; i < len(*records); i++ { for i := 0; i < len(*records); i++ {
copy(rawData[copyOffset:], (*records)[i]) copy(rawData[copyOffset:], (*records)[i])
@ -178,12 +177,12 @@ func (s *Segment) SegmentInsert(offset int64, entityIDs *[]int64, timestamps *[]
func (s *Segment) SegmentDelete(offset int64, entityIDs *[]int64, timestamps *[]uint64) error { func (s *Segment) SegmentDelete(offset int64, entityIDs *[]int64, timestamps *[]uint64) error {
/* /*
int int
Delete(CSegmentBase c_segment, Delete(CSegmentBase c_segment,
long int reserved_offset, long int reserved_offset,
long size, long size,
const long* primary_keys, const long* primary_keys,
const unsigned long* timestamps); const unsigned long* timestamps);
*/ */
var cOffset = C.long(offset) var cOffset = C.long(offset)
var cSize = C.long(len(*entityIDs)) var cSize = C.long(len(*entityIDs))
@ -201,14 +200,14 @@ func (s *Segment) SegmentDelete(offset int64, entityIDs *[]int64, timestamps *[]
func (s *Segment) SegmentSearch(query *QueryInfo, timestamp uint64, vectorRecord *msgPb.VectorRowRecord) (*SearchResult, error) { func (s *Segment) SegmentSearch(query *QueryInfo, timestamp uint64, vectorRecord *msgPb.VectorRowRecord) (*SearchResult, error) {
/* /*
int int
Search(CSegmentBase c_segment, Search(CSegmentBase c_segment,
CQueryInfo c_query_info, CQueryInfo c_query_info,
unsigned long timestamp, unsigned long timestamp,
float* query_raw_data, float* query_raw_data,
int num_of_query_raw_data, int num_of_query_raw_data,
long int* result_ids, long int* result_ids,
float* result_distances); float* result_distances);
*/ */
//type CQueryInfo C.CQueryInfo //type CQueryInfo C.CQueryInfo
@ -218,8 +217,8 @@ func (s *Segment) SegmentSearch(query *QueryInfo, timestamp uint64, vectorRecord
field_name: C.CString(query.FieldName), field_name: C.CString(query.FieldName),
} }
resultIds := make([]int64, int64(query.TopK) * query.NumQueries) resultIds := make([]int64, int64(query.TopK)*query.NumQueries)
resultDistances := make([]float32, int64(query.TopK) * query.NumQueries) resultDistances := make([]float32, int64(query.TopK)*query.NumQueries)
var cTimestamp = C.ulong(timestamp) var cTimestamp = C.ulong(timestamp)
var cResultIds = (*C.long)(&resultIds[0]) var cResultIds = (*C.long)(&resultIds[0])

View File

@ -9,61 +9,45 @@ import (
"time" "time"
) )
func (node *QueryNode) SegmentsManagement() { //func (node *QueryNode) SegmentsManagement() {
//node.queryNodeTimeSync.UpdateTSOTimeSync() // //node.queryNodeTimeSync.UpdateTSOTimeSync()
//var timeNow = node.queryNodeTimeSync.TSOTimeSync // //var timeNow = node.queryNodeTimeSync.TSOTimeSync
//
// timeNow := node.messageClient.GetTimeNow() >> 18
//
// for _, collection := range node.Collections {
// for _, partition := range collection.Partitions {
// for _, segment := range partition.Segments {
// if segment.SegmentStatus != SegmentOpened {
// continue
// }
//
// // fmt.Println("timeNow = ", timeNow, "SegmentCloseTime = ", segment.SegmentCloseTime)
// if timeNow >= segment.SegmentCloseTime {
// go segment.CloseSegment(collection)
// }
// }
// }
// }
//}
timeNow := node.messageClient.GetTimeNow() >> 18 //func (node *QueryNode) SegmentManagementService() {
// sleepMillisecondTime := 1000
for _, collection := range node.Collections { // fmt.Println("do segments management in ", strconv.Itoa(sleepMillisecondTime), "ms")
for _, partition := range collection.Partitions { // for {
for _, segment := range partition.Segments { // select {
if segment.SegmentStatus != SegmentOpened { // case <-node.ctx.Done():
continue // return
} // default:
// time.Sleep(time.Duration(sleepMillisecondTime) * time.Millisecond)
// fmt.Println("timeNow = ", timeNow, "SegmentCloseTime = ", segment.SegmentCloseTime) // node.SegmentsManagement()
if timeNow >= segment.SegmentCloseTime { // }
go segment.CloseSegment(collection) // }
} //}
}
}
}
}
func (node *QueryNode) SegmentManagementService() {
sleepMillisecondTime := 1000
fmt.Println("do segments management in ", strconv.Itoa(sleepMillisecondTime), "ms")
for {
select {
case <-node.ctx.Done():
return
default:
time.Sleep(time.Duration(sleepMillisecondTime) * time.Millisecond)
node.SegmentsManagement()
}
}
}
func (node *QueryNode) SegmentStatistic(sleepMillisecondTime int) { func (node *QueryNode) SegmentStatistic(sleepMillisecondTime int) {
var statisticData = make([]masterPb.SegmentStat, 0) var statisticData = make([]masterPb.SegmentStat, 0)
//for _, collection := range node.Collections {
// for _, partition := range collection.Partitions {
// for _, openedSegment := range partition.OpenedSegments {
// currentMemSize := openedSegment.GetMemSize()
// memIncreaseRate := float32((int64(currentMemSize))-(int64(openedSegment.LastMemSize))) / (float32(sleepMillisecondTime) / 1000)
// stat := masterPb.SegmentStat{
// // TODO: set master pb's segment id type from uint64 to int64
// SegmentId: uint64(openedSegment.SegmentId),
// MemorySize: currentMemSize,
// MemoryRate: memIncreaseRate,
// }
// statisticData = append(statisticData, stat)
// }
// }
//}
for segmentID, segment := range node.SegmentsMap { for segmentID, segment := range node.SegmentsMap {
currentMemSize := segment.GetMemSize() currentMemSize := segment.GetMemSize()
memIncreaseRate := float32((int64(currentMemSize))-(int64(segment.LastMemSize))) / (float32(sleepMillisecondTime) / 1000) memIncreaseRate := float32((int64(currentMemSize))-(int64(segment.LastMemSize))) / (float32(sleepMillisecondTime) / 1000)

View File

@ -9,39 +9,40 @@ import (
"time" "time"
) )
func TestSegmentManagement_SegmentsManagement(t *testing.T) { //func TestSegmentManagement_SegmentsManagement(t *testing.T) {
// Construct node, collection, partition and segment // // Construct node, collection, partition and segment
ctx := context.Background() // ctx := context.Background()
node := NewQueryNode(ctx, 0, 0) // node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "") // var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0") // var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) // var segment = partition.NewSegment(0)
node.SegmentsMap[0] = segment // node.SegmentsMap[0] = segment
//
// node.SegmentsManagement()
//
// node.Close()
//}
node.SegmentsManagement() //func TestSegmentManagement_SegmentService(t *testing.T) {
// d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
node.Close() // ctx, _ := context.WithDeadline(context.Background(), d)
} //
// // Construct node, collection, partition and segment
func TestSegmentManagement_SegmentService(t *testing.T) { // node := NewQueryNode(ctx, 0, 0)
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond) // var collection = node.NewCollection(0, "collection0", "")
ctx, _ := context.WithDeadline(context.Background(), d) // var partition = collection.NewPartition("partition0")
// var segment = partition.NewSegment(0)
// Construct node, collection, partition and segment // node.SegmentsMap[0] = segment
node := NewQueryNode(ctx, 0, 0) //
var collection = node.NewCollection(0, "collection0", "") // node.SegmentManagementService()
var partition = collection.NewPartition("partition0") //
var segment = partition.NewSegment(0) // node.Close()
node.SegmentsMap[0] = segment //}
node.SegmentManagementService()
node.Close()
}
// NOTE: start pulsar before test // NOTE: start pulsar before test
func TestSegmentManagement_SegmentStatistic(t *testing.T) { func TestSegmentManagement_SegmentStatistic(t *testing.T) {
conf.LoadConfig("config.yaml") conf.LoadConfig("config.yaml")
ctx, _ := context.WithCancel(context.Background()) ctx, _ := context.WithCancel(context.Background())
mc := msgclient.ReaderMessageClient{} mc := msgclient.ReaderMessageClient{}
@ -49,9 +50,10 @@ func TestSegmentManagement_SegmentStatistic(t *testing.T) {
pulsarAddr += conf.Config.Pulsar.Address pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":" pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage() mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc) node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment // Construct node, collection, partition and segment
@ -68,6 +70,7 @@ func TestSegmentManagement_SegmentStatistic(t *testing.T) {
// NOTE: start pulsar before test // NOTE: start pulsar before test
func TestSegmentManagement_SegmentStatisticService(t *testing.T) { func TestSegmentManagement_SegmentStatisticService(t *testing.T) {
conf.LoadConfig("config.yaml") conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond) d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d) ctx, _ := context.WithDeadline(context.Background(), d)
@ -76,9 +79,10 @@ func TestSegmentManagement_SegmentStatisticService(t *testing.T) {
pulsarAddr += conf.Config.Pulsar.Address pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":" pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage() mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc) node := CreateQueryNode(ctx, 0, 0, &mc)
// Construct node, collection, partition and segment // Construct node, collection, partition and segment

View File

@ -7,8 +7,8 @@ import (
"math" "math"
"testing" "testing"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
) )
func TestSegment_ConstructorAndDestructor(t *testing.T) { func TestSegment_ConstructorAndDestructor(t *testing.T) {
@ -19,11 +19,21 @@ func TestSegment_ConstructorAndDestructor(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Destruct collection, partition and segment // 2. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -35,6 +45,13 @@ func TestSegment_SegmentInsert(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps // 2. Create ids and timestamps
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
timestamps := []uint64{0, 0, 0} timestamps := []uint64{0, 0, 0}
@ -68,10 +85,13 @@ func TestSegment_SegmentInsert(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
// 6. Destruct collection, partition and segment // 6. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -83,6 +103,13 @@ func TestSegment_SegmentDelete(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps // 2. Create ids and timestamps
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
timestamps := []uint64{0, 0, 0} timestamps := []uint64{0, 0, 0}
@ -96,10 +123,13 @@ func TestSegment_SegmentDelete(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
// 5. Destruct collection, partition and segment // 5. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -111,6 +141,13 @@ func TestSegment_SegmentSearch(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps // 2. Create ids and timestamps
ids := make([]int64, 0) ids := make([]int64, 0)
timestamps := make([]uint64, 0) timestamps := make([]uint64, 0)
@ -160,10 +197,13 @@ func TestSegment_SegmentSearch(t *testing.T) {
fmt.Println(searchRes) fmt.Println(searchRes)
// 7. Destruct collection, partition and segment // 7. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -175,15 +215,25 @@ func TestSegment_SegmentPreInsert(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Do PreInsert // 2. Do PreInsert
var offset = segment.SegmentPreInsert(10) var offset = segment.SegmentPreInsert(10)
assert.GreaterOrEqual(t, offset, int64(0)) assert.GreaterOrEqual(t, offset, int64(0))
// 3. Destruct collection, partition and segment // 3. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -195,59 +245,69 @@ func TestSegment_SegmentPreDelete(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Do PreDelete // 2. Do PreDelete
var offset = segment.SegmentPreDelete(10) var offset = segment.SegmentPreDelete(10)
assert.GreaterOrEqual(t, offset, int64(0)) assert.GreaterOrEqual(t, offset, int64(0))
// 3. Destruct collection, partition and segment // 3. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
// Segment util functions test // Segment util functions test
//////////////////////////////////////////////////////////////////////////// ////////////////////////////////////////////////////////////////////////////
func TestSegment_GetStatus(t *testing.T) { //func TestSegment_GetStatus(t *testing.T) {
ctx := context.Background() // ctx := context.Background()
// 1. Construct node, collection, partition and segment // // 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0) // node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "") // var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0") // var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) // var segment = partition.NewSegment(0)
//
// // 2. Get segment status
// var status = segment.GetStatus()
// assert.Equal(t, status, SegmentOpened)
//
// // 3. Destruct collection, partition and segment
// partition.DeleteSegment(segment)
// collection.DeletePartition(partition)
// node.DeleteCollection(collection)
//
// node.Close()
//}
// 2. Get segment status //func TestSegment_Close(t *testing.T) {
var status = segment.GetStatus() // ctx := context.Background()
assert.Equal(t, status, SegmentOpened) // // 1. Construct node, collection, partition and segment
// node := NewQueryNode(ctx, 0, 0)
// 3. Destruct collection, partition and segment // var collection = node.NewCollection(0, "collection0", "")
partition.DeleteSegment(segment) // var partition = collection.NewPartition("partition0")
collection.DeletePartition(partition) // var segment = partition.NewSegment(0)
node.DeleteCollection(collection) //
// // 2. Close segment
node.Close() // var err = segment.CloseSegment(collection)
} // assert.NoError(t, err)
//
func TestSegment_Close(t *testing.T) { // // 3. Destruct collection, partition and segment
ctx := context.Background() // partition.DeleteSegment(segment)
// 1. Construct node, collection, partition and segment // collection.DeletePartition(partition)
node := NewQueryNode(ctx, 0, 0) // node.DeleteCollection(collection)
var collection = node.NewCollection(0, "collection0", "") //
var partition = collection.NewPartition("partition0") // node.Close()
var segment = partition.NewSegment(0) //}
// 2. Close segment
var err = segment.CloseSegment(collection)
assert.NoError(t, err)
// 3. Destruct collection, partition and segment
partition.DeleteSegment(segment)
collection.DeletePartition(partition)
node.DeleteCollection(collection)
node.Close()
}
func TestSegment_GetRowCount(t *testing.T) { func TestSegment_GetRowCount(t *testing.T) {
ctx := context.Background() ctx := context.Background()
@ -257,6 +317,13 @@ func TestSegment_GetRowCount(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps // 2. Create ids and timestamps
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
timestamps := []uint64{0, 0, 0} timestamps := []uint64{0, 0, 0}
@ -294,10 +361,13 @@ func TestSegment_GetRowCount(t *testing.T) {
assert.Equal(t, rowCount, int64(len(ids))) assert.Equal(t, rowCount, int64(len(ids)))
// 7. Destruct collection, partition and segment // 7. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -309,6 +379,13 @@ func TestSegment_GetDeletedCount(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps // 2. Create ids and timestamps
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
timestamps := []uint64{0, 0, 0} timestamps := []uint64{0, 0, 0}
@ -327,10 +404,13 @@ func TestSegment_GetDeletedCount(t *testing.T) {
assert.Equal(t, deletedCount, int64(0)) assert.Equal(t, deletedCount, int64(0))
// 6. Destruct collection, partition and segment // 6. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -342,6 +422,13 @@ func TestSegment_GetMemSize(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps // 2. Create ids and timestamps
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
timestamps := []uint64{0, 0, 0} timestamps := []uint64{0, 0, 0}
@ -379,10 +466,13 @@ func TestSegment_GetMemSize(t *testing.T) {
assert.Equal(t, memSize, uint64(2785280)) assert.Equal(t, memSize, uint64(2785280))
// 7. Destruct collection, partition and segment // 7. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -402,6 +492,13 @@ func TestSegment_RealSchemaTest(t *testing.T) {
var partition = collection.NewPartition("partition0") var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0) var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
// 2. Create ids and timestamps // 2. Create ids and timestamps
ids := []int64{1, 2, 3} ids := []int64{1, 2, 3}
timestamps := []uint64{0, 0, 0} timestamps := []uint64{0, 0, 0}
@ -435,9 +532,12 @@ func TestSegment_RealSchemaTest(t *testing.T) {
assert.NoError(t, err) assert.NoError(t, err)
// 6. Destruct collection, partition and segment // 6. Destruct collection, partition and segment
partition.DeleteSegment(segment) partition.DeleteSegment(node, segment)
collection.DeletePartition(partition) collection.DeletePartition(node, partition)
node.DeleteCollection(collection) node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }

View File

@ -64,6 +64,16 @@ func (node *QueryNode) GetSegmentBySegmentID(segmentID int64) (*Segment, error)
return targetSegment, nil return targetSegment, nil
} }
func (node *QueryNode) FoundSegmentBySegmentID(segmentID int64) bool {
targetSegment := node.SegmentsMap[segmentID]
if targetSegment == nil {
return false
}
return true
}
func (c *Collection) GetPartitionByName(partitionName string) (partition *Partition) { func (c *Collection) GetPartitionByName(partitionName string) (partition *Partition) {
for _, partition := range c.Partitions { for _, partition := range c.Partitions {
if partition.PartitionName == partitionName { if partition.PartitionName == partitionName {
@ -124,3 +134,21 @@ func (node *QueryNode) WriteQueryLog() {
fmt.Println("write log done") fmt.Println("write log done")
} }
func (node *QueryNode) PrepareBatchMsg() []int {
var msgLen = node.messageClient.PrepareBatchMsg()
return msgLen
}
func (node *QueryNode) QueryJson2Info(queryJson *string) *QueryInfo {
var query QueryInfo
var err = json.Unmarshal([]byte(*queryJson), &query)
if err != nil {
log.Fatal("Unmarshal query json failed")
return nil
}
//fmt.Println(query)
return &query
}

View File

@ -4,6 +4,7 @@ import (
"context" "context"
"github.com/zilliztech/milvus-distributed/internal/conf" "github.com/zilliztech/milvus-distributed/internal/conf"
"github.com/zilliztech/milvus-distributed/internal/msgclient" "github.com/zilliztech/milvus-distributed/internal/msgclient"
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
"strconv" "strconv"
"testing" "testing"
"time" "time"
@ -14,6 +15,7 @@ import (
// NOTE: start pulsar before test // NOTE: start pulsar before test
func TestUtilFunctions_GetKey2Segments(t *testing.T) { func TestUtilFunctions_GetKey2Segments(t *testing.T) {
conf.LoadConfig("config.yaml") conf.LoadConfig("config.yaml")
d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond) d := time.Now().Add(ctxTimeInMillisecond * time.Millisecond)
ctx, _ := context.WithDeadline(context.Background(), d) ctx, _ := context.WithDeadline(context.Background(), d)
@ -22,13 +24,58 @@ func TestUtilFunctions_GetKey2Segments(t *testing.T) {
pulsarAddr += conf.Config.Pulsar.Address pulsarAddr += conf.Config.Pulsar.Address
pulsarAddr += ":" pulsarAddr += ":"
pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10) pulsarAddr += strconv.FormatInt(int64(conf.Config.Pulsar.Port), 10)
mc.InitClient(ctx, pulsarAddr)
mc.InitClient(ctx, pulsarAddr)
mc.ReceiveMessage() mc.ReceiveMessage()
node := CreateQueryNode(ctx, 0, 0, &mc) node := CreateQueryNode(ctx, 0, 0, &mc)
node.messageClient.PrepareKey2SegmentMsg() node.messageClient.PrepareKey2SegmentMsg()
var _, _, _ = node.GetKey2Segments()
const msgLength = 10
for i := 0; i < msgLength; i++ {
key2SegMsg := msgPb.Key2SegMsg{
Uid: int64(i),
Timestamp: uint64(i + 1000),
SegmentId: []int64{int64(i)},
}
node.messageClient.Key2SegMsg = append(node.messageClient.Key2SegMsg, &key2SegMsg)
}
entityIDs, timestamps, segmentIDs := node.GetKey2Segments()
assert.Equal(t, len(*entityIDs), msgLength)
assert.Equal(t, len(*timestamps), msgLength)
assert.Equal(t, len(*segmentIDs), msgLength)
node.Close()
}
func TestUtilFunctions_GetCollectionByID(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
c := node.GetCollectionByID(uint64(0))
assert.Equal(t, c.CollectionName, "collection0")
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close() node.Close()
} }
@ -43,6 +90,7 @@ func TestUtilFunctions_GetCollectionByCollectionName(t *testing.T) {
var c0, err = node.GetCollectionByCollectionName("collection0") var c0, err = node.GetCollectionByCollectionName("collection0")
assert.NoError(t, err) assert.NoError(t, err)
assert.Equal(t, c0.CollectionName, "collection0") assert.Equal(t, c0.CollectionName, "collection0")
c0 = node.GetCollectionByID(0) c0 = node.GetCollectionByID(0)
assert.NotNil(t, c0) assert.NotNil(t, c0)
assert.Equal(t, c0.CollectionID, uint64(0)) assert.Equal(t, c0.CollectionID, uint64(0))
@ -52,6 +100,7 @@ func TestUtilFunctions_GetCollectionByCollectionName(t *testing.T) {
func TestUtilFunctions_GetSegmentBySegmentID(t *testing.T) { func TestUtilFunctions_GetSegmentBySegmentID(t *testing.T) {
ctx := context.Background() ctx := context.Background()
// 1. Construct node, collection, partition and segment // 1. Construct node, collection, partition and segment
node := NewQueryNode(ctx, 0, 0) node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "") var collection = node.NewCollection(0, "collection0", "")
@ -66,3 +115,85 @@ func TestUtilFunctions_GetSegmentBySegmentID(t *testing.T) {
node.Close() node.Close()
} }
func TestUtilFunctions_FoundSegmentBySegmentID(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
node.SegmentsMap[int64(0)] = segment
assert.Equal(t, collection.CollectionName, "collection0")
assert.Equal(t, partition.PartitionName, "partition0")
assert.Equal(t, segment.SegmentId, int64(0))
assert.Equal(t, len(node.SegmentsMap), 1)
b1 := node.FoundSegmentBySegmentID(int64(0))
assert.Equal(t, b1, true)
b2 := node.FoundSegmentBySegmentID(int64(1))
assert.Equal(t, b2, false)
partition.DeleteSegment(node, segment)
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
assert.Equal(t, len(node.Collections), 0)
assert.Equal(t, len(node.SegmentsMap), 0)
node.Close()
}
func TestUtilFunctions_GetPartitionByName(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var collection = node.NewCollection(0, "collection0", "")
var partition = collection.NewPartition("partition0")
var p = collection.GetPartitionByName("partition0")
assert.Equal(t, p.PartitionName, "partition0")
collection.DeletePartition(node, partition)
node.DeleteCollection(collection)
node.Close()
}
// NOTE: start pulsar before test
func TestUtilFunctions_PrepareBatchMsg(t *testing.T) {
conf.LoadConfig("config.yaml")
ctx, _ := context.WithCancel(context.Background())
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)
node.PrepareBatchMsg()
node.Close()
}
func TestUtilFunctions_QueryJson2Info(t *testing.T) {
ctx := context.Background()
node := NewQueryNode(ctx, 0, 0)
var queryJson = "{\"field_name\":\"age\",\"num_queries\":1,\"topK\":10}"
info := node.QueryJson2Info(&queryJson)
assert.Equal(t, info.FieldName, "age")
assert.Equal(t, info.NumQueries, int64(1))
assert.Equal(t, info.TopK, 10)
node.Close()
}