Add message client and refactor segment go, construct raw data for segment insertion

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
pull/4973/head^2
bigsheeper 2020-09-08 10:39:09 +08:00 committed by yefu.chen
parent c9c7c78340
commit 41d638abb5
5 changed files with 116 additions and 204 deletions

View File

@ -20,7 +20,7 @@ type Partition struct {
ClosedSegments []*Segment
}
func (p *Partition) NewSegment(segmentId uint64) *Segment {
func (p *Partition) NewSegment(segmentId int64) *Segment {
segmentPtr := C.NewSegment(p.PartitionPtr, C.ulong(segmentId))
var newSegment = &Segment{SegmentPtr: segmentPtr, SegmentId: segmentId}

View File

@ -16,8 +16,9 @@ import "C"
import (
"errors"
"fmt"
messageClient "github.com/czs007/suvlim/pkg/message"
schema "github.com/czs007/suvlim/pkg/message"
msgPb "github.com/czs007/suvlim/pkg/message"
"github.com/czs007/suvlim/reader/message_client"
"sort"
"strconv"
"sync"
"time"
@ -35,8 +36,8 @@ type DeleteRecords struct {
}
type QueryNodeDataBuffer struct {
InsertDeleteBuffer []*schema.InsertOrDeleteMsg
SearchBuffer []*schema.SearchMsg
InsertDeleteBuffer []*msgPb.InsertOrDeleteMsg
SearchBuffer []*msgPb.SearchMsg
validInsertDeleteBuffer []bool
validSearchBuffer []bool
}
@ -45,14 +46,14 @@ type QueryNode struct {
QueryNodeId uint64
Collections []*Collection
SegmentsMap map[int64]*Segment
messageClient client_go.MessageClient
messageClient message_client.MessageClient
queryNodeTimeSync *QueryNodeTime
deleteRecordsMap map[TimeRange]DeleteRecords
buffer QueryNodeDataBuffer
}
func NewQueryNode(queryNodeId uint64, timeSync uint64) *QueryNode {
mc := messageClient.MessageClient{}
mc := message_client.MessageClient{}
queryNodeTimeSync := &QueryNodeTime{
ReadTimeSyncMin: timeSync,
@ -70,7 +71,7 @@ func NewQueryNode(queryNodeId uint64, timeSync uint64) *QueryNode {
SegmentsMap: segmentsMap,
messageClient: mc,
queryNodeTimeSync: queryNodeTimeSync,
deleteRecordsMap: make(map[int64]DeleteRecords),
deleteRecordsMap: make(map[TimeRange]DeleteRecords),
}
}
@ -97,7 +98,7 @@ func (node *QueryNode) DeleteCollection(collection *Collection) {
func (node *QueryNode) doQueryNode(wg *sync.WaitGroup) {
wg.Add(3)
// Do insert and delete messages sort, do insert
go node.InsertAndDelete(node.messageClient.InsertMsg, wg)
go node.InsertAndDelete(node.messageClient.InsertOrDeleteMsg, wg)
// Do delete messages sort
go node.searchDeleteInMap()
// Do delete
@ -108,18 +109,18 @@ func (node *QueryNode) doQueryNode(wg *sync.WaitGroup) {
}
func (node *QueryNode) PrepareBatchMsg() {
node.messageClient.PrepareBatchMsg(client_go.JobType(0))
node.messageClient.PrepareBatchMsg()
}
func (node *QueryNode) StartMessageClient() {
topics := []string{"insert", "delete"}
// TODO: add consumerMsgSchema
node.messageClient.InitClient("pulsar://localhost:6650", topics, "")
node.messageClient.InitClient("pulsar://localhost:6650")
go node.messageClient.ReceiveMessage()
}
func (node *QueryNode) GetSegmentByEntityId() ([]int64, []uint64, []int64) {
// Function `GetSegmentByEntityId` should return entityIDs, timestamps and segmentIDs
func (node *QueryNode) GetKey2Segments() ([]int64, []uint64, []int64) {
// TODO: get id2segment info from pulsar
return nil, nil, nil
}
@ -150,6 +151,16 @@ func (node *QueryNode) GetTargetSegment(collectionName *string, partitionTag *st
return nil, errors.New("cannot found target segment")
}
func (node *QueryNode) GetCollectionByCollectionName(collectionName string) (*Collection, error) {
for _, collection := range node.Collections {
if collection.CollectionName == collectionName {
return collection, nil
}
}
return nil, errors.New("Cannot found collection: " + collectionName)
}
func (node *QueryNode) GetSegmentBySegmentID(segmentID int64) (*Segment, error) {
targetSegment := node.SegmentsMap[segmentID]
@ -180,10 +191,12 @@ func (node *QueryNode) SegmentsManagement() {
// TODO: check segment status
if timeNow >= oldSegment.SegmentCloseTime {
// start new segment and add it into partition.OpenedSegments
// TODO: add atomic segment id
var newSegment = partition.NewSegment(0)
// TODO: get segmentID from master
var segmentID int64 = 0
var newSegment = partition.NewSegment(segmentID)
newSegment.SegmentCloseTime = timeNow + SegmentLifetime
partition.OpenedSegments = append(partition.OpenedSegments, newSegment)
node.SegmentsMap[segmentID] = newSegment
// close old segment and move it into partition.ClosedSegments
// TODO: check status
@ -205,7 +218,7 @@ func (node *QueryNode) SegmentService() {
///////////////////////////////////////////////////////////////////////////////////////////////////
// TODO: receive delete messages individually
func (node *QueryNode) InsertAndDelete(insertDeleteMessages []*schema.InsertOrDeleteMsg, wg *sync.WaitGroup) schema.Status {
func (node *QueryNode) InsertAndDelete(insertDeleteMessages []*msgPb.InsertOrDeleteMsg, wg *sync.WaitGroup) msgPb.Status {
node.queryNodeTimeSync.UpdateReadTimeSync()
var tMin = node.queryNodeTimeSync.ReadTimeSyncMin
@ -222,11 +235,11 @@ func (node *QueryNode) InsertAndDelete(insertDeleteMessages []*schema.InsertOrDe
// Set valid bitmap to false.
for i, msg := range node.buffer.InsertDeleteBuffer {
if msg.Timestamp <= tMax {
if msg.Op == schema.OpType_INSERT {
if msg.Op == msgPb.OpType_INSERT {
insertIDs[msg.SegmentId] = append(insertIDs[msg.SegmentId], msg.Uid)
insertTimestamps[msg.SegmentId] = append(insertTimestamps[msg.SegmentId], msg.Timestamp)
insertRecords[msg.SegmentId] = append(insertRecords[msg.SegmentId], msg.RowsData.Blob)
} else if msg.Op == schema.OpType_DELETE {
} else if msg.Op == msgPb.OpType_DELETE {
var r = DeleteRecord {
entityID: msg.Uid,
timestamp: msg.Timestamp,
@ -252,11 +265,11 @@ func (node *QueryNode) InsertAndDelete(insertDeleteMessages []*schema.InsertOrDe
// Set valid bitmap to true.
for _, msg := range insertDeleteMessages {
if msg.Timestamp <= tMax {
if msg.Op == schema.OpType_INSERT {
if msg.Op == msgPb.OpType_INSERT {
insertIDs[msg.SegmentId] = append(insertIDs[msg.SegmentId], msg.Uid)
insertTimestamps[msg.SegmentId] = append(insertTimestamps[msg.SegmentId], msg.Timestamp)
insertRecords[msg.SegmentId] = append(insertRecords[msg.SegmentId], msg.RowsData.Blob)
} else if msg.Op == schema.OpType_DELETE {
} else if msg.Op == msgPb.OpType_DELETE {
var r = DeleteRecord {
entityID: msg.Uid,
timestamp: msg.Timestamp,
@ -276,14 +289,14 @@ func (node *QueryNode) InsertAndDelete(insertDeleteMessages []*schema.InsertOrDe
var targetSegment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return schema.Status{ErrorCode: 1}
return msgPb.Status{ErrorCode: 1}
}
ids := insertIDs[segmentID]
timestamps := insertTimestamps[segmentID]
err = targetSegment.SegmentInsert(&ids, &timestamps, &records, tMin, tMax)
if err != nil {
fmt.Println(err.Error())
return schema.Status{ErrorCode: 1}
return msgPb.Status{ErrorCode: 1}
}
}
@ -291,61 +304,8 @@ func (node *QueryNode) InsertAndDelete(insertDeleteMessages []*schema.InsertOrDe
return publishResult(nil, clientId)
}
//func (node *QueryNode) Insert(insertMessages []*schema.InsertMsg, wg *sync.WaitGroup) schema.Status {
// var timeNow = node.GetTSOTime()
// var collectionName = insertMessages[0].CollectionName
// var partitionTag = insertMessages[0].PartitionTag
// var clientId = insertMessages[0].ClientId
//
// // TODO: prevent Memory copy
// var entityIds []uint64
// var timestamps []uint64
// var vectorRecords [][]*schema.FieldValue
//
// for i, msg := range node.buffer.InsertBuffer {
// if msg.Timestamp <= timeNow {
// entityIds = append(entityIds, msg.EntityId)
// timestamps = append(timestamps, msg.Timestamp)
// vectorRecords = append(vectorRecords, msg.Fields)
// node.buffer.validInsertBuffer[i] = false
// }
// }
//
// for i, isValid := range node.buffer.validInsertBuffer {
// if !isValid {
// copy(node.buffer.InsertBuffer[i:], node.buffer.InsertBuffer[i+1:]) // Shift a[i+1:] left one index.
// node.buffer.InsertBuffer[len(node.buffer.InsertBuffer)-1] = nil // Erase last element (write zero value).
// node.buffer.InsertBuffer = node.buffer.InsertBuffer[:len(node.buffer.InsertBuffer)-1] // Truncate slice.
// }
// }
//
// for _, msg := range insertMessages {
// if msg.Timestamp <= timeNow {
// entityIds = append(entityIds, msg.EntityId)
// timestamps = append(timestamps, msg.Timestamp)
// vectorRecords = append(vectorRecords, msg.Fields)
// } else {
// node.buffer.InsertBuffer = append(node.buffer.InsertBuffer, msg)
// node.buffer.validInsertBuffer = append(node.buffer.validInsertBuffer, true)
// }
// }
//
// var targetSegment, err = node.GetTargetSegment(&collectionName, &partitionTag)
// if err != nil {
// // TODO: throw runtime error
// fmt.Println(err.Error())
// return schema.Status{}
// }
//
// // TODO: check error
// var result, _ = SegmentInsert(targetSegment, &entityIds, &timestamps, vectorRecords)
//
// wg.Done()
// return publishResult(&result, clientId)
//}
func (node *QueryNode) searchDeleteInMap() {
var ids, timestamps, segmentIDs = node.GetSegmentByEntityId()
var ids, timestamps, segmentIDs = node.GetKey2Segments()
for i := 0; i <= len(ids); i++ {
id := ids[i]
@ -364,12 +324,12 @@ func (node *QueryNode) searchDeleteInMap() {
}
}
func (node *QueryNode) Delete() schema.Status {
func (node *QueryNode) Delete() msgPb.Status {
type DeleteData struct {
ids *[]int64
timestamp *[]uint64
}
for _, records := range node.deleteRecordsMap {
for timeRange, records := range node.deleteRecordsMap {
// TODO: multi-thread delete
if <- records.count == 0 {
// 1. Sort delete records by segment id
@ -383,48 +343,83 @@ func (node *QueryNode) Delete() schema.Status {
var segment, err = node.GetSegmentBySegmentID(segmentID)
if err != nil {
fmt.Println(err.Error())
return schema.Status{ErrorCode: 1}
return msgPb.Status{ErrorCode: 1}
}
err = segment.SegmentDelete(deleteData.ids, deleteData.timestamp)
err = segment.SegmentDelete(deleteData.ids, deleteData.timestamp, timeRange.timestampMin, timeRange.timestampMax)
if err != nil {
fmt.Println(err.Error())
return schema.Status{ErrorCode: 1}
return msgPb.Status{ErrorCode: 1}
}
}
}
}
return schema.Status{ErrorCode: 0}
return msgPb.Status{ErrorCode: 0}
}
func (node *QueryNode) Search(searchMessages []*schema.SearchMsg, wg *sync.WaitGroup) schema.Status {
func (node *QueryNode) Search(searchMessages []*msgPb.SearchMsg, wg *sync.WaitGroup) msgPb.Status {
var clientId = searchMessages[0].ClientId
type SearchResultTmp struct {
ResultId int64
ResultDistance float32
}
// Traverse all messages in the current messageClient.
// TODO: Do not receive batched search requests
for _, msg := range searchMessages {
var results []*SearchResult
var collectionName = searchMessages[0].CollectionName
var targetCollection, err = node.GetCollectionByCollectionName(collectionName)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
var resultsTmp []SearchResultTmp
// TODO: get top-k's k from queryString
const TopK = 1
// 1. Do search in all segment
// 1. Do search in all segments
var timestamp = msg.Timestamp
var vector = msg.Records
for _, segment := range node.SegmentsMap {
var res, err = segment.SegmentSearch("", timestamp, vector)
if err != nil {
fmt.Println(err.Error())
return schema.Status{ErrorCode: 1}
for _, partition := range targetCollection.Partitions {
for _, openSegment := range partition.OpenedSegments {
var res, err = openSegment.SegmentSearch("", timestamp, vector)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
for i := 0; i <= len(res.ResultIds); i++ {
resultsTmp = append(resultsTmp, SearchResultTmp{ResultId: res.ResultIds[i], ResultDistance: res.ResultDistances[i]})
}
}
for _, closedSegment := range partition.ClosedSegments {
var res, err = closedSegment.SegmentSearch("", timestamp, vector)
if err != nil {
fmt.Println(err.Error())
return msgPb.Status{ErrorCode: 1}
}
for i := 0; i <= len(res.ResultIds); i++ {
resultsTmp = append(resultsTmp, SearchResultTmp{ResultId: res.ResultIds[i], ResultDistance: res.ResultDistances[i]})
}
}
results = append(results, res)
}
// 2. Reduce results
sort.Slice(resultsTmp, func(i, j int) bool {
return resultsTmp[i].ResultDistance < resultsTmp[j].ResultDistance
})
resultsTmp = resultsTmp[:TopK]
var results SearchResult
for _, res := range resultsTmp {
results.ResultIds = append(results.ResultIds, res.ResultId)
results.ResultDistances = append(results.ResultDistances, res.ResultDistance)
}
// 3. publish result to pulsar
publishSearchResult(&results, clientId)
}
wg.Done()
return schema.Status{ErrorCode: 0}
return msgPb.Status{ErrorCode: 0}
}

View File

@ -2,7 +2,7 @@ package reader
import (
"fmt"
msgpb "github.com/czs007/suvlim/pkg/message"
msgPb "github.com/czs007/suvlim/pkg/message"
"strconv"
)
@ -18,24 +18,24 @@ func getResultTopicByClientId(clientId int64) string {
return "result-topic/partition-" + strconv.FormatInt(clientId, 10)
}
func publishResult(ids *ResultEntityIds, clientId int64) msgpb.Status {
func publishResult(ids *ResultEntityIds, clientId int64) msgPb.Status {
// TODO: Pulsar publish
var resultTopic = getResultTopicByClientId(clientId)
fmt.Println(resultTopic)
return msgpb.Status{ErrorCode: msgpb.ErrorCode_SUCCESS}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func publishSearchResult(searchResults *[]*SearchResult, clientId int64) msgpb.Status {
func publishSearchResult(searchResults *SearchResult, clientId int64) msgPb.Status {
// TODO: Pulsar publish
var resultTopic = getResultTopicByClientId(clientId)
fmt.Println(resultTopic)
return msgpb.Status{ErrorCode: msgpb.ErrorCode_SUCCESS}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}
func publicStatistic(statisticTopic string) msgpb.Status {
func publicStatistic(statisticTopic string) msgPb.Status {
// TODO: get statistic info
// getStatisticInfo()
// var info = getStatisticInfo()
// TODO: Pulsar publish
return msgpb.Status{ErrorCode: msgpb.ErrorCode_SUCCESS}
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
}

View File

@ -45,57 +45,6 @@ func (s *Segment) GetStatus() int {
}
}
func (s *Segment) GetSegmentID() int64 {
/*C.GetSegmentId
unsigned long
GetSegmentId(CSegmentBase c_segment);
*/
var segmentID = C.GetSegmentId(s.SegmentPtr)
return int64(segmentID)
}
func (s *Segment) SetSegmentID(segmentID int64) {
/*C.SetSegmentId
void
SetSegmentId(CSegmentBase c_segment, unsigned long segment_id);
*/
C.SetSegmentId(s.SegmentPtr, C.long(segmentID))
}
func (s *Segment) GetMaxTimestamp() uint64 {
/*C.GetTimeEnd
unsigned long
GetTimeEnd(CSegmentBase c_segment);
*/
var maxTimestamp = C.GetTimeEnd(s.SegmentPtr)
return uint64(maxTimestamp)
}
func (s *Segment) SetMaxTimestamp(maxTimestamp uint64) {
/*C.SetTimeEnd
void
SetTimeEnd(CSegmentBase c_segment, unsigned long time_end);
*/
C.SetTimeEnd(s.SegmentPtr, C.ulong(maxTimestamp))
}
func (s *Segment) GetMinTimestamp() uint64 {
/*C.GetTimeBegin
unsigned long
GetTimeBegin(CSegmentBase c_segment);
*/
var minTimestamp = C.GetTimeBegin(s.SegmentPtr)
return uint64(minTimestamp)
}
func (s *Segment) SetMinTimestamp(minTimestamp uint64) {
/*C.SetTimeBegin
void
SetTimeBegin(CSegmentBase c_segment, unsigned long time_begin);
*/
C.SetTimeBegin(s.SegmentPtr, C.ulong(minTimestamp))
}
func (s *Segment) GetRowCount() int64 {
/*C.GetRowCount
long int
@ -136,8 +85,15 @@ func (s *Segment) SegmentInsert(entityIds *[]int64, timestamps *[]uint64, record
const unsigned long* timestamps,
void* raw_data,
int sizeof_per_row,
signed long int count);
signed long int count,
const unsigned long timestamp_min,
const unsigned long timestamp_max);
*/
// Blobs to one big blob
var rowData []byte
for i := 0; i < len(*records); i++ {
copy(rowData, (*records)[i])
}
// TODO: remove hard code schema
// auto schema_tmp = std::make_shared<Schema>();
@ -156,7 +112,7 @@ func (s *Segment) SegmentInsert(entityIds *[]int64, timestamps *[]uint64, record
}
const sizeofPerRow = 4 + DIM * 4
var status = C.Insert(s.SegmentPtr, C.long(N), (*C.ulong)(&(*entityIds)[0]), (*C.ulong)(&(*timestamps)[0]), unsafe.Pointer(&rawData[0]), C.int(sizeofPerRow), C.long(N))
var status = C.Insert(s.SegmentPtr, C.long(N), (*C.ulong)(&(*entityIds)[0]), (*C.ulong)(&(*timestamps)[0]), unsafe.Pointer(&rawData[0]), C.int(sizeofPerRow), C.long(N), C.ulong(timestampMin), C.ulong(timestampMax))
if status != 0 {
return errors.New("Insert failed, error code = " + strconv.Itoa(int(status)))
@ -165,7 +121,7 @@ func (s *Segment) SegmentInsert(entityIds *[]int64, timestamps *[]uint64, record
return nil
}
func (s *Segment) SegmentDelete(entityIds *[]int64, timestamps *[]uint64) error {
func (s *Segment) SegmentDelete(entityIds *[]int64, timestamps *[]uint64, timestampMin uint64, timestampMax uint64) error {
/*C.Delete
int
Delete(CSegmentBase c_segment,
@ -175,7 +131,7 @@ func (s *Segment) SegmentDelete(entityIds *[]int64, timestamps *[]uint64) error
*/
size := len(*entityIds)
var status = C.Delete(s.SegmentPtr, C.long(size), (*C.ulong)(&(*entityIds)[0]), (*C.ulong)(&(*timestamps)[0]))
var status = C.Delete(s.SegmentPtr, C.long(size), (*C.ulong)(&(*entityIds)[0]), (*C.ulong)(&(*timestamps)[0]), C.ulong(timestampMin), C.ulong(timestampMax))
if status != 0 {
return errors.New("Delete failed, error code = " + strconv.Itoa(int(status)))

View File

@ -23,10 +23,10 @@ func TestSegmentInsert(t *testing.T) {
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
ids :=[] uint64{1, 2, 3}
ids :=[] int64{1, 2, 3}
timestamps :=[] uint64 {0, 0, 0}
var _, err = SegmentInsert(segment, &ids, &timestamps, nil)
var err = segment.SegmentInsert(&ids, &timestamps, nil, 0, 0)
assert.NoError(t, err)
partition.DeleteSegment(segment)
@ -40,10 +40,10 @@ func TestSegmentDelete(t *testing.T) {
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
ids :=[] uint64{1, 2, 3}
ids :=[] int64{1, 2, 3}
timestamps :=[] uint64 {0, 0, 0}
var _, err = SegmentDelete(segment, &ids, &timestamps)
var err = segment.SegmentDelete(&ids, &timestamps, 0, 0)
assert.NoError(t, err)
partition.DeleteSegment(segment)
@ -57,13 +57,13 @@ func TestSegmentSearch(t *testing.T) {
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
ids :=[] uint64{1, 2, 3}
ids :=[] int64{1, 2, 3}
timestamps :=[] uint64 {0, 0, 0}
var _, insertErr = SegmentInsert(segment, &ids, &timestamps, nil)
var insertErr = segment.SegmentInsert(&ids, &timestamps, nil, 0, 0)
assert.NoError(t, insertErr)
var searchRes, searchErr = SegmentSearch(segment, "fake query string", &timestamps, nil)
var searchRes, searchErr = segment.SegmentSearch("fake query string", timestamps[0], nil)
assert.NoError(t, searchErr)
fmt.Println(searchRes)
@ -106,10 +106,10 @@ func TestSegment_GetRowCount(t *testing.T) {
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
ids :=[] uint64{1, 2, 3}
ids :=[] int64{1, 2, 3}
timestamps :=[] uint64 {0, 0, 0}
var _, err = SegmentInsert(segment, &ids, &timestamps, nil)
var err = segment.SegmentInsert(&ids, &timestamps, nil, 0, 0)
assert.NoError(t, err)
var rowCount = segment.GetRowCount()
@ -126,10 +126,10 @@ func TestSegment_GetDeletedCount(t *testing.T) {
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
ids :=[] uint64{1, 2, 3}
ids :=[] int64{1, 2, 3}
timestamps :=[] uint64 {0, 0, 0}
var _, err = SegmentDelete(segment, &ids, &timestamps)
var err = segment.SegmentDelete(&ids, &timestamps, 0, 0)
assert.NoError(t, err)
var deletedCount = segment.GetDeletedCount()
@ -140,42 +140,3 @@ func TestSegment_GetDeletedCount(t *testing.T) {
collection.DeletePartition(partition)
node.DeleteCollection(collection)
}
func TestSegment_TimestampGetterAndSetter(t *testing.T) {
node := NewQueryNode(0, 0)
var collection = node.NewCollection("collection0", "fake schema")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
const MinTimestamp = 100
const MaxTimestamp = 200
segment.SetMinTimestamp(MinTimestamp)
var minTimestamp = segment.GetMinTimestamp()
assert.Equal(t, minTimestamp, uint64(MinTimestamp))
segment.SetMaxTimestamp(MaxTimestamp)
var maxTimestamp = segment.GetMaxTimestamp()
assert.Equal(t, maxTimestamp, uint64(MaxTimestamp))
partition.DeleteSegment(segment)
collection.DeletePartition(partition)
node.DeleteCollection(collection)
}
func TestSegment_SegmentIDGetterAndSetter(t *testing.T) {
node := NewQueryNode(0, 0)
var collection = node.NewCollection("collection0", "fake schema")
var partition = collection.NewPartition("partition0")
var segment = partition.NewSegment(0)
const SegmentID = 1
segment.SetSegmentID(SegmentID)
var segmentID = segment.GetSegmentID()
assert.Equal(t, segmentID, uint64(SegmentID))
partition.DeleteSegment(segment)
collection.DeletePartition(partition)
node.DeleteCollection(collection)
}