From 67e6ea805143bcbcc65f005b6b72d2b4b8d37b37 Mon Sep 17 00:00:00 2001
From: sunby <bingyi.sun@zilliz.com>
Date: Fri, 15 Jan 2021 17:09:41 +0800
Subject: [PATCH] Add data service

Signed-off-by: sunby <bingyi.sun@zilliz.com>
---
 internal/dataservice/channel.go           |   47 +
 internal/dataservice/data_service.go      |    5 +
 internal/dataservice/grpc_service.go      |   40 +
 internal/dataservice/meta.go              |  180 +++
 internal/dataservice/param.go             |  370 ++++++
 internal/dataservice/segment_allocator.go |  250 ++++
 internal/dataservice/watcher.go           |  107 ++
 internal/proto/data_service.proto         |   10 +
 internal/proto/datapb/data_service.pb.go  |  224 +++-
 internal/proto/milvus.proto               |   75 +-
 internal/proto/milvuspb/milvus.pb.go      | 1367 ++-------------------
 internal/proxynode/proxy.go               |    6 +-
 internal/proxyservice/interface.go        |   16 -
 internal/proxyservice/proxyservice.go     |   54 -
 internal/querynode/query_node.go          |    6 +-
 internal/timesync/time_sync_producer.go   |  141 +++
 internal/timesync/timesync.go             |  235 ++++
 internal/timesync/timetick_watcher.go     |   11 +
 internal/writenode/write_node.go          |    6 +-
 19 files changed, 1677 insertions(+), 1473 deletions(-)
 create mode 100644 internal/dataservice/channel.go
 create mode 100644 internal/dataservice/data_service.go
 create mode 100644 internal/dataservice/grpc_service.go
 create mode 100644 internal/dataservice/meta.go
 create mode 100644 internal/dataservice/param.go
 create mode 100644 internal/dataservice/segment_allocator.go
 create mode 100644 internal/dataservice/watcher.go
 delete mode 100644 internal/proxyservice/interface.go
 delete mode 100644 internal/proxyservice/proxyservice.go
 create mode 100644 internal/timesync/time_sync_producer.go
 create mode 100644 internal/timesync/timesync.go
 create mode 100644 internal/timesync/timetick_watcher.go

diff --git a/internal/dataservice/channel.go b/internal/dataservice/channel.go
new file mode 100644
index 000000000..a974d8928
--- /dev/null
+++ b/internal/dataservice/channel.go
@@ -0,0 +1,47 @@
+package dataservice
+
+import "fmt"
+
+type (
+	channelRange        []string
+	insertChannelMapper struct {
+		channelRanges []channelRange
+	}
+)
+
+func (cr channelRange) Contains(channelName string) bool {
+	for _, name := range cr {
+		if name == channelName {
+			return true
+		}
+	}
+	return false
+}
+
+func newInsertChannelMapper() *insertChannelMapper {
+	mapper := &insertChannelMapper{channelRanges: make([]channelRange, Params.QueryNodeNum)}
+	channelNames, numOfChannels, numOfQueryNodes := Params.InsertChannelNames, len(Params.InsertChannelNames), Params.QueryNodeNum
+	div, rem := numOfChannels/numOfQueryNodes, numOfChannels%numOfQueryNodes
+	for i, j := 0, 0; i < numOfChannels; j++ {
+		numOfRange := div
+		if j < rem {
+			numOfRange++
+		}
+		cRange := channelRange{}
+		k := i + numOfRange
+		for ; i < k; i++ {
+			cRange = append(cRange, channelNames[i])
+		}
+		mapper.channelRanges = append(mapper.channelRanges, cRange)
+	}
+	return mapper
+}
+
+func (mapper *insertChannelMapper) GetChannelRange(channelName string) (channelRange, error) {
+	for _, cr := range mapper.channelRanges {
+		if cr.Contains(channelName) {
+			return cr, nil
+		}
+	}
+	return nil, fmt.Errorf("channel name %s not found", channelName)
+}
diff --git a/internal/dataservice/data_service.go b/internal/dataservice/data_service.go
new file mode 100644
index 000000000..32ebc5d9b
--- /dev/null
+++ b/internal/dataservice/data_service.go
@@ -0,0 +1,5 @@
+package dataservice
+
+type DataService struct {
+	segAllocator segmentAllocator
+}
diff --git a/internal/dataservice/grpc_service.go b/internal/dataservice/grpc_service.go
new file mode 100644
index 000000000..e4b9184a7
--- /dev/null
+++ b/internal/dataservice/grpc_service.go
@@ -0,0 +1,40 @@
+package dataservice
+
+import (
+	"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
+	"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
+	"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
+	"golang.org/x/net/context"
+)
+
+func (ds *DataService) RegisterNode(context.Context, *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) {
+	return nil, nil
+}
+func (ds *DataService) Flush(context.Context, *datapb.FlushRequest) (*commonpb.Status, error) {
+	return nil, nil
+}
+func (ds *DataService) AssignSegmentID(ctx context.Context, request *datapb.AssignSegIDRequest) (*datapb.AssignSegIDResponse, error) {
+	//for _, req := range request.SegIDRequests {
+	//	segmentID, retCount, expireTs, err := ds.segAllocator.AllocSegment(req.CollectionID, req.PartitionID, req.ChannelID, int(req.Count))
+	//	if err != nil {
+	//		log.Printf()
+	//	}
+	//}
+	return nil, nil
+}
+func (ds *DataService) ShowSegments(context.Context, *datapb.ShowSegmentRequest) (*datapb.ShowSegmentResponse, error) {
+	return nil, nil
+
+}
+func (ds *DataService) GetSegmentStates(context.Context, *datapb.SegmentStatesRequest) (*datapb.SegmentStatesResponse, error) {
+	return nil, nil
+
+}
+func (ds *DataService) GetInsertBinlogPaths(context.Context, *datapb.InsertBinlogPathRequest) (*datapb.InsertBinlogPathsResponse, error) {
+	return nil, nil
+
+}
+func (ds *DataService) GetInsertChannels(context.Context, *datapb.InsertChannelRequest) (*internalpb2.StringList, error) {
+	return nil, nil
+
+}
diff --git a/internal/dataservice/meta.go b/internal/dataservice/meta.go
new file mode 100644
index 000000000..88b058e4b
--- /dev/null
+++ b/internal/dataservice/meta.go
@@ -0,0 +1,180 @@
+package dataservice
+
+import (
+	"fmt"
+	"strconv"
+	"sync"
+
+	log "github.com/sirupsen/logrus"
+
+	"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
+	"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
+
+	"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
+
+	"github.com/golang/protobuf/proto"
+	"github.com/zilliztech/milvus-distributed/internal/errors"
+	"github.com/zilliztech/milvus-distributed/internal/kv"
+)
+
+type (
+	UniqueID       = typeutil.UniqueID
+	Timestamp      = typeutil.Timestamp
+	collectionInfo struct {
+		ID     UniqueID
+		Schema *schemapb.CollectionSchema
+	}
+	meta struct {
+		client      kv.TxnBase                       // client of a reliable kv service, i.e. etcd client
+		collID2Info map[UniqueID]*collectionInfo     // collection id to collection info
+		segID2Info  map[UniqueID]*datapb.SegmentInfo // segment id to segment info
+
+		ddLock sync.RWMutex
+	}
+)
+
+func NewMetaTable(kv kv.TxnBase) (*meta, error) {
+	mt := &meta{
+		client:      kv,
+		collID2Info: make(map[UniqueID]*collectionInfo),
+		segID2Info:  make(map[UniqueID]*datapb.SegmentInfo),
+	}
+	err := mt.reloadFromKV()
+	if err != nil {
+		return nil, err
+	}
+	return mt, nil
+}
+
+func (mt *meta) reloadFromKV() error {
+	_, values, err := mt.client.LoadWithPrefix("segment")
+	if err != nil {
+		return err
+	}
+
+	for _, value := range values {
+		segmentInfo := &datapb.SegmentInfo{}
+		err = proto.UnmarshalText(value, segmentInfo)
+		if err != nil {
+			return err
+		}
+		mt.segID2Info[segmentInfo.SegmentID] = segmentInfo
+	}
+
+	return nil
+}
+
+func (mt *meta) AddCollection(collectionInfo *collectionInfo) error {
+	mt.ddLock.Lock()
+	defer mt.ddLock.Unlock()
+	if _, ok := mt.collID2Info[collectionInfo.ID]; ok {
+		return fmt.Errorf("collection %s with id %d already exist", collectionInfo.Schema.Name, collectionInfo.ID)
+	}
+	mt.collID2Info[collectionInfo.ID] = collectionInfo
+	return nil
+}
+
+func (mt *meta) DropCollection(collID UniqueID) error {
+	mt.ddLock.Lock()
+	defer mt.ddLock.Unlock()
+
+	if _, ok := mt.collID2Info[collID]; !ok {
+		return errors.Errorf("can't find collection. id = " + strconv.FormatInt(collID, 10))
+	}
+	delete(mt.collID2Info, collID)
+	for id, segment := range mt.segID2Info {
+		if segment.CollectionID != collID {
+			continue
+		}
+		delete(mt.segID2Info, id)
+		if err := mt.removeSegmentInfo(id); err != nil {
+			log.Printf("remove segment info failed, %s", err.Error())
+			_ = mt.reloadFromKV()
+		}
+	}
+	return nil
+}
+
+func (mt *meta) HasCollection(collID UniqueID) bool {
+	mt.ddLock.RLock()
+	defer mt.ddLock.RUnlock()
+	_, ok := mt.collID2Info[collID]
+	return ok
+}
+
+func (mt *meta) AddSegment(segmentInfo *datapb.SegmentInfo) error {
+	mt.ddLock.Lock()
+	defer mt.ddLock.Unlock()
+	if _, ok := mt.segID2Info[segmentInfo.SegmentID]; !ok {
+		return fmt.Errorf("segment %d already exist", segmentInfo.SegmentID)
+	}
+	mt.segID2Info[segmentInfo.SegmentID] = segmentInfo
+	if err := mt.saveSegmentInfo(segmentInfo); err != nil {
+		_ = mt.reloadFromKV()
+		return err
+	}
+	return nil
+}
+
+func (mt *meta) UpdateSegment(segmentInfo *datapb.SegmentInfo) error {
+	mt.ddLock.Lock()
+	defer mt.ddLock.Unlock()
+
+	mt.segID2Info[segmentInfo.SegmentID] = segmentInfo
+	if err := mt.saveSegmentInfo(segmentInfo); err != nil {
+		_ = mt.reloadFromKV()
+		return err
+	}
+	return nil
+}
+
+func (mt *meta) GetSegmentByID(segID UniqueID) (*datapb.SegmentInfo, error) {
+	mt.ddLock.RLock()
+	defer mt.ddLock.RUnlock()
+
+	segmentInfo, ok := mt.segID2Info[segID]
+	if !ok {
+		return nil, errors.Errorf("GetSegmentByID:can't find segment id = %d", segID)
+	}
+	return segmentInfo, nil
+}
+
+func (mt *meta) CloseSegment(segID UniqueID, closeTs Timestamp) error {
+	mt.ddLock.Lock()
+	defer mt.ddLock.Unlock()
+
+	segInfo, ok := mt.segID2Info[segID]
+	if !ok {
+		return errors.Errorf("DropSegment:can't find segment id = " + strconv.FormatInt(segID, 10))
+	}
+
+	segInfo.CloseTime = closeTs
+
+	err := mt.saveSegmentInfo(segInfo)
+	if err != nil {
+		_ = mt.reloadFromKV()
+		return err
+	}
+	return nil
+}
+
+func (mt *meta) GetCollection(collectionID UniqueID) (*collectionInfo, error) {
+	mt.ddLock.RLock()
+	defer mt.ddLock.RUnlock()
+
+	collectionInfo, ok := mt.collID2Info[collectionID]
+	if !ok {
+		return nil, fmt.Errorf("collection %d not found", collectionID)
+	}
+	return collectionInfo, nil
+}
+
+func (mt *meta) saveSegmentInfo(segmentInfo *datapb.SegmentInfo) error {
+	segBytes := proto.MarshalTextString(segmentInfo)
+
+	return mt.client.Save("/segment/"+strconv.FormatInt(segmentInfo.SegmentID, 10), segBytes)
+}
+
+func (mt *meta) removeSegmentInfo(segID UniqueID) error {
+	return mt.client.Remove("/segment/" + strconv.FormatInt(segID, 10))
+}
diff --git a/internal/dataservice/param.go b/internal/dataservice/param.go
new file mode 100644
index 000000000..db4ddb6a1
--- /dev/null
+++ b/internal/dataservice/param.go
@@ -0,0 +1,370 @@
+package dataservice
+
+import (
+	"log"
+	"strconv"
+	"strings"
+
+	"github.com/zilliztech/milvus-distributed/internal/util/paramtable"
+	"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
+	"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
+)
+
+type ParamTable struct {
+	paramtable.BaseTable
+
+	Address string
+	Port    int
+
+	EtcdAddress           string
+	MetaRootPath          string
+	KvRootPath            string
+	WriteNodeSegKvSubPath string
+	PulsarAddress         string
+	IndexBuilderAddress   string
+
+	// nodeID
+	ProxyIDList     []typeutil.UniqueID
+	WriteNodeIDList []typeutil.UniqueID
+
+	TopicNum                    int
+	QueryNodeNum                int
+	SoftTimeTickBarrierInterval typeutil.Timestamp
+
+	// segment
+	SegmentSize           float64
+	SegmentSizeFactor     float64
+	DefaultRecordSize     int64
+	MinSegIDAssignCnt     int64
+	MaxSegIDAssignCnt     int64
+	SegIDAssignExpiration int64
+
+	// msgChannel
+	ProxyTimeTickChannelNames     []string
+	WriteNodeTimeTickChannelNames []string
+	DDChannelNames                []string
+	InsertChannelNames            []string
+	K2SChannelNames               []string
+	QueryNodeStatsChannelName     string
+	MsgChannelSubName             string
+
+	MaxPartitionNum     int64
+	DefaultPartitionTag string
+
+	LoadIndexChannelNames []string
+}
+
+var Params ParamTable
+
+func (p *ParamTable) Init() {
+	// load yaml
+	p.BaseTable.Init()
+
+	err := p.LoadYaml("advanced/master.yaml")
+	if err != nil {
+		panic(err)
+	}
+
+	// set members
+	p.initAddress()
+	p.initPort()
+
+	p.initEtcdAddress()
+	p.initMetaRootPath()
+	p.initKvRootPath()
+	p.initWriteNodeSegKvSubPath()
+	p.initPulsarAddress()
+	p.initIndexBuilderAddress()
+
+	p.initProxyIDList()
+	p.initWriteNodeIDList()
+
+	p.initTopicNum()
+	p.initQueryNodeNum()
+	p.initSoftTimeTickBarrierInterval()
+
+	p.initSegmentSize()
+	p.initSegmentSizeFactor()
+	p.initDefaultRecordSize()
+	p.initMinSegIDAssignCnt()
+	p.initMaxSegIDAssignCnt()
+	p.initSegIDAssignExpiration()
+
+	p.initProxyTimeTickChannelNames()
+	p.initWriteNodeTimeTickChannelNames()
+	p.initInsertChannelNames()
+	p.initDDChannelNames()
+	p.initK2SChannelNames()
+	p.initQueryNodeStatsChannelName()
+	p.initMsgChannelSubName()
+	p.initMaxPartitionNum()
+	p.initDefaultPartitionTag()
+
+	p.initLoadIndexChannelNames()
+}
+
+func (p *ParamTable) initAddress() {
+	masterAddress, err := p.Load("master.address")
+	if err != nil {
+		panic(err)
+	}
+	p.Address = masterAddress
+}
+
+func (p *ParamTable) initPort() {
+	p.Port = p.ParseInt("master.port")
+}
+
+func (p *ParamTable) initEtcdAddress() {
+	addr, err := p.Load("_EtcdAddress")
+	if err != nil {
+		panic(err)
+	}
+	p.EtcdAddress = addr
+}
+
+func (p *ParamTable) initPulsarAddress() {
+	addr, err := p.Load("_PulsarAddress")
+	if err != nil {
+		panic(err)
+	}
+	p.PulsarAddress = addr
+}
+
+func (p *ParamTable) initIndexBuilderAddress() {
+	ret, err := p.Load("_IndexBuilderAddress")
+	if err != nil {
+		panic(err)
+	}
+	p.IndexBuilderAddress = ret
+}
+
+func (p *ParamTable) initMetaRootPath() {
+	rootPath, err := p.Load("etcd.rootPath")
+	if err != nil {
+		panic(err)
+	}
+	subPath, err := p.Load("etcd.metaSubPath")
+	if err != nil {
+		panic(err)
+	}
+	p.MetaRootPath = rootPath + "/" + subPath
+}
+
+func (p *ParamTable) initKvRootPath() {
+	rootPath, err := p.Load("etcd.rootPath")
+	if err != nil {
+		panic(err)
+	}
+	subPath, err := p.Load("etcd.kvSubPath")
+	if err != nil {
+		panic(err)
+	}
+	p.KvRootPath = rootPath + "/" + subPath
+}
+
+func (p *ParamTable) initWriteNodeSegKvSubPath() {
+	subPath, err := p.Load("etcd.writeNodeSegKvSubPath")
+	if err != nil {
+		panic(err)
+	}
+	p.WriteNodeSegKvSubPath = subPath + "/"
+}
+
+func (p *ParamTable) initTopicNum() {
+	iRangeStr, err := p.Load("msgChannel.channelRange.insert")
+	if err != nil {
+		panic(err)
+	}
+	rangeSlice := paramtable.ConvertRangeToIntRange(iRangeStr, ",")
+	p.TopicNum = rangeSlice[1] - rangeSlice[0]
+}
+
+func (p *ParamTable) initSegmentSize() {
+	p.SegmentSize = p.ParseFloat("master.segment.size")
+}
+
+func (p *ParamTable) initSegmentSizeFactor() {
+	p.SegmentSizeFactor = p.ParseFloat("master.segment.sizeFactor")
+}
+
+func (p *ParamTable) initDefaultRecordSize() {
+	p.DefaultRecordSize = p.ParseInt64("master.segment.defaultSizePerRecord")
+}
+
+func (p *ParamTable) initMinSegIDAssignCnt() {
+	p.MinSegIDAssignCnt = p.ParseInt64("master.segment.minIDAssignCnt")
+}
+
+func (p *ParamTable) initMaxSegIDAssignCnt() {
+	p.MaxSegIDAssignCnt = p.ParseInt64("master.segment.maxIDAssignCnt")
+}
+
+func (p *ParamTable) initSegIDAssignExpiration() {
+	p.SegIDAssignExpiration = p.ParseInt64("master.segment.IDAssignExpiration")
+}
+
+func (p *ParamTable) initQueryNodeNum() {
+	p.QueryNodeNum = len(p.QueryNodeIDList())
+}
+
+func (p *ParamTable) initQueryNodeStatsChannelName() {
+	channels, err := p.Load("msgChannel.chanNamePrefix.queryNodeStats")
+	if err != nil {
+		panic(err)
+	}
+	p.QueryNodeStatsChannelName = channels
+}
+
+func (p *ParamTable) initProxyIDList() {
+	p.ProxyIDList = p.BaseTable.ProxyIDList()
+}
+
+func (p *ParamTable) initProxyTimeTickChannelNames() {
+	ch, err := p.Load("msgChannel.chanNamePrefix.proxyTimeTick")
+	if err != nil {
+		log.Panic(err)
+	}
+	id, err := p.Load("nodeID.proxyIDList")
+	if err != nil {
+		log.Panicf("load proxy id list error, %s", err.Error())
+	}
+	ids := strings.Split(id, ",")
+	channels := make([]string, 0, len(ids))
+	for _, i := range ids {
+		_, err := strconv.ParseInt(i, 10, 64)
+		if err != nil {
+			log.Panicf("load proxy id list error, %s", err.Error())
+		}
+		channels = append(channels, ch+"-"+i)
+	}
+	p.ProxyTimeTickChannelNames = channels
+}
+
+func (p *ParamTable) initMsgChannelSubName() {
+	name, err := p.Load("msgChannel.subNamePrefix.masterSubNamePrefix")
+	if err != nil {
+		log.Panic(err)
+	}
+	p.MsgChannelSubName = name
+}
+
+func (p *ParamTable) initSoftTimeTickBarrierInterval() {
+	t, err := p.Load("master.timeSync.softTimeTickBarrierInterval")
+	if err != nil {
+		log.Panic(err)
+	}
+	v, err := strconv.ParseInt(t, 10, 64)
+	if err != nil {
+		log.Panic(err)
+	}
+	p.SoftTimeTickBarrierInterval = tsoutil.ComposeTS(v, 0)
+}
+
+func (p *ParamTable) initWriteNodeIDList() {
+	p.WriteNodeIDList = p.BaseTable.WriteNodeIDList()
+}
+
+func (p *ParamTable) initWriteNodeTimeTickChannelNames() {
+	ch, err := p.Load("msgChannel.chanNamePrefix.writeNodeTimeTick")
+	if err != nil {
+		log.Fatal(err)
+	}
+	id, err := p.Load("nodeID.writeNodeIDList")
+	if err != nil {
+		log.Panicf("load write node id list error, %s", err.Error())
+	}
+	ids := strings.Split(id, ",")
+	channels := make([]string, 0, len(ids))
+	for _, i := range ids {
+		_, err := strconv.ParseInt(i, 10, 64)
+		if err != nil {
+			log.Panicf("load write node id list error, %s", err.Error())
+		}
+		channels = append(channels, ch+"-"+i)
+	}
+	p.WriteNodeTimeTickChannelNames = channels
+}
+
+func (p *ParamTable) initDDChannelNames() {
+	prefix, err := p.Load("msgChannel.chanNamePrefix.dataDefinition")
+	if err != nil {
+		panic(err)
+	}
+	prefix += "-"
+	iRangeStr, err := p.Load("msgChannel.channelRange.dataDefinition")
+	if err != nil {
+		panic(err)
+	}
+	channelIDs := paramtable.ConvertRangeToIntSlice(iRangeStr, ",")
+	var ret []string
+	for _, ID := range channelIDs {
+		ret = append(ret, prefix+strconv.Itoa(ID))
+	}
+	p.DDChannelNames = ret
+}
+
+func (p *ParamTable) initInsertChannelNames() {
+	prefix, err := p.Load("msgChannel.chanNamePrefix.insert")
+	if err != nil {
+		panic(err)
+	}
+	prefix += "-"
+	iRangeStr, err := p.Load("msgChannel.channelRange.insert")
+	if err != nil {
+		panic(err)
+	}
+	channelIDs := paramtable.ConvertRangeToIntSlice(iRangeStr, ",")
+	var ret []string
+	for _, ID := range channelIDs {
+		ret = append(ret, prefix+strconv.Itoa(ID))
+	}
+	p.InsertChannelNames = ret
+}
+
+func (p *ParamTable) initK2SChannelNames() {
+	prefix, err := p.Load("msgChannel.chanNamePrefix.k2s")
+	if err != nil {
+		panic(err)
+	}
+	prefix += "-"
+	iRangeStr, err := p.Load("msgChannel.channelRange.k2s")
+	if err != nil {
+		panic(err)
+	}
+	channelIDs := paramtable.ConvertRangeToIntSlice(iRangeStr, ",")
+	var ret []string
+	for _, ID := range channelIDs {
+		ret = append(ret, prefix+strconv.Itoa(ID))
+	}
+	p.K2SChannelNames = ret
+}
+
+func (p *ParamTable) initMaxPartitionNum() {
+	str, err := p.Load("master.maxPartitionNum")
+	if err != nil {
+		panic(err)
+	}
+	maxPartitionNum, err := strconv.ParseInt(str, 10, 64)
+	if err != nil {
+		panic(err)
+	}
+	p.MaxPartitionNum = maxPartitionNum
+}
+
+func (p *ParamTable) initDefaultPartitionTag() {
+	defaultTag, err := p.Load("common.defaultPartitionTag")
+	if err != nil {
+		panic(err)
+	}
+
+	p.DefaultPartitionTag = defaultTag
+}
+
+func (p *ParamTable) initLoadIndexChannelNames() {
+	loadIndexChannelName, err := p.Load("msgChannel.chanNamePrefix.cmd")
+	if err != nil {
+		panic(err)
+	}
+	p.LoadIndexChannelNames = []string{loadIndexChannelName}
+}
diff --git a/internal/dataservice/segment_allocator.go b/internal/dataservice/segment_allocator.go
new file mode 100644
index 000000000..6c1ac9dea
--- /dev/null
+++ b/internal/dataservice/segment_allocator.go
@@ -0,0 +1,250 @@
+package dataservice
+
+import (
+	"fmt"
+	"strconv"
+	"sync"
+	"time"
+
+	"github.com/zilliztech/milvus-distributed/internal/proto/datapb"
+
+	"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
+
+	"github.com/zilliztech/milvus-distributed/internal/util/tsoutil"
+)
+
+type errRemainInSufficient struct {
+	requestRows int
+}
+
+func newErrRemainInSufficient(requestRows int) *errRemainInSufficient {
+	return &errRemainInSufficient{requestRows: requestRows}
+}
+
+func (err *errRemainInSufficient) Error() string {
+	return "segment remaining is insufficient for" + strconv.Itoa(err.requestRows)
+}
+
+// segmentAllocator is used to allocate rows for segments and record the allocations.
+type segmentAllocator interface {
+	// OpenSegment add the segment to allocator and set it allocatable
+	OpenSegment(segmentInfo *datapb.SegmentInfo) error
+	// AllocSegment allocate rows and record the allocation.
+	AllocSegment(collectionID UniqueID, partitionID UniqueID, channelName string, requestRows int) (UniqueID, int, Timestamp, error)
+	// GetSealedSegments get all sealed segment.
+	GetSealedSegments() ([]UniqueID, error)
+	// SealSegment set segment sealed, the segment will not be allocated anymore.
+	SealSegment(segmentID UniqueID)
+	// DropSegment drop the segment from allocator.
+	DropSegment(segmentID UniqueID)
+	// ExpireAllocations check all allocations' expire time and remove the expired allocation.
+	ExpireAllocations(timeTick Timestamp) error
+	// IsAllocationsExpired check all allocations of segment expired.
+	IsAllocationsExpired(segmentID UniqueID, ts Timestamp) (bool, error)
+}
+
+type (
+	segmentStatus struct {
+		id             UniqueID
+		collectionID   UniqueID
+		partitionID    UniqueID
+		total          int
+		sealed         bool
+		lastExpireTime Timestamp
+		allocations    []*allocation
+		cRange         channelRange
+	}
+	allocation struct {
+		rowNums    int
+		expireTime Timestamp
+	}
+	segmentAllocatorImpl struct {
+		mt                     *meta
+		segments               map[UniqueID]*segmentStatus //segment id -> status
+		cMapper                *insertChannelMapper
+		segmentExpireDuration  int64
+		defaultSizePerRecord   int64
+		segmentThreshold       float64
+		segmentThresholdFactor float64
+		numOfChannels          int
+		numOfQueryNodes        int
+		mu                     sync.RWMutex
+		globalIDAllocator      func() (UniqueID, error)
+		globalTSOAllocator     func() (Timestamp, error)
+	}
+)
+
+func newSegmentAssigner(metaTable *meta, globalIDAllocator func() (UniqueID, error),
+	globalTSOAllocator func() (Timestamp, error)) (*segmentAllocatorImpl, error) {
+	segmentAllocator := &segmentAllocatorImpl{
+		mt:                     metaTable,
+		segments:               make(map[UniqueID]*segmentStatus),
+		segmentExpireDuration:  Params.SegIDAssignExpiration,
+		defaultSizePerRecord:   Params.DefaultRecordSize,
+		segmentThreshold:       Params.SegmentSize * 1024 * 1024,
+		segmentThresholdFactor: Params.SegmentSizeFactor,
+		numOfChannels:          Params.TopicNum,
+		numOfQueryNodes:        Params.QueryNodeNum,
+		globalIDAllocator:      globalIDAllocator,
+		globalTSOAllocator:     globalTSOAllocator,
+	}
+	return segmentAllocator, nil
+}
+
+func (allocator *segmentAllocatorImpl) OpenSegment(segmentInfo *datapb.SegmentInfo) error {
+	if _, ok := allocator.segments[segmentInfo.SegmentID]; ok {
+		return fmt.Errorf("segment %d already exist", segmentInfo.SegmentID)
+	}
+	totalRows, err := allocator.estimateTotalRows(segmentInfo.CollectionID)
+	if err != nil {
+		return err
+	}
+	allocator.segments[segmentInfo.SegmentID] = &segmentStatus{
+		id:             segmentInfo.SegmentID,
+		collectionID:   segmentInfo.CollectionID,
+		partitionID:    segmentInfo.PartitionID,
+		total:          totalRows,
+		sealed:         false,
+		lastExpireTime: 0,
+		cRange:         segmentInfo.InsertChannels,
+	}
+	return nil
+}
+
+func (allocator *segmentAllocatorImpl) AllocSegment(collectionID UniqueID, partitionID UniqueID, channelName string, requestRows int) (segID UniqueID, retCount int, expireTime Timestamp, err error) {
+	allocator.mu.Lock()
+	defer allocator.mu.Unlock()
+
+	for _, segStatus := range allocator.segments {
+		if segStatus.sealed || segStatus.collectionID != collectionID || segStatus.partitionID != partitionID ||
+			!segStatus.cRange.Contains(channelName) {
+			continue
+		}
+		var success bool
+		success, err = allocator.alloc(segStatus, requestRows)
+		if err != nil {
+			return
+		}
+		if !success {
+			continue
+		}
+		segID = segStatus.id
+		retCount = requestRows
+		expireTime = segStatus.lastExpireTime
+		return
+	}
+
+	err = newErrRemainInSufficient(requestRows)
+	return
+}
+
+func (allocator *segmentAllocatorImpl) alloc(segStatus *segmentStatus, numRows int) (bool, error) {
+	totalOfAllocations := 0
+	for _, allocation := range segStatus.allocations {
+		totalOfAllocations += allocation.rowNums
+	}
+	segMeta, err := allocator.mt.GetSegmentByID(segStatus.id)
+	if err != nil {
+		return false, err
+	}
+	free := segStatus.total - int(segMeta.NumRows) - totalOfAllocations
+	if numRows > free {
+		return false, nil
+	}
+
+	ts, err := allocator.globalTSOAllocator()
+	if err != nil {
+		return false, err
+	}
+	physicalTs, logicalTs := tsoutil.ParseTS(ts)
+	expirePhysicalTs := physicalTs.Add(time.Duration(allocator.segmentExpireDuration) * time.Millisecond)
+	expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs))
+	segStatus.lastExpireTime = expireTs
+	segStatus.allocations = append(segStatus.allocations, &allocation{
+		numRows,
+		ts,
+	})
+
+	return true, nil
+}
+
+func (allocator *segmentAllocatorImpl) estimateTotalRows(collectionID UniqueID) (int, error) {
+	collMeta, err := allocator.mt.GetCollection(collectionID)
+	if err != nil {
+		return -1, err
+	}
+	sizePerRecord, err := typeutil.EstimateSizePerRecord(collMeta.Schema)
+	if err != nil {
+		return -1, err
+	}
+	return int(allocator.segmentThreshold / float64(sizePerRecord)), nil
+}
+
+func (allocator *segmentAllocatorImpl) GetSealedSegments() ([]UniqueID, error) {
+	allocator.mu.Lock()
+	defer allocator.mu.Unlock()
+	keys := make([]UniqueID, 0)
+	for _, segStatus := range allocator.segments {
+		if !segStatus.sealed {
+			sealed, err := allocator.checkSegmentSealed(segStatus)
+			if err != nil {
+				return nil, err
+			}
+			segStatus.sealed = sealed
+		}
+		if segStatus.sealed {
+			keys = append(keys, segStatus.id)
+		}
+	}
+	return keys, nil
+}
+
+func (allocator *segmentAllocatorImpl) checkSegmentSealed(segStatus *segmentStatus) (bool, error) {
+	segMeta, err := allocator.mt.GetSegmentByID(segStatus.id)
+	if err != nil {
+		return false, err
+	}
+	return float64(segMeta.NumRows) >= allocator.segmentThresholdFactor*float64(segStatus.total), nil
+}
+
+func (allocator *segmentAllocatorImpl) SealSegment(segmentID UniqueID) {
+	allocator.mu.Lock()
+	defer allocator.mu.Unlock()
+	status, ok := allocator.segments[segmentID]
+	if !ok {
+		return
+	}
+	status.sealed = true
+}
+
+func (allocator *segmentAllocatorImpl) DropSegment(segmentID UniqueID) {
+	allocator.mu.Lock()
+	defer allocator.mu.Unlock()
+	delete(allocator.segments, segmentID)
+}
+
+func (allocator *segmentAllocatorImpl) ExpireAllocations(timeTick Timestamp) error {
+	allocator.mu.Lock()
+	defer allocator.mu.Unlock()
+	for _, segStatus := range allocator.segments {
+		for i := 0; i < len(segStatus.allocations); i++ {
+			if timeTick < segStatus.allocations[i].expireTime {
+				continue
+			}
+			segStatus.allocations = append(segStatus.allocations[:i], segStatus.allocations[i+1:]...)
+			i--
+		}
+	}
+
+	return nil
+}
+
+func (allocator *segmentAllocatorImpl) IsAllocationsExpired(segmentID UniqueID, ts Timestamp) (bool, error) {
+	allocator.mu.RLock()
+	defer allocator.mu.RUnlock()
+	status, ok := allocator.segments[segmentID]
+	if !ok {
+		return false, fmt.Errorf("segment %d not found", segmentID)
+	}
+	return status.lastExpireTime <= ts, nil
+}
diff --git a/internal/dataservice/watcher.go b/internal/dataservice/watcher.go
new file mode 100644
index 000000000..4db2fdfbb
--- /dev/null
+++ b/internal/dataservice/watcher.go
@@ -0,0 +1,107 @@
+package dataservice
+
+import (
+	"log"
+
+	"golang.org/x/net/context"
+
+	"github.com/zilliztech/milvus-distributed/internal/msgstream"
+)
+
+type (
+	proxyTimeTickWatcher struct {
+		allocator  segmentAllocator
+		msgQueue   chan *msgstream.TimeTickMsg
+		ctx        context.Context
+		cancelFunc context.CancelFunc
+	}
+	dataNodeTimeTickWatcher struct {
+		allocator  segmentAllocator
+		msgQueue   chan *msgstream.TimeTickMsg
+		ctx        context.Context
+		cancelFunc context.CancelFunc
+	}
+)
+
+func newProxyTimeTickWatcher(ctx context.Context, allocator segmentAllocator) *proxyTimeTickWatcher {
+	cancel, cancelFunc := context.WithCancel(ctx)
+	return &proxyTimeTickWatcher{
+		allocator:  allocator,
+		msgQueue:   make(chan *msgstream.TimeTickMsg, 1),
+		ctx:        cancel,
+		cancelFunc: cancelFunc,
+	}
+}
+
+func (watcher *proxyTimeTickWatcher) Start() {
+	go watcher.handleProxyTimeTickMsg()
+}
+
+func (watcher *proxyTimeTickWatcher) Close() {
+	watcher.cancelFunc()
+}
+
+func (watcher *proxyTimeTickWatcher) Watch(msg *msgstream.TimeTickMsg) {
+	watcher.msgQueue <- msg
+}
+
+func (watcher *proxyTimeTickWatcher) handleProxyTimeTickMsg() {
+	for {
+		select {
+		case <-watcher.ctx.Done():
+			return
+		case msg := <-watcher.msgQueue:
+			if err := watcher.allocator.ExpireAllocations(msg.Timestamp); err != nil {
+				log.Printf("expire allocations error : %s", err.Error())
+			}
+		}
+	}
+}
+
+func newDataNodeTimeTickWatcher(ctx context.Context, allocator segmentAllocator) *dataNodeTimeTickWatcher {
+	cancel, cancelFunc := context.WithCancel(ctx)
+	return &dataNodeTimeTickWatcher{
+		allocator:  allocator,
+		msgQueue:   make(chan *msgstream.TimeTickMsg, 1),
+		ctx:        cancel,
+		cancelFunc: cancelFunc,
+	}
+}
+
+func (watcher *dataNodeTimeTickWatcher) Watch(msg *msgstream.TimeTickMsg) {
+	watcher.msgQueue <- msg
+}
+
+func (watcher *dataNodeTimeTickWatcher) Start() {
+	go watcher.handleDataNodeTimeTickMsg()
+}
+
+func (watcher *dataNodeTimeTickWatcher) Close() {
+	watcher.cancelFunc()
+}
+
+func (watcher *dataNodeTimeTickWatcher) handleDataNodeTimeTickMsg() {
+	for {
+		select {
+		case <-watcher.ctx.Done():
+			return
+		case msg := <-watcher.msgQueue:
+			segments, err := watcher.allocator.GetSealedSegments()
+			if err != nil {
+				log.Printf("get sealed segments error %s", err.Error())
+				continue
+			}
+			for _, id := range segments {
+				expired, err := watcher.allocator.IsAllocationsExpired(id, msg.Timestamp)
+				if err != nil {
+					log.Printf("check allocations expired error %s", err.Error())
+					continue
+				}
+				if expired {
+					// TODO: flush segment
+					watcher.allocator.DropSegment(id)
+				}
+			}
+		}
+	}
+}
diff --git a/internal/proto/data_service.proto b/internal/proto/data_service.proto
index d1a4f95d1..29f1f6f99 100644
--- a/internal/proto/data_service.proto
+++ b/internal/proto/data_service.proto
@@ -113,6 +113,16 @@ message FlushSegRequest {
   repeated int64 segmentIDs = 4;
 }
 
+message SegmentInfo {
+  int64 segmentID=1;
+  int64 collectionID =2;
+  int64 partitionID=3;
+  repeated string insert_channels = 4;
+  uint64 open_time=5;
+  uint64 close_time=6;
+  int64 num_rows=7;
+  int64 mem_size=8;
+}
 
 service DataService {
 
diff --git a/internal/proto/datapb/data_service.pb.go b/internal/proto/datapb/data_service.pb.go
index 141c5ac8c..c5a9e14cb 100644
--- a/internal/proto/datapb/data_service.pb.go
+++ b/internal/proto/datapb/data_service.pb.go
@@ -897,6 +897,101 @@ func (m *FlushSegRequest) GetSegmentIDs() []int64 {
 	return nil
 }
 
+type SegmentInfo struct {
+	SegmentID            int64    `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
+	CollectionID         int64    `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
+	PartitionID          int64    `protobuf:"varint,3,opt,name=partitionID,proto3" json:"partitionID,omitempty"`
+	InsertChannels       []string `protobuf:"bytes,4,rep,name=insert_channels,json=insertChannels,proto3" json:"insert_channels,omitempty"`
+	OpenTime             uint64   `protobuf:"varint,5,opt,name=open_time,json=openTime,proto3" json:"open_time,omitempty"`
+	CloseTime            uint64   `protobuf:"varint,6,opt,name=close_time,json=closeTime,proto3" json:"close_time,omitempty"`
+	NumRows              int64    `protobuf:"varint,7,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"`
+	MemSize              int64    `protobuf:"varint,8,opt,name=mem_size,json=memSize,proto3" json:"mem_size,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *SegmentInfo) Reset()         { *m = SegmentInfo{} }
+func (m *SegmentInfo) String() string { return proto.CompactTextString(m) }
+func (*SegmentInfo) ProtoMessage()    {}
+func (*SegmentInfo) Descriptor() ([]byte, []int) {
+	return fileDescriptor_3385cd32ad6cfe64, []int{16}
+}
+
+func (m *SegmentInfo) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_SegmentInfo.Unmarshal(m, b)
+}
+func (m *SegmentInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_SegmentInfo.Marshal(b, m, deterministic)
+}
+func (m *SegmentInfo) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SegmentInfo.Merge(m, src)
+}
+func (m *SegmentInfo) XXX_Size() int {
+	return xxx_messageInfo_SegmentInfo.Size(m)
+}
+func (m *SegmentInfo) XXX_DiscardUnknown() {
+	xxx_messageInfo_SegmentInfo.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_SegmentInfo proto.InternalMessageInfo
+
+func (m *SegmentInfo) GetSegmentID() int64 {
+	if m != nil {
+		return m.SegmentID
+	}
+	return 0
+}
+
+func (m *SegmentInfo) GetCollectionID() int64 {
+	if m != nil {
+		return m.CollectionID
+	}
+	return 0
+}
+
+func (m *SegmentInfo) GetPartitionID() int64 {
+	if m != nil {
+		return m.PartitionID
+	}
+	return 0
+}
+
+func (m *SegmentInfo) GetInsertChannels() []string {
+	if m != nil {
+		return m.InsertChannels
+	}
+	return nil
+}
+
+func (m *SegmentInfo) GetOpenTime() uint64 {
+	if m != nil {
+		return m.OpenTime
+	}
+	return 0
+}
+
+func (m *SegmentInfo) GetCloseTime() uint64 {
+	if m != nil {
+		return m.CloseTime
+	}
+	return 0
+}
+
+func (m *SegmentInfo) GetNumRows() int64 {
+	if m != nil {
+		return m.NumRows
+	}
+	return 0
+}
+
+func (m *SegmentInfo) GetMemSize() int64 {
+	if m != nil {
+		return m.MemSize
+	}
+	return 0
+}
+
 func init() {
 	proto.RegisterEnum("milvus.proto.data.SegmentState", SegmentState_name, SegmentState_value)
 	proto.RegisterType((*RegisterNodeRequest)(nil), "milvus.proto.data.RegisterNodeRequest")
@@ -915,72 +1010,79 @@ func init() {
 	proto.RegisterType((*InsertChannelRequest)(nil), "milvus.proto.data.InsertChannelRequest")
 	proto.RegisterType((*WatchDmChannelRequest)(nil), "milvus.proto.data.WatchDmChannelRequest")
 	proto.RegisterType((*FlushSegRequest)(nil), "milvus.proto.data.FlushSegRequest")
+	proto.RegisterType((*SegmentInfo)(nil), "milvus.proto.data.SegmentInfo")
 }
 
 func init() { proto.RegisterFile("data_service.proto", fileDescriptor_3385cd32ad6cfe64) }
 
 var fileDescriptor_3385cd32ad6cfe64 = []byte{
-	// 952 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x41, 0x8f, 0xdb, 0x44,
-	0x14, 0x8e, 0xe3, 0x64, 0xdb, 0xbc, 0xa4, 0xbb, 0xd9, 0xd9, 0x14, 0x42, 0xba, 0xda, 0x0d, 0x23,
-	0xd1, 0x46, 0x15, 0x24, 0xd2, 0x56, 0x85, 0x1b, 0xa2, 0x4b, 0xca, 0x2a, 0x12, 0xac, 0xaa, 0x09,
-	0x12, 0x52, 0x39, 0x44, 0x93, 0xf8, 0xe1, 0x8c, 0xea, 0xd8, 0xc6, 0x33, 0x69, 0xab, 0xbd, 0xc1,
-	0x89, 0x13, 0xe2, 0x0f, 0xf0, 0x57, 0xf8, 0x5b, 0x5c, 0x91, 0x67, 0x1c, 0xc7, 0x4e, 0x1c, 0x52,
-	0x69, 0xa9, 0xb8, 0x65, 0x5e, 0xbe, 0x79, 0xef, 0x9b, 0x6f, 0xde, 0xfb, 0xc6, 0x40, 0x1c, 0xae,
-	0xf8, 0x44, 0x62, 0xf4, 0x5a, 0xcc, 0xb0, 0x1f, 0x46, 0x81, 0x0a, 0xc8, 0xf1, 0x42, 0x78, 0xaf,
-	0x97, 0xd2, 0xac, 0xfa, 0x31, 0xa0, 0xd3, 0x98, 0x05, 0x8b, 0x45, 0xe0, 0x9b, 0x50, 0xe7, 0x50,
-	0xf8, 0x0a, 0x23, 0x9f, 0x7b, 0x66, 0x4d, 0x7f, 0xb1, 0xe0, 0x84, 0xa1, 0x2b, 0xa4, 0xc2, 0xe8,
-	0x3a, 0x70, 0x90, 0xe1, 0xcf, 0x4b, 0x94, 0x8a, 0x5c, 0x40, 0x65, 0xca, 0x25, 0xb6, 0xad, 0xae,
-	0xd5, 0xab, 0x5f, 0x9c, 0xf5, 0x73, 0x79, 0xd3, 0x1c, 0xdf, 0x49, 0xf7, 0x92, 0x4b, 0x64, 0x1a,
-	0x4b, 0x3e, 0x87, 0x3b, 0xdc, 0x71, 0x22, 0x94, 0xb2, 0x5d, 0xd6, 0xdb, 0x4e, 0xf3, 0xdb, 0x12,
-	0x22, 0xcf, 0x0c, 0x86, 0xad, 0xc0, 0xf4, 0x25, 0xb4, 0xf2, 0x14, 0x64, 0x18, 0xf8, 0x12, 0xc9,
-	0x25, 0xd4, 0x85, 0x2f, 0xd4, 0x24, 0xe4, 0x11, 0x5f, 0xc8, 0x84, 0xca, 0xc7, 0x3b, 0xa8, 0x8c,
-	0x7c, 0xa1, 0x5e, 0x68, 0x20, 0x03, 0x91, 0xfe, 0xa6, 0xbf, 0x59, 0xd0, 0x18, 0xa3, 0x3b, 0x1a,
-	0xae, 0x0e, 0xd6, 0x82, 0xea, 0x2c, 0x58, 0xfa, 0x4a, 0xa7, 0xab, 0x32, 0xb3, 0x20, 0xa7, 0x50,
-	0x9b, 0xcd, 0xb9, 0xef, 0xa3, 0x37, 0x1a, 0x6a, 0xf2, 0x35, 0xb6, 0x0e, 0x10, 0x0a, 0x8d, 0x59,
-	0xe0, 0x79, 0x38, 0x53, 0x22, 0xf0, 0x47, 0xc3, 0xb6, 0xdd, 0xb5, 0x7a, 0x36, 0xcb, 0xc5, 0x48,
-	0x17, 0xea, 0x21, 0x8f, 0x94, 0x48, 0x20, 0x15, 0x0d, 0xc9, 0x86, 0xe8, 0x1f, 0x16, 0x90, 0x67,
-	0x52, 0x0a, 0xd7, 0xcf, 0x11, 0xfa, 0x00, 0x0e, 0xfc, 0xc0, 0xc1, 0xd1, 0x50, 0x33, 0xb2, 0x59,
-	0xb2, 0x22, 0x0f, 0xa0, 0x16, 0x22, 0x46, 0x93, 0x28, 0xf0, 0x30, 0xa1, 0x74, 0x37, 0x0e, 0xb0,
-	0xc0, 0x43, 0xf2, 0x1c, 0xee, 0xc9, 0x4c, 0x12, 0xd9, 0xb6, 0xbb, 0x76, 0xaf, 0x7e, 0x71, 0xde,
-	0xdf, 0xba, 0xff, 0x7e, 0xb6, 0x18, 0xcb, 0xef, 0xa2, 0x7f, 0x5b, 0x70, 0xa4, 0xff, 0x37, 0xbc,
-	0x16, 0xe8, 0x6b, 0x81, 0x34, 0x28, 0xa1, 0x63, 0x16, 0xdb, 0x02, 0x55, 0xb3, 0x02, 0xa5, 0xa2,
-	0xc6, 0xca, 0xdc, 0x5b, 0x89, 0xba, 0x29, 0x5b, 0x65, 0xbf, 0x6c, 0xd5, 0x2d, 0xd9, 0xc8, 0x39,
-	0xd4, 0xf1, 0x6d, 0x28, 0x22, 0x9c, 0x28, 0xb1, 0xc0, 0xf6, 0x41, 0xd7, 0xea, 0x55, 0x18, 0x98,
-	0xd0, 0xf7, 0x62, 0x81, 0xe4, 0x09, 0x1c, 0x48, 0xc5, 0xd5, 0x52, 0xb6, 0xef, 0xe8, 0x0e, 0x79,
-	0x50, 0xd8, 0x75, 0x63, 0x0d, 0x61, 0x09, 0x94, 0x22, 0x9c, 0xe4, 0xee, 0x22, 0x69, 0xb9, 0x6b,
-	0x68, 0xca, 0xbc, 0x1e, 0x71, 0xdf, 0xc5, 0xd2, 0xd2, 0x5d, 0xd2, 0xae, 0xa1, 0x6c, 0x6b, 0x2f,
-	0xbd, 0x81, 0xc6, 0x37, 0xde, 0x52, 0xce, 0x6f, 0x33, 0x56, 0x04, 0x2a, 0xce, 0x34, 0x51, 0xdd,
-	0x66, 0xfa, 0xf7, 0xbb, 0x48, 0x4b, 0x7f, 0xb7, 0x80, 0x8c, 0xe7, 0xc1, 0x9b, 0x31, 0xba, 0x9a,
-	0xdd, 0x2d, 0x28, 0x6c, 0x96, 0x2b, 0xef, 0xbf, 0x49, 0x7b, 0x7b, 0x00, 0x9e, 0xc2, 0x49, 0x8e,
-	0x4f, 0xa2, 0xf9, 0x19, 0x80, 0x34, 0xa1, 0xd1, 0xd0, 0xa8, 0x6d, 0xb3, 0x4c, 0x84, 0xce, 0xa1,
-	0x95, 0x6c, 0x89, 0xef, 0x10, 0xe5, 0x6d, 0x0e, 0x72, 0x0a, 0xb5, 0x34, 0x73, 0x72, 0x8a, 0x75,
-	0x20, 0x9e, 0xd0, 0xfb, 0x1b, 0xa5, 0x12, 0x8e, 0x4f, 0xa1, 0x1a, 0x37, 0x8e, 0x29, 0x76, 0xb8,
-	0x6b, 0xce, 0xd2, 0x8d, 0xcc, 0xa0, 0xe3, 0xde, 0x9d, 0x45, 0xc8, 0x55, 0xd2, 0xbb, 0x65, 0xd3,
-	0xbb, 0x26, 0xa4, 0x7b, 0xf7, 0x1c, 0xea, 0x12, 0xb9, 0x87, 0x8e, 0x01, 0xd8, 0x06, 0x60, 0x42,
-	0x31, 0x80, 0xbe, 0x82, 0x0f, 0x47, 0xbe, 0xc4, 0x48, 0x5d, 0x0a, 0xdf, 0x0b, 0xdc, 0x17, 0x5c,
-	0xcd, 0xdf, 0xdf, 0xf9, 0x43, 0xf8, 0x68, 0xb3, 0xd8, 0x5a, 0x82, 0x0e, 0xdc, 0xfd, 0x49, 0xa0,
-	0xe7, 0xac, 0x2f, 0x29, 0x5d, 0x93, 0x2f, 0xa0, 0x1a, 0xc6, 0xe0, 0x76, 0x59, 0xcf, 0xca, 0x2e,
-	0x8f, 0x1e, 0xab, 0x48, 0xf8, 0xee, 0xb7, 0x42, 0x2a, 0x66, 0xf0, 0xf4, 0x57, 0x0b, 0x5a, 0xa6,
-	0xe4, 0xd7, 0xc6, 0x4c, 0xde, 0xf7, 0xa0, 0x14, 0x58, 0x37, 0x7d, 0x05, 0xf7, 0x7f, 0xe0, 0x6a,
-	0x36, 0x1f, 0x2e, 0xfe, 0x03, 0x12, 0x67, 0x00, 0xa9, 0x2f, 0x1a, 0x3d, 0x6a, 0x2c, 0x13, 0xa1,
-	0x7f, 0x5a, 0x70, 0xa4, 0x2d, 0x61, 0x8c, 0xee, 0xff, 0x70, 0xd8, 0x8d, 0x69, 0xab, 0x6c, 0x4e,
-	0xdb, 0xe3, 0x1f, 0xf5, 0x7b, 0x99, 0x76, 0x32, 0x39, 0x82, 0x7a, 0xb2, 0xbe, 0x0e, 0x7c, 0x6c,
-	0x96, 0xc8, 0x89, 0x7e, 0x32, 0x4c, 0x40, 0x3d, 0x7f, 0x2b, 0xa4, 0x6a, 0x5a, 0x84, 0xc0, 0x61,
-	0x12, 0xbc, 0x8a, 0x82, 0x37, 0xc2, 0x77, 0x9b, 0x65, 0x72, 0x0c, 0xf7, 0x56, 0x99, 0x74, 0x3f,
-	0x37, 0xed, 0x8b, 0xbf, 0xaa, 0x50, 0x1f, 0x72, 0xc5, 0xc7, 0xe6, 0xa3, 0x85, 0x70, 0x68, 0x64,
-	0x5f, 0x7e, 0xf2, 0xb0, 0x60, 0xae, 0x0a, 0xbe, 0x4e, 0x3a, 0x8f, 0xf6, 0xe2, 0x4c, 0xd3, 0xd2,
-	0x12, 0xb9, 0x82, 0xaa, 0x96, 0x9b, 0x14, 0xcd, 0x6c, 0xd6, 0x9b, 0x3b, 0xff, 0xf6, 0x6e, 0xd0,
-	0x12, 0x99, 0xc2, 0x51, 0xfa, 0x62, 0x18, 0xb1, 0xc8, 0x27, 0x05, 0x29, 0xb7, 0x5f, 0xf8, 0xce,
-	0xc3, 0x7d, 0xb0, 0x94, 0xec, 0x04, 0x1a, 0x19, 0x87, 0x94, 0x85, 0x05, 0xb6, 0x2d, 0xbd, 0xb0,
-	0x40, 0x81, 0xd3, 0xd2, 0x12, 0x71, 0xa1, 0x79, 0x85, 0x2a, 0xe7, 0x71, 0xe4, 0xd1, 0x1e, 0x33,
-	0x5b, 0x19, 0x6e, 0xa7, 0xb7, 0x1f, 0x98, 0x16, 0x8a, 0xa0, 0x75, 0x85, 0x6a, 0xcb, 0x4d, 0xc8,
-	0xe3, 0x82, 0x1c, 0x3b, 0x0c, 0xae, 0xf3, 0xe9, 0x3b, 0x60, 0xb3, 0x35, 0x39, 0x1c, 0xa7, 0x35,
-	0x93, 0x49, 0x2e, 0x3e, 0x5d, 0x91, 0xe3, 0x74, 0xf6, 0x9b, 0x16, 0x2d, 0x5d, 0x7e, 0xf5, 0xf2,
-	0x4b, 0x57, 0xa8, 0xf9, 0x72, 0x1a, 0xb7, 0xc7, 0xe0, 0x46, 0x78, 0x9e, 0xb8, 0x51, 0x38, 0x9b,
-	0x0f, 0xcc, 0xde, 0xcf, 0x1c, 0x21, 0x55, 0x24, 0xa6, 0x4b, 0x85, 0xce, 0x60, 0x95, 0x61, 0xa0,
-	0x13, 0x0e, 0xe2, 0xca, 0xe1, 0x74, 0x7a, 0xa0, 0x57, 0x4f, 0xfe, 0x09, 0x00, 0x00, 0xff, 0xff,
-	0x63, 0x83, 0xa9, 0x0a, 0xbe, 0x0b, 0x00, 0x00,
+	// 1051 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x51, 0x6f, 0xdb, 0x36,
+	0x10, 0x8e, 0x2c, 0x3b, 0x89, 0xcf, 0x4e, 0xe2, 0x30, 0xe9, 0xe6, 0xba, 0x59, 0x92, 0x09, 0x58,
+	0x13, 0x14, 0x5b, 0x02, 0xa4, 0xe8, 0xf6, 0x36, 0xac, 0x99, 0xbb, 0xc0, 0xc0, 0x16, 0x14, 0xf4,
+	0x80, 0x01, 0xdd, 0x83, 0x21, 0x5b, 0x57, 0x99, 0xa8, 0x44, 0x6a, 0x22, 0xdd, 0x14, 0x79, 0xdb,
+	0x9e, 0xf6, 0x30, 0x0c, 0xfb, 0x03, 0xfb, 0x2b, 0xfb, 0x5b, 0x7b, 0x1d, 0x44, 0xca, 0xb2, 0x64,
+	0xcb, 0x73, 0x81, 0xb4, 0xe8, 0x9b, 0x78, 0xfa, 0x78, 0x77, 0xfc, 0xf8, 0xdd, 0x1d, 0x81, 0x78,
+	0xae, 0x72, 0x07, 0x12, 0xe3, 0xd7, 0x6c, 0x84, 0x67, 0x51, 0x2c, 0x94, 0x20, 0xbb, 0x21, 0x0b,
+	0x5e, 0x4f, 0xa4, 0x59, 0x9d, 0x25, 0x80, 0x4e, 0x73, 0x24, 0xc2, 0x50, 0x70, 0x63, 0xea, 0x6c,
+	0x33, 0xae, 0x30, 0xe6, 0x6e, 0x60, 0xd6, 0xce, 0xaf, 0x16, 0xec, 0x51, 0xf4, 0x99, 0x54, 0x18,
+	0x5f, 0x0b, 0x0f, 0x29, 0xfe, 0x32, 0x41, 0xa9, 0xc8, 0x05, 0x54, 0x87, 0xae, 0xc4, 0xb6, 0x75,
+	0x6c, 0x9d, 0x36, 0x2e, 0x0e, 0xcf, 0x0a, 0x7e, 0x33, 0x1f, 0x3f, 0x48, 0xff, 0xd2, 0x95, 0x48,
+	0x35, 0x96, 0x7c, 0x09, 0x1b, 0xae, 0xe7, 0xc5, 0x28, 0x65, 0xbb, 0xa2, 0xb7, 0x1d, 0x14, 0xb7,
+	0xa5, 0x89, 0x3c, 0x35, 0x18, 0x3a, 0x05, 0x3b, 0x2f, 0x60, 0xbf, 0x98, 0x82, 0x8c, 0x04, 0x97,
+	0x48, 0x2e, 0xa1, 0xc1, 0x38, 0x53, 0x83, 0xc8, 0x8d, 0xdd, 0x50, 0xa6, 0xa9, 0x7c, 0xba, 0x24,
+	0x95, 0x1e, 0x67, 0xea, 0xb9, 0x06, 0x52, 0x60, 0xd9, 0xb7, 0xf3, 0xbb, 0x05, 0xcd, 0x3e, 0xfa,
+	0xbd, 0xee, 0xf4, 0x60, 0xfb, 0x50, 0x1b, 0x89, 0x09, 0x57, 0xda, 0x5d, 0x8d, 0x9a, 0x05, 0x39,
+	0x80, 0xfa, 0x68, 0xec, 0x72, 0x8e, 0x41, 0xaf, 0xab, 0x93, 0xaf, 0xd3, 0x99, 0x81, 0x38, 0xd0,
+	0x1c, 0x89, 0x20, 0xc0, 0x91, 0x62, 0x82, 0xf7, 0xba, 0x6d, 0xfb, 0xd8, 0x3a, 0xb5, 0x69, 0xc1,
+	0x46, 0x8e, 0xa1, 0x11, 0xb9, 0xb1, 0x62, 0x29, 0xa4, 0xaa, 0x21, 0x79, 0x93, 0xf3, 0x97, 0x05,
+	0xe4, 0xa9, 0x94, 0xcc, 0xe7, 0x85, 0x84, 0x3e, 0x82, 0x75, 0x2e, 0x3c, 0xec, 0x75, 0x75, 0x46,
+	0x36, 0x4d, 0x57, 0xe4, 0x01, 0xd4, 0x23, 0xc4, 0x78, 0x10, 0x8b, 0x00, 0xd3, 0x94, 0x36, 0x13,
+	0x03, 0x15, 0x01, 0x92, 0x67, 0xb0, 0x25, 0x73, 0x4e, 0x64, 0xdb, 0x3e, 0xb6, 0x4f, 0x1b, 0x17,
+	0x47, 0x67, 0x0b, 0xf7, 0x7f, 0x96, 0x0f, 0x46, 0x8b, 0xbb, 0x9c, 0x7f, 0x2d, 0xd8, 0xd1, 0xff,
+	0x4d, 0x5e, 0x21, 0x72, 0x4d, 0x90, 0x06, 0xa5, 0xe9, 0x98, 0xc5, 0x22, 0x41, 0xb5, 0x3c, 0x41,
+	0x19, 0xa9, 0x09, 0x33, 0x5b, 0x53, 0x52, 0xe7, 0x69, 0xab, 0xae, 0xa6, 0xad, 0xb6, 0x40, 0x1b,
+	0x39, 0x82, 0x06, 0xbe, 0x89, 0x58, 0x8c, 0x03, 0xc5, 0x42, 0x6c, 0xaf, 0x1f, 0x5b, 0xa7, 0x55,
+	0x0a, 0xc6, 0xf4, 0x23, 0x0b, 0x91, 0x3c, 0x86, 0x75, 0xa9, 0x5c, 0x35, 0x91, 0xed, 0x0d, 0xad,
+	0x90, 0x07, 0xa5, 0xaa, 0xeb, 0x6b, 0x08, 0x4d, 0xa1, 0x0e, 0xc2, 0x5e, 0xe1, 0x2e, 0x52, 0xc9,
+	0x5d, 0x43, 0x4b, 0x16, 0xf9, 0x48, 0x74, 0x97, 0x50, 0xeb, 0x2c, 0xa3, 0x76, 0x06, 0xa5, 0x0b,
+	0x7b, 0x9d, 0x5b, 0x68, 0x7e, 0x17, 0x4c, 0xe4, 0xf8, 0x2e, 0x65, 0x45, 0xa0, 0xea, 0x0d, 0x53,
+	0xd6, 0x6d, 0xaa, 0xbf, 0xdf, 0x86, 0x5a, 0xe7, 0x4f, 0x0b, 0x48, 0x7f, 0x2c, 0x6e, 0xfa, 0xe8,
+	0xeb, 0xec, 0xee, 0x90, 0xc2, 0x7c, 0xb8, 0xca, 0xea, 0x9b, 0xb4, 0x17, 0x0b, 0xe0, 0x09, 0xec,
+	0x15, 0xf2, 0x49, 0x39, 0x3f, 0x04, 0x90, 0xc6, 0xd4, 0xeb, 0x1a, 0xb6, 0x6d, 0x9a, 0xb3, 0x38,
+	0x63, 0xd8, 0x4f, 0xb7, 0x24, 0x77, 0x88, 0xf2, 0x2e, 0x07, 0x39, 0x80, 0x7a, 0xe6, 0x39, 0x3d,
+	0xc5, 0xcc, 0x90, 0x54, 0xe8, 0xbd, 0xb9, 0x50, 0x69, 0x8e, 0x4f, 0xa0, 0x96, 0x08, 0xc7, 0x04,
+	0xdb, 0x5e, 0x56, 0x67, 0xd9, 0x46, 0x6a, 0xd0, 0x89, 0x76, 0x47, 0x31, 0xba, 0x2a, 0xd5, 0x6e,
+	0xc5, 0x68, 0xd7, 0x98, 0xb4, 0x76, 0x8f, 0xa0, 0x21, 0xd1, 0x0d, 0xd0, 0x33, 0x00, 0xdb, 0x00,
+	0x8c, 0x29, 0x01, 0x38, 0xaf, 0xe0, 0xe3, 0x1e, 0x97, 0x18, 0xab, 0x4b, 0xc6, 0x03, 0xe1, 0x3f,
+	0x77, 0xd5, 0xf8, 0xfd, 0x9d, 0x3f, 0x82, 0xfb, 0xf3, 0xc1, 0x66, 0x14, 0x74, 0x60, 0xf3, 0x25,
+	0xc3, 0xc0, 0x9b, 0x5d, 0x52, 0xb6, 0x26, 0x5f, 0x41, 0x2d, 0x4a, 0xc0, 0xed, 0x8a, 0xae, 0x95,
+	0x65, 0x3d, 0xba, 0xaf, 0x62, 0xc6, 0xfd, 0xef, 0x99, 0x54, 0xd4, 0xe0, 0x9d, 0xdf, 0x2c, 0xd8,
+	0x37, 0x21, 0xbf, 0x35, 0xcd, 0xe4, 0x7d, 0x17, 0x4a, 0x49, 0xeb, 0x76, 0x5e, 0xc1, 0xbd, 0x9f,
+	0x5c, 0x35, 0x1a, 0x77, 0xc3, 0x77, 0x90, 0xc4, 0x21, 0x40, 0xd6, 0x17, 0x0d, 0x1f, 0x75, 0x9a,
+	0xb3, 0x38, 0x7f, 0x5b, 0xb0, 0xa3, 0x5b, 0x42, 0x1f, 0xfd, 0x0f, 0x70, 0xd8, 0xb9, 0x6a, 0xab,
+	0x2e, 0x54, 0xdb, 0x1f, 0x15, 0x68, 0xa4, 0x52, 0xee, 0xf1, 0x97, 0xa2, 0xa8, 0x18, 0x6b, 0x4e,
+	0x31, 0xef, 0xa6, 0x31, 0x90, 0x13, 0xd8, 0x61, 0x5a, 0x04, 0x83, 0x94, 0x28, 0x93, 0x58, 0x9d,
+	0x6e, 0xb3, 0xbc, 0x36, 0x64, 0x32, 0x13, 0x45, 0x84, 0xdc, 0x14, 0x4b, 0x4d, 0x17, 0xcb, 0x66,
+	0x62, 0xd0, 0xb5, 0xf4, 0x09, 0xc0, 0x28, 0x10, 0xb2, 0x30, 0x27, 0xea, 0xda, 0xa2, 0x7f, 0xdf,
+	0x87, 0x4d, 0x3e, 0x09, 0x07, 0xb1, 0xb8, 0x31, 0x83, 0xc2, 0xa6, 0x1b, 0x7c, 0x12, 0x52, 0x71,
+	0x23, 0x93, 0x5f, 0x21, 0x86, 0x03, 0xc9, 0x6e, 0xb1, 0xbd, 0x69, 0x7e, 0x85, 0x18, 0xf6, 0xd9,
+	0x2d, 0x3e, 0xfa, 0x59, 0x3f, 0x1f, 0xb2, 0xc2, 0x26, 0x3b, 0x19, 0x3b, 0xd7, 0x82, 0x63, 0x6b,
+	0x8d, 0xec, 0xe9, 0x09, 0x6a, 0x0c, 0xea, 0xd9, 0x1b, 0x26, 0x55, 0xcb, 0x22, 0x04, 0xb6, 0x53,
+	0xe3, 0x55, 0x2c, 0x6e, 0x18, 0xf7, 0x5b, 0x15, 0xb2, 0x0b, 0x5b, 0x53, 0x4f, 0xba, 0xbc, 0x5b,
+	0xf6, 0xc5, 0x3f, 0x35, 0x68, 0x74, 0x5d, 0xe5, 0xf6, 0xcd, 0x1b, 0x8e, 0xb8, 0xd0, 0xcc, 0x3f,
+	0x84, 0xc8, 0xc3, 0x92, 0x36, 0x53, 0xf2, 0x58, 0xeb, 0x9c, 0xac, 0xc4, 0x99, 0x1a, 0x76, 0xd6,
+	0xc8, 0x15, 0xd4, 0xb4, 0xfa, 0x48, 0x59, 0x0b, 0xcb, 0x8f, 0xaa, 0xce, 0xff, 0x8d, 0x51, 0x67,
+	0x8d, 0x0c, 0x61, 0x27, 0x1b, 0xa0, 0xa9, 0x18, 0x3e, 0x2b, 0x71, 0xb9, 0xf8, 0xe0, 0xe9, 0x3c,
+	0x5c, 0x05, 0xcb, 0x92, 0x1d, 0x40, 0x33, 0x37, 0x30, 0x64, 0x69, 0x80, 0xc5, 0x09, 0x57, 0x1a,
+	0xa0, 0x64, 0xf0, 0x38, 0x6b, 0xc4, 0x87, 0xd6, 0x15, 0xaa, 0x42, 0xcb, 0x27, 0x27, 0x2b, 0x7a,
+	0xfb, 0x74, 0xfe, 0x74, 0x4e, 0x57, 0x03, 0xb3, 0x40, 0x31, 0xec, 0x5f, 0xa1, 0x5a, 0x68, 0xae,
+	0xe4, 0x51, 0x89, 0x8f, 0x25, 0xfd, 0xbe, 0xf3, 0xf9, 0x5b, 0x60, 0xf3, 0x31, 0x5d, 0xd8, 0xcd,
+	0x62, 0x66, 0x15, 0x74, 0xb2, 0xd4, 0x49, 0xb1, 0xf7, 0x75, 0x56, 0xf7, 0x70, 0x67, 0xed, 0xf2,
+	0x9b, 0x17, 0x5f, 0xfb, 0x4c, 0x8d, 0x27, 0xc3, 0x44, 0x1e, 0xe7, 0xb7, 0x2c, 0x08, 0xd8, 0xad,
+	0xc2, 0xd1, 0xf8, 0xdc, 0xec, 0xfd, 0xc2, 0x63, 0x52, 0xc5, 0x6c, 0x38, 0x51, 0xe8, 0x9d, 0x4f,
+	0x3d, 0x9c, 0x6b, 0x87, 0xe7, 0x49, 0xe4, 0x68, 0x38, 0x5c, 0xd7, 0xab, 0xc7, 0xff, 0x05, 0x00,
+	0x00, 0xff, 0xff, 0x45, 0x70, 0x7b, 0x12, 0xcd, 0x0c, 0x00, 0x00,
 }
 
 // Reference imports to suppress errors if they are not otherwise used.
diff --git a/internal/proto/milvus.proto b/internal/proto/milvus.proto
index 6be24a638..be31d5e04 100644
--- a/internal/proto/milvus.proto
+++ b/internal/proto/milvus.proto
@@ -5,13 +5,11 @@ option go_package = "github.com/zilliztech/milvus-distributed/internal/proto/mil
 
 import "common.proto";
 import "internal.proto";
-import "schema.proto";
 
 message CreateCollectionRequest {
   internal.MsgBase base = 1;
   string db_name = 2;
   string collectionName = 3;
-  // `schema` is the serialized `schema.CollectionSchema`
   bytes schema = 4;
 }
 
@@ -28,11 +26,6 @@ message HasCollectionRequest {
   string collection_name = 3;
 }
 
-message BoolResponse {
-  common.Status status = 1;
-  bool value = 2;
-}
-
 message DescribeCollectionRequest {
   internal.MsgBase base = 1;
   string db_name = 2;
@@ -40,8 +33,7 @@ message DescribeCollectionRequest {
 }
 
 message DescribeCollectionResponse {
-  common.Status status = 1;
-  schema.CollectionSchema schema = 2;
+  repeated bytes schema = 1;
 }
 
 message LoadCollectionRequest {
@@ -64,7 +56,6 @@ message CollectionStatsRequest {
 
 message CollectionStatsResponse {
   repeated common.KeyValuePair stats = 1;
-  common.Status status = 2;
 }
 
 
@@ -75,7 +66,6 @@ message ShowCollectionRequest {
 
 message ShowCollectionResponse {
   repeated string collection_names = 1;
-  common.Status status = 2;
 }
 
 
@@ -124,7 +114,6 @@ message PartitionStatsRequest  {
 
 message PartitionStatsResponse {
   repeated common.KeyValuePair stats = 1;
-  common.Status status = 2;
 }
 
 message ShowPartitionRequest {
@@ -135,7 +124,6 @@ message ShowPartitionRequest {
 
 message ShowPartitionResponse {
   repeated  string partition_names = 1;
-  common.Status status = 2;
 }
 
 
@@ -161,7 +149,6 @@ message IndexDescription {
 
 message DescribeIndexResponse {
   repeated IndexDescription index_descriptions = 1;
-  common.Status status = 2;
 }
 
 message InsertRequest {
@@ -176,24 +163,6 @@ message InsertRequest {
 message InsertResponse {
   int64  rowID_begin = 1;
   int64  rowID_end = 2;
-  common.Status status = 3;
-}
-
-enum PlaceholderType {
-  NONE = 0;
-  VECTOR_BINARY = 100;
-  VECTOR_FLOAT = 101;
-}
-
-message PlaceholderValue {
-  string tag = 1;
-  PlaceholderType type = 2;
-  // values is a 2d-array, every array contains a vector
-  repeated bytes values = 3;
-}
-
-message PlaceholderGroup {
-  repeated PlaceholderValue placeholders = 1;
 }
 
 message SearchRequest {
@@ -202,49 +171,11 @@ message SearchRequest {
   string collection_name = 3;
   repeated  string partition_names = 4;
   string dsl = 5;
-  // serialized `PlaceholderGroup`
-  bytes placeholder_group = 6;
-}
-
-message Hits {
-  repeated int64 IDs = 1;
-  repeated bytes row_data = 2;
-  repeated float scores = 3;
-}
-
-message QueryResult {
-  common.Status status = 1;
-  repeated bytes hits = 2;
+  repeated bytes placeholder_group = 6;
 }
 
 message FlushRequest {
   internal.MsgBase base = 1;
   string db_name = 2;
   string collection_name = 3;
-}
-
-service MilvusService {
-  rpc CreateCollection(CreateCollectionRequest) returns (common.Status) {}
-  rpc DropCollection(DropCollectionRequest) returns (common.Status) {}
-  rpc HasCollection(HasCollectionRequest) returns (BoolResponse) {}
-  rpc LoadCollection(LoadCollectionRequest) returns (common.Status) {}
-  rpc ReleaseCollection(ReleaseCollectionRequest) returns (common.Status) {}
-  rpc DescribeCollection(DescribeCollectionRequest) returns (DescribeCollectionResponse) {}
-  rpc GetCollectionStatistics(CollectionStatsRequest) returns (CollectionStatsResponse) {}
-  rpc ShowCollections(ShowCollectionRequest) returns (ShowCollectionResponse) {}
-
-  rpc CreatePartition(CreatePartitionRequest) returns (common.Status) {}
-  rpc DropPartition(DropPartitionRequest) returns (common.Status) {}
-  rpc HasPartition(HasPartitionRequest) returns (BoolResponse) {}
-  rpc LoadPartitions(LoadPartitonRequest) returns (common.Status) {}
-  rpc ReleasePartitions(ReleasePartitionRequest) returns (common.Status) {}
-  rpc GetPartitionStatistics(PartitionStatsRequest) returns (PartitionStatsResponse) {}
-  rpc ShowPartitions(ShowPartitionRequest) returns (ShowPartitionResponse) {}
-
-  rpc CreateIndex(CreateIndexRequest) returns (common.Status) {}
-  rpc DescribeIndex(DescribeIndexRequest) returns (DescribeIndexResponse) {}
-
-  rpc Insert(InsertRequest) returns (InsertResponse) {}
-  rpc Search(SearchRequest) returns (QueryResult) {}
-  rpc Flush(FlushRequest) returns (common.Status) {}
-}
+}
\ No newline at end of file
diff --git a/internal/proto/milvuspb/milvus.pb.go b/internal/proto/milvuspb/milvus.pb.go
index 0ff875d43..dc330e557 100644
--- a/internal/proto/milvuspb/milvus.pb.go
+++ b/internal/proto/milvuspb/milvus.pb.go
@@ -4,15 +4,10 @@
 package milvuspb
 
 import (
-	context "context"
 	fmt "fmt"
 	proto "github.com/golang/protobuf/proto"
 	commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
 	internalpb2 "github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
-	schemapb "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
-	grpc "google.golang.org/grpc"
-	codes "google.golang.org/grpc/codes"
-	status "google.golang.org/grpc/status"
 	math "math"
 )
 
@@ -27,43 +22,14 @@ var _ = math.Inf
 // proto package needs to be updated.
 const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
 
-type PlaceholderType int32
-
-const (
-	PlaceholderType_NONE          PlaceholderType = 0
-	PlaceholderType_VECTOR_BINARY PlaceholderType = 100
-	PlaceholderType_VECTOR_FLOAT  PlaceholderType = 101
-)
-
-var PlaceholderType_name = map[int32]string{
-	0:   "NONE",
-	100: "VECTOR_BINARY",
-	101: "VECTOR_FLOAT",
-}
-
-var PlaceholderType_value = map[string]int32{
-	"NONE":          0,
-	"VECTOR_BINARY": 100,
-	"VECTOR_FLOAT":  101,
-}
-
-func (x PlaceholderType) String() string {
-	return proto.EnumName(PlaceholderType_name, int32(x))
-}
-
-func (PlaceholderType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{0}
-}
-
 type CreateCollectionRequest struct {
-	Base           *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
-	DbName         string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
-	CollectionName string               `protobuf:"bytes,3,opt,name=collectionName,proto3" json:"collectionName,omitempty"`
-	// `schema` is the serialized `schema.CollectionSchema`
-	Schema               []byte   `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Base                 *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
+	DbName               string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
+	CollectionName       string               `protobuf:"bytes,3,opt,name=collectionName,proto3" json:"collectionName,omitempty"`
+	Schema               []byte               `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
 }
 
 func (m *CreateCollectionRequest) Reset()         { *m = CreateCollectionRequest{} }
@@ -229,53 +195,6 @@ func (m *HasCollectionRequest) GetCollectionName() string {
 	return ""
 }
 
-type BoolResponse struct {
-	Status               *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
-	Value                bool             `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *BoolResponse) Reset()         { *m = BoolResponse{} }
-func (m *BoolResponse) String() string { return proto.CompactTextString(m) }
-func (*BoolResponse) ProtoMessage()    {}
-func (*BoolResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{3}
-}
-
-func (m *BoolResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_BoolResponse.Unmarshal(m, b)
-}
-func (m *BoolResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_BoolResponse.Marshal(b, m, deterministic)
-}
-func (m *BoolResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_BoolResponse.Merge(m, src)
-}
-func (m *BoolResponse) XXX_Size() int {
-	return xxx_messageInfo_BoolResponse.Size(m)
-}
-func (m *BoolResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_BoolResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_BoolResponse proto.InternalMessageInfo
-
-func (m *BoolResponse) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
-func (m *BoolResponse) GetValue() bool {
-	if m != nil {
-		return m.Value
-	}
-	return false
-}
-
 type DescribeCollectionRequest struct {
 	Base                 *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
 	DbName               string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
@@ -289,7 +208,7 @@ func (m *DescribeCollectionRequest) Reset()         { *m = DescribeCollectionReq
 func (m *DescribeCollectionRequest) String() string { return proto.CompactTextString(m) }
 func (*DescribeCollectionRequest) ProtoMessage()    {}
 func (*DescribeCollectionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{4}
+	return fileDescriptor_02345ba45cc0e303, []int{3}
 }
 
 func (m *DescribeCollectionRequest) XXX_Unmarshal(b []byte) error {
@@ -332,18 +251,17 @@ func (m *DescribeCollectionRequest) GetCollectionName() string {
 }
 
 type DescribeCollectionResponse struct {
-	Status               *commonpb.Status           `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
-	Schema               *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
+	Schema               [][]byte `protobuf:"bytes,1,rep,name=schema,proto3" json:"schema,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
 }
 
 func (m *DescribeCollectionResponse) Reset()         { *m = DescribeCollectionResponse{} }
 func (m *DescribeCollectionResponse) String() string { return proto.CompactTextString(m) }
 func (*DescribeCollectionResponse) ProtoMessage()    {}
 func (*DescribeCollectionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{5}
+	return fileDescriptor_02345ba45cc0e303, []int{4}
 }
 
 func (m *DescribeCollectionResponse) XXX_Unmarshal(b []byte) error {
@@ -364,14 +282,7 @@ func (m *DescribeCollectionResponse) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_DescribeCollectionResponse proto.InternalMessageInfo
 
-func (m *DescribeCollectionResponse) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
-func (m *DescribeCollectionResponse) GetSchema() *schemapb.CollectionSchema {
+func (m *DescribeCollectionResponse) GetSchema() [][]byte {
 	if m != nil {
 		return m.Schema
 	}
@@ -391,7 +302,7 @@ func (m *LoadCollectionRequest) Reset()         { *m = LoadCollectionRequest{} }
 func (m *LoadCollectionRequest) String() string { return proto.CompactTextString(m) }
 func (*LoadCollectionRequest) ProtoMessage()    {}
 func (*LoadCollectionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{6}
+	return fileDescriptor_02345ba45cc0e303, []int{5}
 }
 
 func (m *LoadCollectionRequest) XXX_Unmarshal(b []byte) error {
@@ -446,7 +357,7 @@ func (m *ReleaseCollectionRequest) Reset()         { *m = ReleaseCollectionReque
 func (m *ReleaseCollectionRequest) String() string { return proto.CompactTextString(m) }
 func (*ReleaseCollectionRequest) ProtoMessage()    {}
 func (*ReleaseCollectionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{7}
+	return fileDescriptor_02345ba45cc0e303, []int{6}
 }
 
 func (m *ReleaseCollectionRequest) XXX_Unmarshal(b []byte) error {
@@ -501,7 +412,7 @@ func (m *CollectionStatsRequest) Reset()         { *m = CollectionStatsRequest{}
 func (m *CollectionStatsRequest) String() string { return proto.CompactTextString(m) }
 func (*CollectionStatsRequest) ProtoMessage()    {}
 func (*CollectionStatsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{8}
+	return fileDescriptor_02345ba45cc0e303, []int{7}
 }
 
 func (m *CollectionStatsRequest) XXX_Unmarshal(b []byte) error {
@@ -545,7 +456,6 @@ func (m *CollectionStatsRequest) GetCollectionName() string {
 
 type CollectionStatsResponse struct {
 	Stats                []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=stats,proto3" json:"stats,omitempty"`
-	Status               *commonpb.Status         `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
 	XXX_unrecognized     []byte                   `json:"-"`
 	XXX_sizecache        int32                    `json:"-"`
@@ -555,7 +465,7 @@ func (m *CollectionStatsResponse) Reset()         { *m = CollectionStatsResponse
 func (m *CollectionStatsResponse) String() string { return proto.CompactTextString(m) }
 func (*CollectionStatsResponse) ProtoMessage()    {}
 func (*CollectionStatsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{9}
+	return fileDescriptor_02345ba45cc0e303, []int{8}
 }
 
 func (m *CollectionStatsResponse) XXX_Unmarshal(b []byte) error {
@@ -583,13 +493,6 @@ func (m *CollectionStatsResponse) GetStats() []*commonpb.KeyValuePair {
 	return nil
 }
 
-func (m *CollectionStatsResponse) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
 type ShowCollectionRequest struct {
 	Base                 *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
 	DbName               string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
@@ -602,7 +505,7 @@ func (m *ShowCollectionRequest) Reset()         { *m = ShowCollectionRequest{} }
 func (m *ShowCollectionRequest) String() string { return proto.CompactTextString(m) }
 func (*ShowCollectionRequest) ProtoMessage()    {}
 func (*ShowCollectionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{10}
+	return fileDescriptor_02345ba45cc0e303, []int{9}
 }
 
 func (m *ShowCollectionRequest) XXX_Unmarshal(b []byte) error {
@@ -638,18 +541,17 @@ func (m *ShowCollectionRequest) GetDbName() string {
 }
 
 type ShowCollectionResponse struct {
-	CollectionNames      []string         `protobuf:"bytes,1,rep,name=collection_names,json=collectionNames,proto3" json:"collection_names,omitempty"`
-	Status               *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
+	CollectionNames      []string `protobuf:"bytes,1,rep,name=collection_names,json=collectionNames,proto3" json:"collection_names,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
 }
 
 func (m *ShowCollectionResponse) Reset()         { *m = ShowCollectionResponse{} }
 func (m *ShowCollectionResponse) String() string { return proto.CompactTextString(m) }
 func (*ShowCollectionResponse) ProtoMessage()    {}
 func (*ShowCollectionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{11}
+	return fileDescriptor_02345ba45cc0e303, []int{10}
 }
 
 func (m *ShowCollectionResponse) XXX_Unmarshal(b []byte) error {
@@ -677,13 +579,6 @@ func (m *ShowCollectionResponse) GetCollectionNames() []string {
 	return nil
 }
 
-func (m *ShowCollectionResponse) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
 type CreatePartitionRequest struct {
 	Base                 *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
 	DbName               string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
@@ -698,7 +593,7 @@ func (m *CreatePartitionRequest) Reset()         { *m = CreatePartitionRequest{}
 func (m *CreatePartitionRequest) String() string { return proto.CompactTextString(m) }
 func (*CreatePartitionRequest) ProtoMessage()    {}
 func (*CreatePartitionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{12}
+	return fileDescriptor_02345ba45cc0e303, []int{11}
 }
 
 func (m *CreatePartitionRequest) XXX_Unmarshal(b []byte) error {
@@ -761,7 +656,7 @@ func (m *DropPartitionRequest) Reset()         { *m = DropPartitionRequest{} }
 func (m *DropPartitionRequest) String() string { return proto.CompactTextString(m) }
 func (*DropPartitionRequest) ProtoMessage()    {}
 func (*DropPartitionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{13}
+	return fileDescriptor_02345ba45cc0e303, []int{12}
 }
 
 func (m *DropPartitionRequest) XXX_Unmarshal(b []byte) error {
@@ -824,7 +719,7 @@ func (m *HasPartitionRequest) Reset()         { *m = HasPartitionRequest{} }
 func (m *HasPartitionRequest) String() string { return proto.CompactTextString(m) }
 func (*HasPartitionRequest) ProtoMessage()    {}
 func (*HasPartitionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{14}
+	return fileDescriptor_02345ba45cc0e303, []int{13}
 }
 
 func (m *HasPartitionRequest) XXX_Unmarshal(b []byte) error {
@@ -887,7 +782,7 @@ func (m *LoadPartitonRequest) Reset()         { *m = LoadPartitonRequest{} }
 func (m *LoadPartitonRequest) String() string { return proto.CompactTextString(m) }
 func (*LoadPartitonRequest) ProtoMessage()    {}
 func (*LoadPartitonRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{15}
+	return fileDescriptor_02345ba45cc0e303, []int{14}
 }
 
 func (m *LoadPartitonRequest) XXX_Unmarshal(b []byte) error {
@@ -950,7 +845,7 @@ func (m *ReleasePartitionRequest) Reset()         { *m = ReleasePartitionRequest
 func (m *ReleasePartitionRequest) String() string { return proto.CompactTextString(m) }
 func (*ReleasePartitionRequest) ProtoMessage()    {}
 func (*ReleasePartitionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{16}
+	return fileDescriptor_02345ba45cc0e303, []int{15}
 }
 
 func (m *ReleasePartitionRequest) XXX_Unmarshal(b []byte) error {
@@ -1013,7 +908,7 @@ func (m *PartitionStatsRequest) Reset()         { *m = PartitionStatsRequest{} }
 func (m *PartitionStatsRequest) String() string { return proto.CompactTextString(m) }
 func (*PartitionStatsRequest) ProtoMessage()    {}
 func (*PartitionStatsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{17}
+	return fileDescriptor_02345ba45cc0e303, []int{16}
 }
 
 func (m *PartitionStatsRequest) XXX_Unmarshal(b []byte) error {
@@ -1064,7 +959,6 @@ func (m *PartitionStatsRequest) GetPartitionName() string {
 
 type PartitionStatsResponse struct {
 	Stats                []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=stats,proto3" json:"stats,omitempty"`
-	Status               *commonpb.Status         `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
 	XXX_unrecognized     []byte                   `json:"-"`
 	XXX_sizecache        int32                    `json:"-"`
@@ -1074,7 +968,7 @@ func (m *PartitionStatsResponse) Reset()         { *m = PartitionStatsResponse{}
 func (m *PartitionStatsResponse) String() string { return proto.CompactTextString(m) }
 func (*PartitionStatsResponse) ProtoMessage()    {}
 func (*PartitionStatsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{18}
+	return fileDescriptor_02345ba45cc0e303, []int{17}
 }
 
 func (m *PartitionStatsResponse) XXX_Unmarshal(b []byte) error {
@@ -1102,13 +996,6 @@ func (m *PartitionStatsResponse) GetStats() []*commonpb.KeyValuePair {
 	return nil
 }
 
-func (m *PartitionStatsResponse) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
 type ShowPartitionRequest struct {
 	Base                 *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
 	DbName               string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
@@ -1122,7 +1009,7 @@ func (m *ShowPartitionRequest) Reset()         { *m = ShowPartitionRequest{} }
 func (m *ShowPartitionRequest) String() string { return proto.CompactTextString(m) }
 func (*ShowPartitionRequest) ProtoMessage()    {}
 func (*ShowPartitionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{19}
+	return fileDescriptor_02345ba45cc0e303, []int{18}
 }
 
 func (m *ShowPartitionRequest) XXX_Unmarshal(b []byte) error {
@@ -1165,18 +1052,17 @@ func (m *ShowPartitionRequest) GetCollectionName() string {
 }
 
 type ShowPartitionResponse struct {
-	PartitionNames       []string         `protobuf:"bytes,1,rep,name=partition_names,json=partitionNames,proto3" json:"partition_names,omitempty"`
-	Status               *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
+	PartitionNames       []string `protobuf:"bytes,1,rep,name=partition_names,json=partitionNames,proto3" json:"partition_names,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
 }
 
 func (m *ShowPartitionResponse) Reset()         { *m = ShowPartitionResponse{} }
 func (m *ShowPartitionResponse) String() string { return proto.CompactTextString(m) }
 func (*ShowPartitionResponse) ProtoMessage()    {}
 func (*ShowPartitionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{20}
+	return fileDescriptor_02345ba45cc0e303, []int{19}
 }
 
 func (m *ShowPartitionResponse) XXX_Unmarshal(b []byte) error {
@@ -1204,13 +1090,6 @@ func (m *ShowPartitionResponse) GetPartitionNames() []string {
 	return nil
 }
 
-func (m *ShowPartitionResponse) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
 type CreateIndexRequest struct {
 	Base                 *internalpb2.MsgBase     `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
 	DbName               string                   `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
@@ -1226,7 +1105,7 @@ func (m *CreateIndexRequest) Reset()         { *m = CreateIndexRequest{} }
 func (m *CreateIndexRequest) String() string { return proto.CompactTextString(m) }
 func (*CreateIndexRequest) ProtoMessage()    {}
 func (*CreateIndexRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{21}
+	return fileDescriptor_02345ba45cc0e303, []int{20}
 }
 
 func (m *CreateIndexRequest) XXX_Unmarshal(b []byte) error {
@@ -1296,7 +1175,7 @@ func (m *DescribeIndexRequest) Reset()         { *m = DescribeIndexRequest{} }
 func (m *DescribeIndexRequest) String() string { return proto.CompactTextString(m) }
 func (*DescribeIndexRequest) ProtoMessage()    {}
 func (*DescribeIndexRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{22}
+	return fileDescriptor_02345ba45cc0e303, []int{21}
 }
 
 func (m *DescribeIndexRequest) XXX_Unmarshal(b []byte) error {
@@ -1357,7 +1236,7 @@ func (m *IndexDescription) Reset()         { *m = IndexDescription{} }
 func (m *IndexDescription) String() string { return proto.CompactTextString(m) }
 func (*IndexDescription) ProtoMessage()    {}
 func (*IndexDescription) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{23}
+	return fileDescriptor_02345ba45cc0e303, []int{22}
 }
 
 func (m *IndexDescription) XXX_Unmarshal(b []byte) error {
@@ -1394,7 +1273,6 @@ func (m *IndexDescription) GetParams() []*commonpb.KeyValuePair {
 
 type DescribeIndexResponse struct {
 	IndexDescriptions    []*IndexDescription `protobuf:"bytes,1,rep,name=index_descriptions,json=indexDescriptions,proto3" json:"index_descriptions,omitempty"`
-	Status               *commonpb.Status    `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
 	XXX_unrecognized     []byte              `json:"-"`
 	XXX_sizecache        int32               `json:"-"`
@@ -1404,7 +1282,7 @@ func (m *DescribeIndexResponse) Reset()         { *m = DescribeIndexResponse{} }
 func (m *DescribeIndexResponse) String() string { return proto.CompactTextString(m) }
 func (*DescribeIndexResponse) ProtoMessage()    {}
 func (*DescribeIndexResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{24}
+	return fileDescriptor_02345ba45cc0e303, []int{23}
 }
 
 func (m *DescribeIndexResponse) XXX_Unmarshal(b []byte) error {
@@ -1432,13 +1310,6 @@ func (m *DescribeIndexResponse) GetIndexDescriptions() []*IndexDescription {
 	return nil
 }
 
-func (m *DescribeIndexResponse) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
 type InsertRequest struct {
 	Base                 *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
 	DbName               string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
@@ -1455,7 +1326,7 @@ func (m *InsertRequest) Reset()         { *m = InsertRequest{} }
 func (m *InsertRequest) String() string { return proto.CompactTextString(m) }
 func (*InsertRequest) ProtoMessage()    {}
 func (*InsertRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{25}
+	return fileDescriptor_02345ba45cc0e303, []int{24}
 }
 
 func (m *InsertRequest) XXX_Unmarshal(b []byte) error {
@@ -1519,19 +1390,18 @@ func (m *InsertRequest) GetHashKeys() []uint32 {
 }
 
 type InsertResponse struct {
-	RowIDBegin           int64            `protobuf:"varint,1,opt,name=rowID_begin,json=rowIDBegin,proto3" json:"rowID_begin,omitempty"`
-	RowIDEnd             int64            `protobuf:"varint,2,opt,name=rowID_end,json=rowIDEnd,proto3" json:"rowID_end,omitempty"`
-	Status               *commonpb.Status `protobuf:"bytes,3,opt,name=status,proto3" json:"status,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
+	RowIDBegin           int64    `protobuf:"varint,1,opt,name=rowID_begin,json=rowIDBegin,proto3" json:"rowID_begin,omitempty"`
+	RowIDEnd             int64    `protobuf:"varint,2,opt,name=rowID_end,json=rowIDEnd,proto3" json:"rowID_end,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
 }
 
 func (m *InsertResponse) Reset()         { *m = InsertResponse{} }
 func (m *InsertResponse) String() string { return proto.CompactTextString(m) }
 func (*InsertResponse) ProtoMessage()    {}
 func (*InsertResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{26}
+	return fileDescriptor_02345ba45cc0e303, []int{25}
 }
 
 func (m *InsertResponse) XXX_Unmarshal(b []byte) error {
@@ -1566,126 +1436,23 @@ func (m *InsertResponse) GetRowIDEnd() int64 {
 	return 0
 }
 
-func (m *InsertResponse) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
-type PlaceholderValue struct {
-	Tag  string          `protobuf:"bytes,1,opt,name=tag,proto3" json:"tag,omitempty"`
-	Type PlaceholderType `protobuf:"varint,2,opt,name=type,proto3,enum=milvus.proto.milvus.PlaceholderType" json:"type,omitempty"`
-	// values is a 2d-array, every array contains a vector
-	Values               [][]byte `protobuf:"bytes,3,rep,name=values,proto3" json:"values,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *PlaceholderValue) Reset()         { *m = PlaceholderValue{} }
-func (m *PlaceholderValue) String() string { return proto.CompactTextString(m) }
-func (*PlaceholderValue) ProtoMessage()    {}
-func (*PlaceholderValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{27}
-}
-
-func (m *PlaceholderValue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PlaceholderValue.Unmarshal(m, b)
-}
-func (m *PlaceholderValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PlaceholderValue.Marshal(b, m, deterministic)
-}
-func (m *PlaceholderValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PlaceholderValue.Merge(m, src)
-}
-func (m *PlaceholderValue) XXX_Size() int {
-	return xxx_messageInfo_PlaceholderValue.Size(m)
-}
-func (m *PlaceholderValue) XXX_DiscardUnknown() {
-	xxx_messageInfo_PlaceholderValue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PlaceholderValue proto.InternalMessageInfo
-
-func (m *PlaceholderValue) GetTag() string {
-	if m != nil {
-		return m.Tag
-	}
-	return ""
-}
-
-func (m *PlaceholderValue) GetType() PlaceholderType {
-	if m != nil {
-		return m.Type
-	}
-	return PlaceholderType_NONE
-}
-
-func (m *PlaceholderValue) GetValues() [][]byte {
-	if m != nil {
-		return m.Values
-	}
-	return nil
-}
-
-type PlaceholderGroup struct {
-	Placeholders         []*PlaceholderValue `protobuf:"bytes,1,rep,name=placeholders,proto3" json:"placeholders,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
-}
-
-func (m *PlaceholderGroup) Reset()         { *m = PlaceholderGroup{} }
-func (m *PlaceholderGroup) String() string { return proto.CompactTextString(m) }
-func (*PlaceholderGroup) ProtoMessage()    {}
-func (*PlaceholderGroup) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{28}
-}
-
-func (m *PlaceholderGroup) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PlaceholderGroup.Unmarshal(m, b)
-}
-func (m *PlaceholderGroup) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PlaceholderGroup.Marshal(b, m, deterministic)
-}
-func (m *PlaceholderGroup) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PlaceholderGroup.Merge(m, src)
-}
-func (m *PlaceholderGroup) XXX_Size() int {
-	return xxx_messageInfo_PlaceholderGroup.Size(m)
-}
-func (m *PlaceholderGroup) XXX_DiscardUnknown() {
-	xxx_messageInfo_PlaceholderGroup.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PlaceholderGroup proto.InternalMessageInfo
-
-func (m *PlaceholderGroup) GetPlaceholders() []*PlaceholderValue {
-	if m != nil {
-		return m.Placeholders
-	}
-	return nil
-}
-
 type SearchRequest struct {
-	Base           *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
-	DbName         string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
-	CollectionName string               `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
-	PartitionNames []string             `protobuf:"bytes,4,rep,name=partition_names,json=partitionNames,proto3" json:"partition_names,omitempty"`
-	Dsl            string               `protobuf:"bytes,5,opt,name=dsl,proto3" json:"dsl,omitempty"`
-	// serialized `PlaceholderGroup`
-	PlaceholderGroup     []byte   `protobuf:"bytes,6,opt,name=placeholder_group,json=placeholderGroup,proto3" json:"placeholder_group,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Base                 *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
+	DbName               string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
+	CollectionName       string               `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
+	PartitionNames       []string             `protobuf:"bytes,4,rep,name=partition_names,json=partitionNames,proto3" json:"partition_names,omitempty"`
+	Dsl                  string               `protobuf:"bytes,5,opt,name=dsl,proto3" json:"dsl,omitempty"`
+	PlaceholderGroup     [][]byte             `protobuf:"bytes,6,rep,name=placeholder_group,json=placeholderGroup,proto3" json:"placeholder_group,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
 }
 
 func (m *SearchRequest) Reset()         { *m = SearchRequest{} }
 func (m *SearchRequest) String() string { return proto.CompactTextString(m) }
 func (*SearchRequest) ProtoMessage()    {}
 func (*SearchRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{29}
+	return fileDescriptor_02345ba45cc0e303, []int{26}
 }
 
 func (m *SearchRequest) XXX_Unmarshal(b []byte) error {
@@ -1741,115 +1508,13 @@ func (m *SearchRequest) GetDsl() string {
 	return ""
 }
 
-func (m *SearchRequest) GetPlaceholderGroup() []byte {
+func (m *SearchRequest) GetPlaceholderGroup() [][]byte {
 	if m != nil {
 		return m.PlaceholderGroup
 	}
 	return nil
 }
 
-type Hits struct {
-	IDs                  []int64   `protobuf:"varint,1,rep,packed,name=IDs,proto3" json:"IDs,omitempty"`
-	RowData              [][]byte  `protobuf:"bytes,2,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"`
-	Scores               []float32 `protobuf:"fixed32,3,rep,packed,name=scores,proto3" json:"scores,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
-}
-
-func (m *Hits) Reset()         { *m = Hits{} }
-func (m *Hits) String() string { return proto.CompactTextString(m) }
-func (*Hits) ProtoMessage()    {}
-func (*Hits) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{30}
-}
-
-func (m *Hits) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Hits.Unmarshal(m, b)
-}
-func (m *Hits) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Hits.Marshal(b, m, deterministic)
-}
-func (m *Hits) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Hits.Merge(m, src)
-}
-func (m *Hits) XXX_Size() int {
-	return xxx_messageInfo_Hits.Size(m)
-}
-func (m *Hits) XXX_DiscardUnknown() {
-	xxx_messageInfo_Hits.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Hits proto.InternalMessageInfo
-
-func (m *Hits) GetIDs() []int64 {
-	if m != nil {
-		return m.IDs
-	}
-	return nil
-}
-
-func (m *Hits) GetRowData() [][]byte {
-	if m != nil {
-		return m.RowData
-	}
-	return nil
-}
-
-func (m *Hits) GetScores() []float32 {
-	if m != nil {
-		return m.Scores
-	}
-	return nil
-}
-
-type QueryResult struct {
-	Status               *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
-	Hits                 [][]byte         `protobuf:"bytes,2,rep,name=hits,proto3" json:"hits,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
-}
-
-func (m *QueryResult) Reset()         { *m = QueryResult{} }
-func (m *QueryResult) String() string { return proto.CompactTextString(m) }
-func (*QueryResult) ProtoMessage()    {}
-func (*QueryResult) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{31}
-}
-
-func (m *QueryResult) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_QueryResult.Unmarshal(m, b)
-}
-func (m *QueryResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_QueryResult.Marshal(b, m, deterministic)
-}
-func (m *QueryResult) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_QueryResult.Merge(m, src)
-}
-func (m *QueryResult) XXX_Size() int {
-	return xxx_messageInfo_QueryResult.Size(m)
-}
-func (m *QueryResult) XXX_DiscardUnknown() {
-	xxx_messageInfo_QueryResult.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_QueryResult proto.InternalMessageInfo
-
-func (m *QueryResult) GetStatus() *commonpb.Status {
-	if m != nil {
-		return m.Status
-	}
-	return nil
-}
-
-func (m *QueryResult) GetHits() [][]byte {
-	if m != nil {
-		return m.Hits
-	}
-	return nil
-}
-
 type FlushRequest struct {
 	Base                 *internalpb2.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
 	DbName               string               `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
@@ -1863,7 +1528,7 @@ func (m *FlushRequest) Reset()         { *m = FlushRequest{} }
 func (m *FlushRequest) String() string { return proto.CompactTextString(m) }
 func (*FlushRequest) ProtoMessage()    {}
 func (*FlushRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_02345ba45cc0e303, []int{32}
+	return fileDescriptor_02345ba45cc0e303, []int{27}
 }
 
 func (m *FlushRequest) XXX_Unmarshal(b []byte) error {
@@ -1906,11 +1571,9 @@ func (m *FlushRequest) GetCollectionName() string {
 }
 
 func init() {
-	proto.RegisterEnum("milvus.proto.milvus.PlaceholderType", PlaceholderType_name, PlaceholderType_value)
 	proto.RegisterType((*CreateCollectionRequest)(nil), "milvus.proto.milvus.CreateCollectionRequest")
 	proto.RegisterType((*DropCollectionRequest)(nil), "milvus.proto.milvus.DropCollectionRequest")
 	proto.RegisterType((*HasCollectionRequest)(nil), "milvus.proto.milvus.HasCollectionRequest")
-	proto.RegisterType((*BoolResponse)(nil), "milvus.proto.milvus.BoolResponse")
 	proto.RegisterType((*DescribeCollectionRequest)(nil), "milvus.proto.milvus.DescribeCollectionRequest")
 	proto.RegisterType((*DescribeCollectionResponse)(nil), "milvus.proto.milvus.DescribeCollectionResponse")
 	proto.RegisterType((*LoadCollectionRequest)(nil), "milvus.proto.milvus.LoadCollectionRequest")
@@ -1934,869 +1597,63 @@ func init() {
 	proto.RegisterType((*DescribeIndexResponse)(nil), "milvus.proto.milvus.DescribeIndexResponse")
 	proto.RegisterType((*InsertRequest)(nil), "milvus.proto.milvus.InsertRequest")
 	proto.RegisterType((*InsertResponse)(nil), "milvus.proto.milvus.InsertResponse")
-	proto.RegisterType((*PlaceholderValue)(nil), "milvus.proto.milvus.PlaceholderValue")
-	proto.RegisterType((*PlaceholderGroup)(nil), "milvus.proto.milvus.PlaceholderGroup")
 	proto.RegisterType((*SearchRequest)(nil), "milvus.proto.milvus.SearchRequest")
-	proto.RegisterType((*Hits)(nil), "milvus.proto.milvus.Hits")
-	proto.RegisterType((*QueryResult)(nil), "milvus.proto.milvus.QueryResult")
 	proto.RegisterType((*FlushRequest)(nil), "milvus.proto.milvus.FlushRequest")
 }
 
 func init() { proto.RegisterFile("milvus.proto", fileDescriptor_02345ba45cc0e303) }
 
 var fileDescriptor_02345ba45cc0e303 = []byte{
-	// 1409 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x58, 0xdd, 0x6f, 0x1b, 0x45,
-	0x10, 0xcf, 0xc5, 0x8e, 0x9b, 0x4c, 0xfc, 0x95, 0x6d, 0x93, 0xb8, 0xae, 0x80, 0xf4, 0xa0, 0xd4,
-	0xfd, 0x4a, 0xa4, 0x14, 0x09, 0x78, 0x40, 0x6a, 0xdd, 0xf4, 0xc3, 0x6a, 0x9b, 0x86, 0x4b, 0xa8,
-	0x14, 0x50, 0x65, 0xd6, 0xbe, 0x25, 0x3e, 0x71, 0xf6, 0x1d, 0xb7, 0xeb, 0xa4, 0xe9, 0x2b, 0x08,
-	0x5e, 0x00, 0xc1, 0xbf, 0xc0, 0x03, 0x2f, 0xbc, 0x41, 0x41, 0xfc, 0x53, 0x88, 0x3f, 0x01, 0xb4,
-	0x1f, 0xb9, 0xdc, 0x5d, 0xf6, 0x62, 0x13, 0x54, 0xd9, 0x79, 0xf3, 0x8e, 0x67, 0x67, 0x7e, 0x3b,
-	0xf3, 0xdb, 0xbd, 0x99, 0x81, 0x7c, 0xd7, 0x71, 0x77, 0xfb, 0x74, 0xd9, 0x0f, 0x3c, 0xe6, 0xa1,
-	0xb3, 0xd1, 0xd5, 0xb2, 0x5c, 0x54, 0xf3, 0x6d, 0xaf, 0xdb, 0xf5, 0x7a, 0x52, 0x58, 0x2d, 0x3a,
-	0x3d, 0x46, 0x82, 0x1e, 0x76, 0xd5, 0x3a, 0x4f, 0xdb, 0x1d, 0xd2, 0xc5, 0x72, 0x65, 0xfe, 0x6c,
-	0xc0, 0xe2, 0x9d, 0x80, 0x60, 0x46, 0xee, 0x78, 0xae, 0x4b, 0xda, 0xcc, 0xf1, 0x7a, 0x16, 0xf9,
-	0xa2, 0x4f, 0x28, 0x43, 0xab, 0x90, 0x6d, 0x61, 0x4a, 0x2a, 0xc6, 0x92, 0x51, 0x9b, 0x5d, 0x7d,
-	0x7d, 0x39, 0xe6, 0x2b, 0xb4, 0xfa, 0x98, 0xee, 0xd4, 0x31, 0x25, 0x96, 0xd0, 0x45, 0x8b, 0x70,
-	0xc6, 0x6e, 0x35, 0x7b, 0xb8, 0x4b, 0x2a, 0x93, 0x4b, 0x46, 0x6d, 0xc6, 0xca, 0xd9, 0xad, 0x75,
-	0xdc, 0x25, 0xe8, 0x6d, 0x28, 0xb6, 0x43, 0x0f, 0x5c, 0x52, 0xc9, 0x88, 0xff, 0x13, 0x52, 0xb4,
-	0x00, 0x39, 0x09, 0xb0, 0x92, 0x5d, 0x32, 0x6a, 0x79, 0x4b, 0xad, 0xcc, 0x6f, 0x0d, 0x98, 0x5f,
-	0x0b, 0x3c, 0x7f, 0x3c, 0x60, 0x72, 0x38, 0xe7, 0x1e, 0x60, 0xfa, 0x8a, 0xd1, 0x5c, 0x86, 0xd2,
-	0xa1, 0x5f, 0xa9, 0xa0, 0x87, 0xb3, 0x0d, 0xf9, 0xba, 0xe7, 0xb9, 0x16, 0xa1, 0xbe, 0xd7, 0xa3,
-	0x04, 0xdd, 0x84, 0x1c, 0x65, 0x98, 0xf5, 0xa9, 0xc2, 0x71, 0x21, 0x8e, 0x43, 0x11, 0x64, 0x53,
-	0xa8, 0x58, 0x4a, 0x15, 0x9d, 0x83, 0xa9, 0x5d, 0xec, 0xf6, 0x25, 0x88, 0x69, 0x4b, 0x2e, 0xcc,
-	0x1f, 0x0d, 0x38, 0xbf, 0x46, 0x68, 0x3b, 0x70, 0x5a, 0x64, 0x5c, 0x8e, 0xfb, 0x83, 0x01, 0x55,
-	0x1d, 0xa6, 0xff, 0x73, 0xfa, 0x0f, 0x42, 0xe2, 0x4d, 0x8a, 0x4d, 0x97, 0xe2, 0x9b, 0xd4, 0xad,
-	0x39, 0xf4, 0xb6, 0x29, 0x04, 0x21, 0x3f, 0xbf, 0x33, 0x60, 0xfe, 0x91, 0x87, 0xed, 0x31, 0x0a,
-	0x51, 0xc5, 0x22, 0x2e, 0xc1, 0x74, 0x6c, 0xb2, 0xf6, 0xbd, 0x01, 0x0b, 0x91, 0xf8, 0x31, 0xcc,
-	0xe8, 0x68, 0x01, 0x7d, 0xc3, 0x1f, 0xbf, 0x24, 0x20, 0xc5, 0xa1, 0x77, 0x61, 0x8a, 0x13, 0x83,
-	0x53, 0x28, 0x53, 0x9b, 0x5d, 0xbd, 0xa8, 0xa5, 0xd0, 0x43, 0xb2, 0xff, 0x94, 0x5f, 0x92, 0x0d,
-	0xec, 0x04, 0x96, 0xd4, 0x8f, 0x90, 0x6f, 0x72, 0x68, 0xf2, 0x99, 0x36, 0xcc, 0x6f, 0x76, 0xbc,
-	0xbd, 0x57, 0x9b, 0x29, 0xf3, 0x39, 0x2c, 0x24, 0xbd, 0xa8, 0xd3, 0x5e, 0x81, 0x72, 0x22, 0x64,
-	0xf2, 0xe0, 0x33, 0x56, 0x29, 0x1e, 0xb3, 0x13, 0x9e, 0xef, 0x77, 0x9e, 0x7a, 0xf1, 0x99, 0xd9,
-	0xc0, 0x01, 0x73, 0x46, 0xce, 0x45, 0x74, 0x09, 0x8a, 0xfe, 0x01, 0x12, 0xa9, 0x97, 0x15, 0x7a,
-	0x85, 0x50, 0x2a, 0x22, 0xf6, 0x9b, 0x01, 0xe7, 0xf8, 0x57, 0xe7, 0x74, 0xa1, 0xfe, 0xd5, 0x80,
-	0xb3, 0x0f, 0x30, 0x3d, 0x5d, 0xa0, 0x5f, 0x1a, 0x70, 0x96, 0x3f, 0xa0, 0x12, 0xf5, 0xa8, 0x41,
-	0x5f, 0x86, 0x52, 0x1c, 0x34, 0xad, 0x64, 0xc5, 0x7d, 0x28, 0xc6, 0x50, 0x53, 0xf3, 0x4f, 0x03,
-	0x16, 0xd5, 0x3b, 0x3b, 0x26, 0xf1, 0x1e, 0x1a, 0xfa, 0x4b, 0x03, 0xe6, 0x43, 0xcc, 0xa3, 0x7f,
-	0x8e, 0x87, 0x25, 0xca, 0xd7, 0x06, 0x2c, 0x24, 0x61, 0x8f, 0xe4, 0xd1, 0xe6, 0x35, 0x20, 0x7f,
-	0x4f, 0xc7, 0x23, 0xef, 0x66, 0x5f, 0x7e, 0x43, 0x22, 0x68, 0x54, 0x54, 0x34, 0x84, 0x30, 0x74,
-	0x84, 0x38, 0x59, 0x14, 0xfe, 0x36, 0x00, 0xc9, 0xa7, 0xbd, 0xd1, 0xb3, 0xc9, 0xf3, 0xd1, 0x52,
-	0xe8, 0x35, 0x80, 0xcf, 0x1c, 0xe2, 0xda, 0x51, 0xfa, 0xcc, 0x08, 0x89, 0xf8, 0x7b, 0x0d, 0xf2,
-	0xe4, 0x39, 0x0b, 0x70, 0xd3, 0xc7, 0x01, 0xee, 0xd2, 0xca, 0xd4, 0xb0, 0x34, 0x99, 0x15, 0xdb,
-	0x36, 0xc4, 0x2e, 0xf3, 0x17, 0xfe, 0x51, 0x50, 0xd5, 0xe7, 0xd8, 0x9f, 0xd9, 0x74, 0xa1, 0x2c,
-	0x40, 0x4a, 0xc4, 0x3e, 0xdf, 0xc6, 0xb7, 0x38, 0x5c, 0x26, 0xb7, 0x18, 0x72, 0x8b, 0x90, 0x08,
-	0x8b, 0xef, 0x43, 0x4e, 0x05, 0x68, 0x72, 0xd8, 0x00, 0xa9, 0x0d, 0xe6, 0x4f, 0xbc, 0x4d, 0x8b,
-	0xc7, 0x46, 0xb1, 0x70, 0x0b, 0x90, 0xf4, 0x69, 0x1f, 0x02, 0x39, 0xb8, 0xa8, 0x89, 0x5a, 0x5b,
-	0x2d, 0x92, 0xb0, 0xad, 0x39, 0x27, 0x21, 0x39, 0x21, 0x65, 0xff, 0x31, 0xa0, 0xd0, 0xe8, 0x51,
-	0x12, 0xb0, 0xd3, 0xf0, 0xe0, 0xa1, 0x77, 0x60, 0x3a, 0xf0, 0xf6, 0x9a, 0x36, 0x66, 0x58, 0x31,
-	0xf6, 0xbc, 0xf6, 0x94, 0x75, 0xd7, 0x6b, 0x59, 0x67, 0x02, 0x6f, 0x6f, 0x0d, 0x33, 0x8c, 0x2e,
-	0xc0, 0x4c, 0x07, 0xd3, 0x4e, 0xf3, 0x73, 0xb2, 0x4f, 0x2b, 0xb9, 0xa5, 0x4c, 0xad, 0x60, 0x4d,
-	0x73, 0xc1, 0x43, 0xb2, 0x4f, 0xcd, 0x2f, 0x0d, 0x28, 0x1e, 0x44, 0x40, 0xe5, 0xe7, 0x0d, 0x98,
-	0x0d, 0xbc, 0xbd, 0xc6, 0x5a, 0xb3, 0x45, 0x76, 0x9c, 0x9e, 0x88, 0x44, 0xc6, 0x02, 0x21, 0xaa,
-	0x73, 0x09, 0x37, 0x28, 0x15, 0x48, 0xcf, 0x16, 0x27, 0xce, 0x58, 0xd3, 0x42, 0x70, 0xb7, 0x67,
-	0x47, 0xf2, 0x90, 0x19, 0x3e, 0x0f, 0xbb, 0x50, 0xde, 0x70, 0x71, 0x9b, 0x74, 0x3c, 0xd7, 0x26,
-	0x81, 0x20, 0x13, 0x2a, 0x43, 0x86, 0xe1, 0x1d, 0xc5, 0x49, 0xfe, 0x13, 0xbd, 0x07, 0x59, 0xb6,
-	0xef, 0xcb, 0x20, 0x17, 0x57, 0xdf, 0xd2, 0x52, 0x25, 0x62, 0x66, 0x6b, 0xdf, 0x27, 0x96, 0xd8,
-	0x81, 0x16, 0x20, 0x27, 0x7a, 0x58, 0x0e, 0x2a, 0x53, 0xcb, 0x5b, 0x6a, 0x65, 0x3e, 0x8b, 0xf9,
-	0xbd, 0x1f, 0x78, 0x7d, 0x1f, 0x35, 0x20, 0xef, 0x1f, 0xca, 0x8e, 0x27, 0x66, 0x12, 0xb4, 0x15,
-	0xdb, 0x6a, 0xfe, 0x65, 0x40, 0x61, 0x93, 0xe0, 0xa0, 0xdd, 0x39, 0x1d, 0x85, 0x00, 0x8f, 0xb9,
-	0x4d, 0xdd, 0xca, 0x94, 0x8c, 0xb9, 0x4d, 0x5d, 0x74, 0x0d, 0xe6, 0x22, 0x47, 0x6a, 0xee, 0xf0,
-	0x10, 0x55, 0x72, 0x62, 0x20, 0x53, 0xf6, 0x13, 0xa1, 0x33, 0x1f, 0x42, 0xf6, 0x81, 0xc3, 0x84,
-	0x99, 0xc6, 0x9a, 0x8c, 0x5c, 0xc6, 0xe2, 0x3f, 0xd1, 0xf9, 0x08, 0x73, 0x27, 0x45, 0x0a, 0x42,
-	0x7a, 0x8a, 0x39, 0x8f, 0x17, 0xa8, 0xdc, 0x4c, 0x5a, 0x6a, 0x65, 0x3e, 0x85, 0xd9, 0x0f, 0xfb,
-	0x24, 0xd8, 0xb7, 0x08, 0xed, 0xbb, 0xec, 0x64, 0xad, 0x3c, 0x82, 0x6c, 0xc7, 0x61, 0x54, 0xb9,
-	0x14, 0xbf, 0xcd, 0xaf, 0x0c, 0xc8, 0xdf, 0x73, 0xfb, 0x74, 0xb4, 0x39, 0xb9, 0x7a, 0x0b, 0x4a,
-	0x09, 0xae, 0xa2, 0x69, 0xc8, 0xae, 0x3f, 0x59, 0xbf, 0x5b, 0x9e, 0x40, 0x73, 0x50, 0x78, 0x7a,
-	0xf7, 0xce, 0xd6, 0x13, 0xab, 0x59, 0x6f, 0xac, 0xdf, 0xb6, 0xb6, 0xcb, 0x36, 0x2a, 0x43, 0x5e,
-	0x89, 0xee, 0x3d, 0x7a, 0x72, 0x7b, 0xab, 0x4c, 0x56, 0xff, 0x28, 0x41, 0xe1, 0xb1, 0xc0, 0xba,
-	0x49, 0x82, 0x5d, 0xa7, 0x4d, 0x50, 0x13, 0xca, 0xc9, 0x11, 0x1e, 0xba, 0xae, 0x25, 0x6e, 0xca,
-	0xa4, 0xaf, 0x7a, 0x5c, 0x54, 0xcd, 0x09, 0xf4, 0x09, 0x14, 0xe3, 0xa3, 0x37, 0x74, 0x55, 0x6b,
-	0x5e, 0x3b, 0x9f, 0x1b, 0x64, 0xbc, 0x09, 0x85, 0xd8, 0x20, 0x0d, 0x5d, 0xd1, 0xda, 0xd6, 0x0d,
-	0xdb, 0xaa, 0x17, 0xb5, 0xaa, 0xd1, 0x49, 0x98, 0x44, 0x1f, 0x1f, 0xcc, 0xa4, 0xa0, 0xd7, 0x4e,
-	0x6f, 0x06, 0xa1, 0xc7, 0x30, 0x77, 0x64, 0xca, 0x82, 0x6e, 0x68, 0xed, 0xa7, 0x4d, 0x63, 0x06,
-	0xb9, 0xd8, 0x03, 0x74, 0x74, 0xd6, 0x85, 0x96, 0xf5, 0x19, 0x48, 0x1b, 0xd4, 0x55, 0x57, 0x86,
-	0xd6, 0x0f, 0x03, 0xb7, 0x0b, 0x8b, 0xf7, 0x09, 0x8b, 0x0f, 0x48, 0x1c, 0xca, 0x9c, 0x36, 0x45,
-	0xd7, 0xf4, 0xf4, 0xd2, 0x0e, 0x77, 0xaa, 0xd7, 0x87, 0x53, 0x0e, 0xfd, 0xba, 0x50, 0x8a, 0x8f,
-	0x29, 0x68, 0x4a, 0xc6, 0xb4, 0x23, 0x93, 0xea, 0xb5, 0xa1, 0x74, 0x43, 0x6f, 0xcf, 0xa0, 0x94,
-	0x98, 0x4c, 0xa4, 0x9d, 0x4e, 0x3b, 0xbf, 0x18, 0x94, 0xbd, 0x6d, 0x28, 0xc4, 0x06, 0x08, 0x29,
-	0xf4, 0xd6, 0x0d, 0x19, 0x06, 0x99, 0x7e, 0x06, 0xf9, 0x68, 0x97, 0x8f, 0x6a, 0x69, 0x17, 0xe7,
-	0x88, 0xe1, 0xa1, 0xee, 0xcd, 0xb6, 0xbc, 0x37, 0xe1, 0x66, 0x9a, 0xe2, 0x40, 0xd3, 0xb4, 0x0f,
-	0x42, 0xfe, 0x69, 0x78, 0x6b, 0x22, 0xd6, 0xaf, 0x1f, 0x77, 0x6b, 0xfe, 0x6b, 0x6c, 0x28, 0x2c,
-	0xdc, 0x27, 0x2c, 0xd6, 0x26, 0x2a, 0xea, 0xea, 0xa9, 0xa4, 0xed, 0x83, 0x53, 0xa8, 0xa4, 0x6f,
-	0x3e, 0xcd, 0x09, 0xe4, 0x40, 0x31, 0xd6, 0x81, 0xd1, 0x94, 0x64, 0xeb, 0x9a, 0xc6, 0xea, 0xd5,
-	0x61, 0x54, 0x43, 0x57, 0x1f, 0xc1, 0x6c, 0xa4, 0xe9, 0x42, 0x97, 0x8f, 0x61, 0x6c, 0xb4, 0x45,
-	0x19, 0x14, 0xb6, 0x0e, 0x14, 0x62, 0xd5, 0x7b, 0x1a, 0x5b, 0x35, 0xdd, 0x4f, 0xca, 0x01, 0xb4,
-	0xcd, 0x80, 0x39, 0x81, 0x36, 0x21, 0x27, 0x0b, 0x50, 0x64, 0xa6, 0x14, 0xff, 0x91, 0xfa, 0xbc,
-	0xfa, 0xe6, 0xb1, 0x3a, 0xa1, 0xd1, 0x0d, 0xc8, 0xc9, 0xc2, 0x2b, 0xc5, 0x68, 0xac, 0x2a, 0xab,
-	0x2e, 0x69, 0x75, 0x22, 0xd5, 0x87, 0x39, 0x81, 0x1a, 0x30, 0x25, 0xaa, 0x06, 0xa4, 0xbf, 0x32,
-	0xd1, 0x8a, 0x62, 0x40, 0x6c, 0xeb, 0xf5, 0x8f, 0x6f, 0xed, 0x38, 0xac, 0xd3, 0x6f, 0xf1, 0x7f,
-	0x56, 0x5e, 0x38, 0xae, 0xeb, 0xbc, 0x60, 0xa4, 0xdd, 0x59, 0x91, 0xbb, 0x6e, 0xd8, 0x0e, 0x65,
-	0x81, 0xd3, 0xea, 0x33, 0x62, 0xaf, 0x1c, 0xd4, 0x1f, 0x2b, 0xc2, 0x94, 0xd2, 0xf0, 0x5b, 0xad,
-	0x9c, 0x58, 0xdf, 0xfc, 0x37, 0x00, 0x00, 0xff, 0xff, 0x10, 0x02, 0x02, 0xfd, 0x06, 0x1c, 0x00,
-	0x00,
-}
-
-// Reference imports to suppress errors if they are not otherwise used.
-var _ context.Context
-var _ grpc.ClientConn
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the grpc package it is being compiled against.
-const _ = grpc.SupportPackageIsVersion4
-
-// MilvusServiceClient is the client API for MilvusService service.
-//
-// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
-type MilvusServiceClient interface {
-	CreateCollection(ctx context.Context, in *CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	DropCollection(ctx context.Context, in *DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	HasCollection(ctx context.Context, in *HasCollectionRequest, opts ...grpc.CallOption) (*BoolResponse, error)
-	LoadCollection(ctx context.Context, in *LoadCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	ReleaseCollection(ctx context.Context, in *ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	DescribeCollection(ctx context.Context, in *DescribeCollectionRequest, opts ...grpc.CallOption) (*DescribeCollectionResponse, error)
-	GetCollectionStatistics(ctx context.Context, in *CollectionStatsRequest, opts ...grpc.CallOption) (*CollectionStatsResponse, error)
-	ShowCollections(ctx context.Context, in *ShowCollectionRequest, opts ...grpc.CallOption) (*ShowCollectionResponse, error)
-	CreatePartition(ctx context.Context, in *CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	DropPartition(ctx context.Context, in *DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	HasPartition(ctx context.Context, in *HasPartitionRequest, opts ...grpc.CallOption) (*BoolResponse, error)
-	LoadPartitions(ctx context.Context, in *LoadPartitonRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	ReleasePartitions(ctx context.Context, in *ReleasePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	GetPartitionStatistics(ctx context.Context, in *PartitionStatsRequest, opts ...grpc.CallOption) (*PartitionStatsResponse, error)
-	ShowPartitions(ctx context.Context, in *ShowPartitionRequest, opts ...grpc.CallOption) (*ShowPartitionResponse, error)
-	CreateIndex(ctx context.Context, in *CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-	DescribeIndex(ctx context.Context, in *DescribeIndexRequest, opts ...grpc.CallOption) (*DescribeIndexResponse, error)
-	Insert(ctx context.Context, in *InsertRequest, opts ...grpc.CallOption) (*InsertResponse, error)
-	Search(ctx context.Context, in *SearchRequest, opts ...grpc.CallOption) (*QueryResult, error)
-	Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
-}
-
-type milvusServiceClient struct {
-	cc *grpc.ClientConn
-}
-
-func NewMilvusServiceClient(cc *grpc.ClientConn) MilvusServiceClient {
-	return &milvusServiceClient{cc}
-}
-
-func (c *milvusServiceClient) CreateCollection(ctx context.Context, in *CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/CreateCollection", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) DropCollection(ctx context.Context, in *DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/DropCollection", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) HasCollection(ctx context.Context, in *HasCollectionRequest, opts ...grpc.CallOption) (*BoolResponse, error) {
-	out := new(BoolResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/HasCollection", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) LoadCollection(ctx context.Context, in *LoadCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/LoadCollection", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) ReleaseCollection(ctx context.Context, in *ReleaseCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/ReleaseCollection", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) DescribeCollection(ctx context.Context, in *DescribeCollectionRequest, opts ...grpc.CallOption) (*DescribeCollectionResponse, error) {
-	out := new(DescribeCollectionResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/DescribeCollection", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) GetCollectionStatistics(ctx context.Context, in *CollectionStatsRequest, opts ...grpc.CallOption) (*CollectionStatsResponse, error) {
-	out := new(CollectionStatsResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/GetCollectionStatistics", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) ShowCollections(ctx context.Context, in *ShowCollectionRequest, opts ...grpc.CallOption) (*ShowCollectionResponse, error) {
-	out := new(ShowCollectionResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/ShowCollections", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) CreatePartition(ctx context.Context, in *CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/CreatePartition", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) DropPartition(ctx context.Context, in *DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/DropPartition", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) HasPartition(ctx context.Context, in *HasPartitionRequest, opts ...grpc.CallOption) (*BoolResponse, error) {
-	out := new(BoolResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/HasPartition", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) LoadPartitions(ctx context.Context, in *LoadPartitonRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/LoadPartitions", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) ReleasePartitions(ctx context.Context, in *ReleasePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/ReleasePartitions", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) GetPartitionStatistics(ctx context.Context, in *PartitionStatsRequest, opts ...grpc.CallOption) (*PartitionStatsResponse, error) {
-	out := new(PartitionStatsResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/GetPartitionStatistics", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) ShowPartitions(ctx context.Context, in *ShowPartitionRequest, opts ...grpc.CallOption) (*ShowPartitionResponse, error) {
-	out := new(ShowPartitionResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/ShowPartitions", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) CreateIndex(ctx context.Context, in *CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/CreateIndex", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) DescribeIndex(ctx context.Context, in *DescribeIndexRequest, opts ...grpc.CallOption) (*DescribeIndexResponse, error) {
-	out := new(DescribeIndexResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/DescribeIndex", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) Insert(ctx context.Context, in *InsertRequest, opts ...grpc.CallOption) (*InsertResponse, error) {
-	out := new(InsertResponse)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/Insert", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) Search(ctx context.Context, in *SearchRequest, opts ...grpc.CallOption) (*QueryResult, error) {
-	out := new(QueryResult)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/Search", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-func (c *milvusServiceClient) Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
-	out := new(commonpb.Status)
-	err := c.cc.Invoke(ctx, "/milvus.proto.milvus.MilvusService/Flush", in, out, opts...)
-	if err != nil {
-		return nil, err
-	}
-	return out, nil
-}
-
-// MilvusServiceServer is the server API for MilvusService service.
-type MilvusServiceServer interface {
-	CreateCollection(context.Context, *CreateCollectionRequest) (*commonpb.Status, error)
-	DropCollection(context.Context, *DropCollectionRequest) (*commonpb.Status, error)
-	HasCollection(context.Context, *HasCollectionRequest) (*BoolResponse, error)
-	LoadCollection(context.Context, *LoadCollectionRequest) (*commonpb.Status, error)
-	ReleaseCollection(context.Context, *ReleaseCollectionRequest) (*commonpb.Status, error)
-	DescribeCollection(context.Context, *DescribeCollectionRequest) (*DescribeCollectionResponse, error)
-	GetCollectionStatistics(context.Context, *CollectionStatsRequest) (*CollectionStatsResponse, error)
-	ShowCollections(context.Context, *ShowCollectionRequest) (*ShowCollectionResponse, error)
-	CreatePartition(context.Context, *CreatePartitionRequest) (*commonpb.Status, error)
-	DropPartition(context.Context, *DropPartitionRequest) (*commonpb.Status, error)
-	HasPartition(context.Context, *HasPartitionRequest) (*BoolResponse, error)
-	LoadPartitions(context.Context, *LoadPartitonRequest) (*commonpb.Status, error)
-	ReleasePartitions(context.Context, *ReleasePartitionRequest) (*commonpb.Status, error)
-	GetPartitionStatistics(context.Context, *PartitionStatsRequest) (*PartitionStatsResponse, error)
-	ShowPartitions(context.Context, *ShowPartitionRequest) (*ShowPartitionResponse, error)
-	CreateIndex(context.Context, *CreateIndexRequest) (*commonpb.Status, error)
-	DescribeIndex(context.Context, *DescribeIndexRequest) (*DescribeIndexResponse, error)
-	Insert(context.Context, *InsertRequest) (*InsertResponse, error)
-	Search(context.Context, *SearchRequest) (*QueryResult, error)
-	Flush(context.Context, *FlushRequest) (*commonpb.Status, error)
-}
-
-// UnimplementedMilvusServiceServer can be embedded to have forward compatible implementations.
-type UnimplementedMilvusServiceServer struct {
-}
-
-func (*UnimplementedMilvusServiceServer) CreateCollection(ctx context.Context, req *CreateCollectionRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CreateCollection not implemented")
-}
-func (*UnimplementedMilvusServiceServer) DropCollection(ctx context.Context, req *DropCollectionRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DropCollection not implemented")
-}
-func (*UnimplementedMilvusServiceServer) HasCollection(ctx context.Context, req *HasCollectionRequest) (*BoolResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method HasCollection not implemented")
-}
-func (*UnimplementedMilvusServiceServer) LoadCollection(ctx context.Context, req *LoadCollectionRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method LoadCollection not implemented")
-}
-func (*UnimplementedMilvusServiceServer) ReleaseCollection(ctx context.Context, req *ReleaseCollectionRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ReleaseCollection not implemented")
-}
-func (*UnimplementedMilvusServiceServer) DescribeCollection(ctx context.Context, req *DescribeCollectionRequest) (*DescribeCollectionResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DescribeCollection not implemented")
-}
-func (*UnimplementedMilvusServiceServer) GetCollectionStatistics(ctx context.Context, req *CollectionStatsRequest) (*CollectionStatsResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetCollectionStatistics not implemented")
-}
-func (*UnimplementedMilvusServiceServer) ShowCollections(ctx context.Context, req *ShowCollectionRequest) (*ShowCollectionResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented")
-}
-func (*UnimplementedMilvusServiceServer) CreatePartition(ctx context.Context, req *CreatePartitionRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CreatePartition not implemented")
-}
-func (*UnimplementedMilvusServiceServer) DropPartition(ctx context.Context, req *DropPartitionRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DropPartition not implemented")
-}
-func (*UnimplementedMilvusServiceServer) HasPartition(ctx context.Context, req *HasPartitionRequest) (*BoolResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method HasPartition not implemented")
-}
-func (*UnimplementedMilvusServiceServer) LoadPartitions(ctx context.Context, req *LoadPartitonRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method LoadPartitions not implemented")
-}
-func (*UnimplementedMilvusServiceServer) ReleasePartitions(ctx context.Context, req *ReleasePartitionRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ReleasePartitions not implemented")
-}
-func (*UnimplementedMilvusServiceServer) GetPartitionStatistics(ctx context.Context, req *PartitionStatsRequest) (*PartitionStatsResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method GetPartitionStatistics not implemented")
-}
-func (*UnimplementedMilvusServiceServer) ShowPartitions(ctx context.Context, req *ShowPartitionRequest) (*ShowPartitionResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
-}
-func (*UnimplementedMilvusServiceServer) CreateIndex(ctx context.Context, req *CreateIndexRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method CreateIndex not implemented")
-}
-func (*UnimplementedMilvusServiceServer) DescribeIndex(ctx context.Context, req *DescribeIndexRequest) (*DescribeIndexResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method DescribeIndex not implemented")
-}
-func (*UnimplementedMilvusServiceServer) Insert(ctx context.Context, req *InsertRequest) (*InsertResponse, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method Insert not implemented")
-}
-func (*UnimplementedMilvusServiceServer) Search(ctx context.Context, req *SearchRequest) (*QueryResult, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method Search not implemented")
-}
-func (*UnimplementedMilvusServiceServer) Flush(ctx context.Context, req *FlushRequest) (*commonpb.Status, error) {
-	return nil, status.Errorf(codes.Unimplemented, "method Flush not implemented")
-}
-
-func RegisterMilvusServiceServer(s *grpc.Server, srv MilvusServiceServer) {
-	s.RegisterService(&_MilvusService_serviceDesc, srv)
-}
-
-func _MilvusService_CreateCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(CreateCollectionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).CreateCollection(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/CreateCollection",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).CreateCollection(ctx, req.(*CreateCollectionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_DropCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DropCollectionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).DropCollection(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/DropCollection",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).DropCollection(ctx, req.(*DropCollectionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_HasCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(HasCollectionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).HasCollection(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/HasCollection",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).HasCollection(ctx, req.(*HasCollectionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_LoadCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LoadCollectionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).LoadCollection(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/LoadCollection",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).LoadCollection(ctx, req.(*LoadCollectionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_ReleaseCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ReleaseCollectionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).ReleaseCollection(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/ReleaseCollection",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).ReleaseCollection(ctx, req.(*ReleaseCollectionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_DescribeCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DescribeCollectionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).DescribeCollection(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/DescribeCollection",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).DescribeCollection(ctx, req.(*DescribeCollectionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_GetCollectionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(CollectionStatsRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).GetCollectionStatistics(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/GetCollectionStatistics",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).GetCollectionStatistics(ctx, req.(*CollectionStatsRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_ShowCollections_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ShowCollectionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).ShowCollections(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/ShowCollections",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).ShowCollections(ctx, req.(*ShowCollectionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_CreatePartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(CreatePartitionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).CreatePartition(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/CreatePartition",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).CreatePartition(ctx, req.(*CreatePartitionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_DropPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DropPartitionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).DropPartition(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/DropPartition",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).DropPartition(ctx, req.(*DropPartitionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_HasPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(HasPartitionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).HasPartition(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/HasPartition",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).HasPartition(ctx, req.(*HasPartitionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_LoadPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(LoadPartitonRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).LoadPartitions(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/LoadPartitions",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).LoadPartitions(ctx, req.(*LoadPartitonRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_ReleasePartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ReleasePartitionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).ReleasePartitions(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/ReleasePartitions",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).ReleasePartitions(ctx, req.(*ReleasePartitionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_GetPartitionStatistics_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(PartitionStatsRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).GetPartitionStatistics(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/GetPartitionStatistics",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).GetPartitionStatistics(ctx, req.(*PartitionStatsRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(ShowPartitionRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).ShowPartitions(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/ShowPartitions",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).ShowPartitions(ctx, req.(*ShowPartitionRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_CreateIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(CreateIndexRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).CreateIndex(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/CreateIndex",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).CreateIndex(ctx, req.(*CreateIndexRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_DescribeIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(DescribeIndexRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).DescribeIndex(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/DescribeIndex",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).DescribeIndex(ctx, req.(*DescribeIndexRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_Insert_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(InsertRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).Insert(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/Insert",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).Insert(ctx, req.(*InsertRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_Search_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(SearchRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).Search(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/Search",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).Search(ctx, req.(*SearchRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-func _MilvusService_Flush_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
-	in := new(FlushRequest)
-	if err := dec(in); err != nil {
-		return nil, err
-	}
-	if interceptor == nil {
-		return srv.(MilvusServiceServer).Flush(ctx, in)
-	}
-	info := &grpc.UnaryServerInfo{
-		Server:     srv,
-		FullMethod: "/milvus.proto.milvus.MilvusService/Flush",
-	}
-	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
-		return srv.(MilvusServiceServer).Flush(ctx, req.(*FlushRequest))
-	}
-	return interceptor(ctx, in, info, handler)
-}
-
-var _MilvusService_serviceDesc = grpc.ServiceDesc{
-	ServiceName: "milvus.proto.milvus.MilvusService",
-	HandlerType: (*MilvusServiceServer)(nil),
-	Methods: []grpc.MethodDesc{
-		{
-			MethodName: "CreateCollection",
-			Handler:    _MilvusService_CreateCollection_Handler,
-		},
-		{
-			MethodName: "DropCollection",
-			Handler:    _MilvusService_DropCollection_Handler,
-		},
-		{
-			MethodName: "HasCollection",
-			Handler:    _MilvusService_HasCollection_Handler,
-		},
-		{
-			MethodName: "LoadCollection",
-			Handler:    _MilvusService_LoadCollection_Handler,
-		},
-		{
-			MethodName: "ReleaseCollection",
-			Handler:    _MilvusService_ReleaseCollection_Handler,
-		},
-		{
-			MethodName: "DescribeCollection",
-			Handler:    _MilvusService_DescribeCollection_Handler,
-		},
-		{
-			MethodName: "GetCollectionStatistics",
-			Handler:    _MilvusService_GetCollectionStatistics_Handler,
-		},
-		{
-			MethodName: "ShowCollections",
-			Handler:    _MilvusService_ShowCollections_Handler,
-		},
-		{
-			MethodName: "CreatePartition",
-			Handler:    _MilvusService_CreatePartition_Handler,
-		},
-		{
-			MethodName: "DropPartition",
-			Handler:    _MilvusService_DropPartition_Handler,
-		},
-		{
-			MethodName: "HasPartition",
-			Handler:    _MilvusService_HasPartition_Handler,
-		},
-		{
-			MethodName: "LoadPartitions",
-			Handler:    _MilvusService_LoadPartitions_Handler,
-		},
-		{
-			MethodName: "ReleasePartitions",
-			Handler:    _MilvusService_ReleasePartitions_Handler,
-		},
-		{
-			MethodName: "GetPartitionStatistics",
-			Handler:    _MilvusService_GetPartitionStatistics_Handler,
-		},
-		{
-			MethodName: "ShowPartitions",
-			Handler:    _MilvusService_ShowPartitions_Handler,
-		},
-		{
-			MethodName: "CreateIndex",
-			Handler:    _MilvusService_CreateIndex_Handler,
-		},
-		{
-			MethodName: "DescribeIndex",
-			Handler:    _MilvusService_DescribeIndex_Handler,
-		},
-		{
-			MethodName: "Insert",
-			Handler:    _MilvusService_Insert_Handler,
-		},
-		{
-			MethodName: "Search",
-			Handler:    _MilvusService_Search_Handler,
-		},
-		{
-			MethodName: "Flush",
-			Handler:    _MilvusService_Flush_Handler,
-		},
-	},
-	Streams:  []grpc.StreamDesc{},
-	Metadata: "milvus.proto",
+	// 809 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x97, 0xdf, 0x6f, 0x2a, 0x45,
+	0x14, 0xc7, 0x33, 0x85, 0xd2, 0x72, 0xf8, 0x51, 0xba, 0x2d, 0xb0, 0xad, 0x51, 0x71, 0x93, 0x5a,
+	0x8c, 0x11, 0x92, 0xda, 0xc4, 0xf8, 0xd6, 0x50, 0xd4, 0x36, 0xd5, 0xa6, 0x6e, 0x8d, 0x0f, 0xbe,
+	0x90, 0xd9, 0xdd, 0x91, 0xdd, 0xb8, 0xbb, 0xb3, 0xce, 0x0c, 0xd2, 0xf6, 0xd9, 0x47, 0x35, 0xfa,
+	0x4f, 0xf8, 0xe2, 0x9b, 0xd6, 0xc4, 0x7f, 0xca, 0xdc, 0x3f, 0xe1, 0xde, 0xcc, 0xcc, 0x42, 0x81,
+	0xcb, 0x03, 0xc9, 0x4d, 0x03, 0xbc, 0x31, 0xdf, 0x3d, 0x67, 0xe6, 0x73, 0xce, 0x19, 0xbe, 0x2c,
+	0x50, 0x8c, 0x82, 0xf0, 0xa7, 0x01, 0x6f, 0x25, 0x8c, 0x0a, 0x6a, 0xec, 0x4d, 0xae, 0x5a, 0x7a,
+	0x71, 0x58, 0x74, 0x69, 0x14, 0xd1, 0x58, 0x8b, 0x87, 0xe5, 0x20, 0x16, 0x84, 0xc5, 0x38, 0xd4,
+	0x6b, 0xeb, 0x4f, 0x04, 0xf5, 0x73, 0x46, 0xb0, 0x20, 0xe7, 0x34, 0x0c, 0x89, 0x2b, 0x02, 0x1a,
+	0xdb, 0xe4, 0xc7, 0x01, 0xe1, 0xc2, 0x38, 0x81, 0xac, 0x83, 0x39, 0x31, 0x51, 0x03, 0x35, 0x0b,
+	0x27, 0xef, 0xb4, 0xa6, 0x76, 0x1f, 0xef, 0xf3, 0x15, 0xef, 0x77, 0x30, 0x27, 0xb6, 0x8a, 0x35,
+	0xea, 0xb0, 0xe5, 0x39, 0xbd, 0x18, 0x47, 0xc4, 0xdc, 0x68, 0xa0, 0x66, 0xde, 0xce, 0x79, 0xce,
+	0x35, 0x8e, 0x88, 0xf1, 0x3e, 0x94, 0xdd, 0xf1, 0x09, 0x52, 0x31, 0x33, 0xea, 0xf9, 0x8c, 0x6a,
+	0xd4, 0x20, 0xc7, 0x5d, 0x9f, 0x44, 0xd8, 0xcc, 0x36, 0x50, 0xb3, 0x68, 0xa7, 0x2b, 0xeb, 0x17,
+	0x04, 0xd5, 0x2e, 0xa3, 0xc9, 0x6a, 0x60, 0x4a, 0x9c, 0xfd, 0x0b, 0xcc, 0x9f, 0x99, 0xe6, 0x18,
+	0x76, 0x9e, 0xce, 0xd5, 0x01, 0xf3, 0x71, 0xfe, 0x40, 0x70, 0xd0, 0x25, 0xdc, 0x65, 0x81, 0x43,
+	0x56, 0x85, 0xe9, 0x14, 0x0e, 0xe7, 0x21, 0xf1, 0x84, 0xc6, 0x7c, 0x72, 0xce, 0xa8, 0x91, 0x99,
+	0x98, 0xf3, 0xaf, 0x08, 0xaa, 0x5f, 0x52, 0xec, 0xad, 0x4a, 0x15, 0xbf, 0x23, 0x30, 0x6d, 0x12,
+	0x12, 0xcc, 0x57, 0xa6, 0xb1, 0xbf, 0x21, 0xa8, 0x3d, 0xb1, 0xdc, 0x0a, 0x2c, 0xf8, 0x72, 0x81,
+	0x6c, 0xa8, 0xbf, 0xc6, 0x93, 0x8e, 0xf9, 0x13, 0xd8, 0xe4, 0x52, 0x50, 0x53, 0x2e, 0x9c, 0xbc,
+	0x37, 0x4d, 0x94, 0x5a, 0xd3, 0x15, 0xb9, 0xff, 0x16, 0x87, 0x03, 0x72, 0x83, 0x03, 0x66, 0xeb,
+	0x78, 0xcb, 0x83, 0xea, 0xad, 0x4f, 0x87, 0xcf, 0xdb, 0x73, 0xeb, 0x1c, 0x6a, 0xb3, 0xa7, 0xa4,
+	0xe0, 0x1f, 0x40, 0x65, 0xa6, 0x78, 0x5d, 0x43, 0xde, 0xde, 0x99, 0xae, 0x9e, 0x5b, 0xff, 0xca,
+	0x79, 0x28, 0x0f, 0xbd, 0xc1, 0x4c, 0x04, 0x4b, 0xbf, 0x20, 0xc6, 0x11, 0x94, 0x93, 0x11, 0x89,
+	0x8e, 0xcb, 0xaa, 0xb8, 0xd2, 0x58, 0x55, 0xc5, 0xff, 0x83, 0x60, 0x5f, 0x5a, 0xea, 0x7a, 0x51,
+	0xff, 0x8d, 0x60, 0xef, 0x02, 0xf3, 0xf5, 0x82, 0x7e, 0x44, 0xb0, 0x27, 0x5d, 0x4d, 0x53, 0x2f,
+	0x1b, 0xfa, 0x18, 0x76, 0xa6, 0xa1, 0xb9, 0x99, 0x55, 0x57, 0xbb, 0x3c, 0x45, 0xcd, 0xad, 0xff,
+	0x10, 0xd4, 0x53, 0xf3, 0x5b, 0x91, 0x7e, 0x2f, 0x8c, 0xfe, 0x88, 0xa0, 0x3a, 0x66, 0x5e, 0xbe,
+	0x47, 0x2e, 0x7a, 0x51, 0xbe, 0x86, 0xda, 0x2c, 0xf5, 0x9b, 0x3a, 0xa9, 0x7c, 0x55, 0x91, 0x26,
+	0xb7, 0x1a, 0x13, 0xb4, 0xce, 0xb4, 0xb1, 0x4f, 0xd0, 0xa4, 0x05, 0xce, 0x19, 0x2d, 0x9a, 0x3b,
+	0xda, 0x17, 0x08, 0x0c, 0xed, 0xb7, 0x97, 0xb1, 0x47, 0xee, 0x96, 0x3b, 0xd7, 0xb7, 0x01, 0xbe,
+	0x0f, 0x48, 0xe8, 0x4d, 0xce, 0x34, 0xaf, 0x14, 0xf5, 0xb8, 0x0b, 0x45, 0x72, 0x27, 0x18, 0xee,
+	0x25, 0x98, 0xe1, 0x88, 0x9b, 0x9b, 0x8b, 0x0e, 0xaf, 0xa0, 0xd2, 0x6e, 0x54, 0x96, 0xf5, 0x97,
+	0x74, 0xea, 0xf4, 0x5d, 0x6a, 0xe5, 0x6b, 0xb6, 0x42, 0xa8, 0x28, 0x48, 0x4d, 0x9c, 0xc8, 0x34,
+	0x99, 0x12, 0x48, 0x4d, 0xa7, 0x20, 0x9d, 0xa2, 0x14, 0xb5, 0xe3, 0xa7, 0x90, 0x4b, 0x1b, 0xb4,
+	0xb1, 0x68, 0x83, 0xd2, 0x04, 0x2b, 0x82, 0xea, 0x4c, 0x6b, 0xd2, 0xfb, 0xf4, 0x0d, 0x18, 0xfa,
+	0x48, 0xef, 0x89, 0x63, 0xf4, 0xed, 0x39, 0x6a, 0xcd, 0xf9, 0xab, 0xd4, 0x9a, 0xa5, 0xb6, 0x77,
+	0x83, 0x19, 0x85, 0x5b, 0x2f, 0x11, 0x94, 0x2e, 0x63, 0x4e, 0x98, 0x58, 0x07, 0x3f, 0x31, 0x4e,
+	0x61, 0x9b, 0xd1, 0x61, 0xcf, 0xc3, 0x02, 0xa7, 0x77, 0xef, 0x60, 0x6e, 0x6b, 0x3b, 0x21, 0x75,
+	0xec, 0x2d, 0x46, 0x87, 0x5d, 0x2c, 0xb0, 0xf1, 0x16, 0xe4, 0x7d, 0xcc, 0xfd, 0xde, 0x0f, 0xe4,
+	0x9e, 0x9b, 0xb9, 0x46, 0xa6, 0x59, 0xb2, 0xb7, 0xa5, 0x70, 0x45, 0xee, 0xb9, 0x75, 0x0d, 0xe5,
+	0x51, 0x03, 0xd2, 0x4e, 0xbf, 0x0b, 0x05, 0x46, 0x87, 0x97, 0xdd, 0x9e, 0x43, 0xfa, 0x41, 0xac,
+	0x1a, 0x91, 0xb1, 0x41, 0x49, 0x1d, 0xa9, 0xc8, 0xfd, 0x74, 0x00, 0x89, 0x3d, 0x55, 0x70, 0xc6,
+	0xde, 0x56, 0xc2, 0x67, 0xb1, 0x67, 0xfd, 0x8f, 0xa0, 0x74, 0x4b, 0x30, 0x73, 0xfd, 0xf5, 0xf8,
+	0x69, 0x31, 0x2a, 0x90, 0xf1, 0x78, 0x68, 0x6e, 0xaa, 0x5d, 0xe4, 0x47, 0xe3, 0x43, 0xd8, 0x4d,
+	0x42, 0xec, 0x12, 0x9f, 0x86, 0x1e, 0x61, 0xbd, 0x3e, 0xa3, 0x83, 0x44, 0xf5, 0xad, 0x68, 0x57,
+	0x26, 0x1e, 0x7c, 0x21, 0x75, 0xeb, 0x67, 0x04, 0xc5, 0xcf, 0xc3, 0x01, 0x5f, 0x6e, 0xb9, 0x9d,
+	0xce, 0x77, 0x67, 0xfd, 0x40, 0xf8, 0x03, 0x47, 0x5e, 0x81, 0xf6, 0x43, 0x10, 0x86, 0xc1, 0x83,
+	0x20, 0xae, 0xdf, 0xd6, 0xc7, 0x7f, 0xe4, 0x05, 0x5c, 0xb0, 0xc0, 0x19, 0x08, 0xe2, 0xb5, 0x47,
+	0x10, 0x6d, 0xc5, 0x94, 0x46, 0x24, 0x8e, 0x93, 0x53, 0xeb, 0x8f, 0x5f, 0x05, 0x00, 0x00, 0xff,
+	0xff, 0x3b, 0xfd, 0x8e, 0x53, 0x87, 0x10, 0x00, 0x00,
 }
diff --git a/internal/proxynode/proxy.go b/internal/proxynode/proxy.go
index c6026d884..fd91c38e0 100644
--- a/internal/proxynode/proxy.go
+++ b/internal/proxynode/proxy.go
@@ -12,7 +12,6 @@ import (
 	"time"
 
 	"github.com/opentracing/opentracing-go"
-	"github.com/uber/jaeger-client-go"
 	"github.com/uber/jaeger-client-go/config"
 
 	"google.golang.org/grpc"
@@ -72,11 +71,8 @@ func CreateProxy(ctx context.Context) (*Proxy, error) {
 			Type:  "const",
 			Param: 1,
 		},
-		Reporter: &config.ReporterConfig{
-			LogSpans: true,
-		},
 	}
-	p.tracer, p.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger))
+	p.tracer, p.closer, err = cfg.NewTracer()
 	if err != nil {
 		panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
 	}
diff --git a/internal/proxyservice/interface.go b/internal/proxyservice/interface.go
deleted file mode 100644
index b57290924..000000000
--- a/internal/proxyservice/interface.go
+++ /dev/null
@@ -1,16 +0,0 @@
-package proxyservice
-
-import (
-	"github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
-	"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
-)
-
-type ServiceBase = typeutil.Service
-
-type Interface interface {
-	ServiceBase
-	RegisterLink() (proxypb.RegisterLinkResponse, error)
-	RegisterNode(request proxypb.RegisterNodeRequest) (proxypb.RegisterNodeResponse, error)
-	// TODO: i'm sure it's not a best way to keep consistency, fix me
-	InvalidateCollectionMetaCache(request proxypb.InvalidateCollMetaCacheRequest) error
-}
diff --git a/internal/proxyservice/proxyservice.go b/internal/proxyservice/proxyservice.go
deleted file mode 100644
index 99b3be15a..000000000
--- a/internal/proxyservice/proxyservice.go
+++ /dev/null
@@ -1,54 +0,0 @@
-package proxyservice
-
-import (
-	"github.com/zilliztech/milvus-distributed/internal/proto/internalpb2"
-	"github.com/zilliztech/milvus-distributed/internal/proto/proxypb"
-)
-
-type ProxyService struct {
-	// implement Service
-
-	//nodeClients [] .Interface
-	// factory method
-
-}
-
-func (s ProxyService) Init() {
-	panic("implement me")
-}
-
-func (s ProxyService) Start() {
-	panic("implement me")
-}
-
-func (s ProxyService) Stop() {
-	panic("implement me")
-}
-
-func (s ProxyService) GetServiceStates() (internalpb2.ServiceStates, error) {
-	panic("implement me")
-}
-
-func (s ProxyService) GetTimeTickChannel() (string, error) {
-	panic("implement me")
-}
-
-func (s ProxyService) GetStatisticsChannel() (string, error) {
-	panic("implement me")
-}
-
-func (s ProxyService) RegisterLink() (proxypb.RegisterLinkResponse, error) {
-	panic("implement me")
-}
-
-func (s ProxyService) RegisterNode(request proxypb.RegisterNodeRequest) (proxypb.RegisterNodeResponse, error) {
-	panic("implement me")
-}
-
-func (s ProxyService) InvalidateCollectionMetaCache(request proxypb.InvalidateCollMetaCacheRequest) error {
-	panic("implement me")
-}
-
-func NewProxyServiceImpl() Interface {
-	return &ProxyService{}
-}
diff --git a/internal/querynode/query_node.go b/internal/querynode/query_node.go
index 38ad79bee..fa49db959 100644
--- a/internal/querynode/query_node.go
+++ b/internal/querynode/query_node.go
@@ -18,7 +18,6 @@ import (
 	"io"
 
 	"github.com/opentracing/opentracing-go"
-	"github.com/uber/jaeger-client-go"
 	"github.com/uber/jaeger-client-go/config"
 	"google.golang.org/grpc"
 
@@ -89,11 +88,8 @@ func newQueryNode(ctx context.Context, queryNodeID uint64) *QueryNode {
 			Type:  "const",
 			Param: 1,
 		},
-		Reporter: &config.ReporterConfig{
-			LogSpans: true,
-		},
 	}
-	q.tracer, q.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger))
+	q.tracer, q.closer, err = cfg.NewTracer()
 	if err != nil {
 		panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
 	}
diff --git a/internal/timesync/time_sync_producer.go b/internal/timesync/time_sync_producer.go
new file mode 100644
index 000000000..fdb0f20bf
--- /dev/null
+++ b/internal/timesync/time_sync_producer.go
@@ -0,0 +1,141 @@
+package timesync
+
+import (
+	"context"
+	"log"
+
+	"github.com/zilliztech/milvus-distributed/internal/errors"
+	ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
+	internalPb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
+)
+
+type timeSyncMsgProducer struct {
+	//softTimeTickBarrier
+	proxyTtBarrier TimeTickBarrier
+	//hardTimeTickBarrier
+	writeNodeTtBarrier TimeTickBarrier
+
+	ddSyncStream  ms.MsgStream // insert & delete
+	dmSyncStream  ms.MsgStream
+	k2sSyncStream ms.MsgStream
+
+	ctx    context.Context
+	cancel context.CancelFunc
+
+	proxyWatchers     []TimeTickWatcher
+	writeNodeWatchers []TimeTickWatcher
+}
+
+func NewTimeSyncMsgProducer(ctx context.Context) (*timeSyncMsgProducer, error) {
+	ctx2, cancel := context.WithCancel(ctx)
+	return &timeSyncMsgProducer{ctx: ctx2, cancel: cancel}, nil
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) SetProxyTtBarrier(proxyTtBarrier TimeTickBarrier) {
+	syncMsgProducer.proxyTtBarrier = proxyTtBarrier
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) SetWriteNodeTtBarrier(writeNodeTtBarrier TimeTickBarrier) {
+	syncMsgProducer.writeNodeTtBarrier = writeNodeTtBarrier
+}
+func (syncMsgProducer *timeSyncMsgProducer) SetDDSyncStream(ddSync ms.MsgStream) {
+	syncMsgProducer.ddSyncStream = ddSync
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) SetDMSyncStream(dmSync ms.MsgStream) {
+	syncMsgProducer.dmSyncStream = dmSync
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) SetK2sSyncStream(k2sSync ms.MsgStream) {
+	syncMsgProducer.k2sSyncStream = k2sSync
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) WatchProxyTtBarrier(watcher TimeTickWatcher) {
+	syncMsgProducer.proxyWatchers = append(syncMsgProducer.proxyWatchers, watcher)
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) WatchWriteNodeTtBarrier(watcher TimeTickWatcher) {
+	syncMsgProducer.writeNodeWatchers = append(syncMsgProducer.writeNodeWatchers, watcher)
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) broadcastMsg(barrier TimeTickBarrier, streams []ms.MsgStream, watchers []TimeTickWatcher) error {
+	for {
+		select {
+		case <-syncMsgProducer.ctx.Done():
+			{
+				log.Printf("broadcast context done, exit")
+				return errors.Errorf("broadcast done exit")
+			}
+		default:
+			timetick, err := barrier.GetTimeTick()
+			if err != nil {
+				log.Printf("broadcast get time tick error")
+			}
+			msgPack := ms.MsgPack{}
+			baseMsg := ms.BaseMsg{
+				BeginTimestamp: timetick,
+				EndTimestamp:   timetick,
+				HashValues:     []uint32{0},
+			}
+			timeTickResult := internalPb.TimeTickMsg{
+				MsgType:   internalPb.MsgType_kTimeTick,
+				PeerID:    0,
+				Timestamp: timetick,
+			}
+			timeTickMsg := &ms.TimeTickMsg{
+				BaseMsg:     baseMsg,
+				TimeTickMsg: timeTickResult,
+			}
+			msgPack.Msgs = append(msgPack.Msgs, timeTickMsg)
+			for _, stream := range streams {
+				err = stream.Broadcast(&msgPack)
+			}
+
+			for _, watcher := range watchers {
+				watcher.Watch(timeTickMsg)
+			}
+			if err != nil {
+				return err
+			}
+		}
+	}
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) Start() error {
+	err := syncMsgProducer.proxyTtBarrier.Start()
+	if err != nil {
+		return err
+	}
+
+	err = syncMsgProducer.writeNodeTtBarrier.Start()
+	if err != nil {
+		return err
+	}
+
+	for _, watcher := range syncMsgProducer.proxyWatchers {
+		watcher.Start()
+	}
+	for _, watcher := range syncMsgProducer.writeNodeWatchers {
+		watcher.Start()
+	}
+
+	go syncMsgProducer.broadcastMsg(syncMsgProducer.proxyTtBarrier, []ms.MsgStream{syncMsgProducer.dmSyncStream, syncMsgProducer.ddSyncStream}, syncMsgProducer.proxyWatchers)
+	go syncMsgProducer.broadcastMsg(syncMsgProducer.writeNodeTtBarrier, []ms.MsgStream{syncMsgProducer.k2sSyncStream}, syncMsgProducer.writeNodeWatchers)
+
+	return nil
+}
+
+func (syncMsgProducer *timeSyncMsgProducer) Close() {
+	syncMsgProducer.ddSyncStream.Close()
+	syncMsgProducer.dmSyncStream.Close()
+	syncMsgProducer.k2sSyncStream.Close()
+	syncMsgProducer.cancel()
+	syncMsgProducer.proxyTtBarrier.Close()
+	syncMsgProducer.writeNodeTtBarrier.Close()
+	for _, watcher := range syncMsgProducer.proxyWatchers {
+		watcher.Close()
+	}
+	for _, watcher := range syncMsgProducer.writeNodeWatchers {
+		watcher.Close()
+	}
+}
diff --git a/internal/timesync/timesync.go b/internal/timesync/timesync.go
new file mode 100644
index 000000000..5cf8a50a9
--- /dev/null
+++ b/internal/timesync/timesync.go
@@ -0,0 +1,235 @@
+package timesync
+
+import (
+	"context"
+	"log"
+	"math"
+	"sync/atomic"
+
+	"github.com/zilliztech/milvus-distributed/internal/util/typeutil"
+
+	"github.com/zilliztech/milvus-distributed/internal/errors"
+	ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
+)
+
+type (
+	Timestamp = typeutil.Timestamp
+	UniqueID  = typeutil.UniqueID
+
+	TimeTickBarrier interface {
+		GetTimeTick() (Timestamp, error)
+		Start() error
+		Close()
+	}
+
+	softTimeTickBarrier struct {
+		peer2LastTt   map[UniqueID]Timestamp
+		minTtInterval Timestamp
+		lastTt        int64
+		outTt         chan Timestamp
+		ttStream      ms.MsgStream
+		ctx           context.Context
+		cancel        context.CancelFunc
+	}
+
+	hardTimeTickBarrier struct {
+		peer2Tt  map[UniqueID]Timestamp
+		outTt    chan Timestamp
+		ttStream ms.MsgStream
+		ctx      context.Context
+		cancel   context.CancelFunc
+	}
+)
+
+func (ttBarrier *softTimeTickBarrier) GetTimeTick() (Timestamp, error) {
+	select {
+	case <-ttBarrier.ctx.Done():
+		return 0, errors.Errorf("[GetTimeTick] closed.")
+	case ts, ok := <-ttBarrier.outTt:
+		if !ok {
+			return 0, errors.Errorf("[GetTimeTick] closed.")
+		}
+		num := len(ttBarrier.outTt)
+		for i := 0; i < num; i++ {
+			ts, ok = <-ttBarrier.outTt
+			if !ok {
+				return 0, errors.Errorf("[GetTimeTick] closed.")
+			}
+		}
+		atomic.StoreInt64(&(ttBarrier.lastTt), int64(ts))
+		return ts, ttBarrier.ctx.Err()
+	}
+}
+
+func (ttBarrier *softTimeTickBarrier) Start() error {
+	go func() {
+		for {
+			select {
+			case <-ttBarrier.ctx.Done():
+				log.Printf("[TtBarrierStart] %s\n", ttBarrier.ctx.Err())
+				return
+
+			case ttmsgs := <-ttBarrier.ttStream.Chan():
+				if len(ttmsgs.Msgs) > 0 {
+					for _, timetickmsg := range ttmsgs.Msgs {
+						ttmsg := timetickmsg.(*ms.TimeTickMsg)
+						oldT, ok := ttBarrier.peer2LastTt[ttmsg.PeerID]
+						// log.Printf("[softTimeTickBarrier] peer(%d)=%d\n", ttmsg.PeerID, ttmsg.Timestamp)
+
+						if !ok {
+							log.Printf("[softTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.PeerID)
+							continue
+						}
+						if ttmsg.Timestamp > oldT {
+							ttBarrier.peer2LastTt[ttmsg.PeerID] = ttmsg.Timestamp
+
+							// get a legal Timestamp
+							ts := ttBarrier.minTimestamp()
+							lastTt := atomic.LoadInt64(&(ttBarrier.lastTt))
+							if lastTt != 0 && ttBarrier.minTtInterval > ts-Timestamp(lastTt) {
+								continue
+							}
+							ttBarrier.outTt <- ts
+						}
+					}
+				}
+			}
+		}
+	}()
+	return nil
+}
+
+func newSoftTimeTickBarrier(ctx context.Context,
+	ttStream *ms.MsgStream,
+	peerIds []UniqueID,
+	minTtInterval Timestamp) *softTimeTickBarrier {
+
+	if len(peerIds) <= 0 {
+		log.Printf("[newSoftTimeTickBarrier] Error: peerIds is empty!\n")
+		return nil
+	}
+
+	sttbarrier := softTimeTickBarrier{}
+	sttbarrier.minTtInterval = minTtInterval
+	sttbarrier.ttStream = *ttStream
+	sttbarrier.outTt = make(chan Timestamp, 1024)
+	sttbarrier.ctx, sttbarrier.cancel = context.WithCancel(ctx)
+	sttbarrier.peer2LastTt = make(map[UniqueID]Timestamp)
+	for _, id := range peerIds {
+		sttbarrier.peer2LastTt[id] = Timestamp(0)
+	}
+	if len(peerIds) != len(sttbarrier.peer2LastTt) {
+		log.Printf("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!\n")
+	}
+
+	return &sttbarrier
+}
+
+func (ttBarrier *softTimeTickBarrier) Close() {
+	ttBarrier.cancel()
+}
+
+func (ttBarrier *softTimeTickBarrier) minTimestamp() Timestamp {
+	tempMin := Timestamp(math.MaxUint64)
+	for _, tt := range ttBarrier.peer2LastTt {
+		if tt < tempMin {
+			tempMin = tt
+		}
+	}
+	return tempMin
+}
+
+func (ttBarrier *hardTimeTickBarrier) GetTimeTick() (Timestamp, error) {
+	select {
+	case <-ttBarrier.ctx.Done():
+		return 0, errors.Errorf("[GetTimeTick] closed.")
+	case ts, ok := <-ttBarrier.outTt:
+		if !ok {
+			return 0, errors.Errorf("[GetTimeTick] closed.")
+		}
+		return ts, ttBarrier.ctx.Err()
+	}
+}
+
+func (ttBarrier *hardTimeTickBarrier) Start() error {
+	go func() {
+		// Last timestamp synchronized
+		state := Timestamp(0)
+		for {
+			select {
+			case <-ttBarrier.ctx.Done():
+				log.Printf("[TtBarrierStart] %s\n", ttBarrier.ctx.Err())
+				return
+
+			case ttmsgs := <-ttBarrier.ttStream.Chan():
+				if len(ttmsgs.Msgs) > 0 {
+					for _, timetickmsg := range ttmsgs.Msgs {
+
+						// Suppose ttmsg.Timestamp from stream is always larger than the previous one,
+						// that `ttmsg.Timestamp > oldT`
+						ttmsg := timetickmsg.(*ms.TimeTickMsg)
+
+						oldT, ok := ttBarrier.peer2Tt[ttmsg.PeerID]
+						if !ok {
+							log.Printf("[hardTimeTickBarrier] Warning: peerID %d not exist\n", ttmsg.PeerID)
+							continue
+						}
+
+						if oldT > state {
+							log.Printf("[hardTimeTickBarrier] Warning: peer(%d) timestamp(%d) ahead\n",
+								ttmsg.PeerID, ttmsg.Timestamp)
+						}
+
+						ttBarrier.peer2Tt[ttmsg.PeerID] = ttmsg.Timestamp
+
+						newState := ttBarrier.minTimestamp()
+						if newState > state {
+							ttBarrier.outTt <- newState
+							state = newState
+						}
+					}
+				}
+			}
+		}
+	}()
+	return nil
+}
+
+func (ttBarrier *hardTimeTickBarrier) minTimestamp() Timestamp {
+	tempMin := Timestamp(math.MaxUint64)
+	for _, tt := range ttBarrier.peer2Tt {
+		if tt < tempMin {
+			tempMin = tt
+		}
+	}
+	return tempMin
+}
+
+func newHardTimeTickBarrier(ctx context.Context,
+	ttStream *ms.MsgStream,
+	peerIds []UniqueID) *hardTimeTickBarrier {
+
+	if len(peerIds) <= 0 {
+		log.Printf("[newSoftTimeTickBarrier] Error: peerIds is empty!")
+		return nil
+	}
+
+	sttbarrier := hardTimeTickBarrier{}
+	sttbarrier.ttStream = *ttStream
+	sttbarrier.outTt = make(chan Timestamp, 1024)
+	sttbarrier.ctx, sttbarrier.cancel = context.WithCancel(ctx)
+
+	sttbarrier.peer2Tt = make(map[UniqueID]Timestamp)
+	for _, id := range peerIds {
+		sttbarrier.peer2Tt[id] = Timestamp(0)
+	}
+	if len(peerIds) != len(sttbarrier.peer2Tt) {
+		log.Printf("[newSoftTimeTickBarrier] Warning: there are duplicate peerIds!")
+	}
+
+	return &sttbarrier
+}
+
+func (ttBarrier *hardTimeTickBarrier) Close() {
+	ttBarrier.cancel()
+}
diff --git a/internal/timesync/timetick_watcher.go b/internal/timesync/timetick_watcher.go
new file mode 100644
index 000000000..27eb9e416
--- /dev/null
+++ b/internal/timesync/timetick_watcher.go
@@ -0,0 +1,11 @@
+package timesync
+
+import (
+	ms "github.com/zilliztech/milvus-distributed/internal/msgstream"
+)
+
+type TimeTickWatcher interface {
+	Watch(msg *ms.TimeTickMsg)
+	Start()
+	Close()
+}
diff --git a/internal/writenode/write_node.go b/internal/writenode/write_node.go
index 5199148c9..280a3298c 100644
--- a/internal/writenode/write_node.go
+++ b/internal/writenode/write_node.go
@@ -6,7 +6,6 @@ import (
 	"io"
 
 	"github.com/opentracing/opentracing-go"
-	"github.com/uber/jaeger-client-go"
 	"github.com/uber/jaeger-client-go/config"
 )
 
@@ -52,12 +51,9 @@ func (node *WriteNode) Start() error {
 			Type:  "const",
 			Param: 1,
 		},
-		Reporter: &config.ReporterConfig{
-			LogSpans: true,
-		},
 	}
 	var err error
-	node.tracer, node.closer, err = cfg.NewTracer(config.Logger(jaeger.StdLogger))
+	node.tracer, node.closer, err = cfg.NewTracer()
 	if err != nil {
 		panic(fmt.Sprintf("ERROR: cannot init Jaeger: %v\n", err))
 	}
-- 
GitLab