mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-08 10:08:42 +08:00
Complete Flush procedure in DataNode (#5454)
* Complete Flush proceduer in DataNode Resolves: #5220 Signed-off-by: yangxuan <xuan.yang@zilliz.com> * fix unittest Signed-off-by: yangxuan <xuan.yang@zilliz.com>
This commit is contained in:
parent
59a3d83f97
commit
55eac3ee7f
@ -274,6 +274,7 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||||||
|
|
||||||
for _, id := range req.SegmentIDs {
|
for _, id := range req.SegmentIDs {
|
||||||
chanName := node.getChannelName(id)
|
chanName := node.getChannelName(id)
|
||||||
|
log.Info("vchannel", zap.String("name", chanName))
|
||||||
if chanName == "" {
|
if chanName == "" {
|
||||||
status.Reason = fmt.Sprintf("DataNode not find segment %d!", id)
|
status.Reason = fmt.Sprintf("DataNode not find segment %d!", id)
|
||||||
return status, errors.New(status.GetReason())
|
return status, errors.New(status.GetReason())
|
||||||
@ -296,23 +297,9 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||||||
dmlFlushedCh: dmlFlushedCh,
|
dmlFlushedCh: dmlFlushedCh,
|
||||||
}
|
}
|
||||||
|
|
||||||
flushCh <- flushmsg
|
|
||||||
|
|
||||||
// GOOSE TODO get binlog paths.
|
|
||||||
// waitReceive := func(wg *sync.WaitGroup, flushedCh <-chan bool, req *datapb.SaveBinlogPathsRequest) {
|
|
||||||
// defer wg.Done()
|
|
||||||
// select {
|
|
||||||
// case <-time.After(300 * time.Second):
|
|
||||||
// return
|
|
||||||
// case isFlushed := <-flushedCh:
|
|
||||||
// if isFlushed {
|
|
||||||
// log.Debug("Yeah! It's safe to notify dataservice")
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
// }
|
|
||||||
|
|
||||||
waitReceive := func(wg *sync.WaitGroup, flushedCh interface{}, req *datapb.SaveBinlogPathsRequest) {
|
waitReceive := func(wg *sync.WaitGroup, flushedCh interface{}, req *datapb.SaveBinlogPathsRequest) {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
|
log.Info("Inside waitReceive")
|
||||||
switch Ch := flushedCh.(type) {
|
switch Ch := flushedCh.(type) {
|
||||||
case chan []*datapb.ID2PathList:
|
case chan []*datapb.ID2PathList:
|
||||||
select {
|
select {
|
||||||
@ -342,7 +329,7 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||||||
|
|
||||||
if len(meta) == 0 {
|
if len(meta) == 0 {
|
||||||
log.Info("Ddl messages flush Done")
|
log.Info("Ddl messages flush Done")
|
||||||
// Modify req with empty ddl binlog paths
|
// Modify req with empty ddl binlog paths and position
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -357,6 +344,8 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||||||
// TODO make a queue for this func
|
// TODO make a queue for this func
|
||||||
currentSegID := id
|
currentSegID := id
|
||||||
go func() {
|
go func() {
|
||||||
|
flushCh <- flushmsg
|
||||||
|
|
||||||
log.Info("Waiting for flush completed", zap.Int64("segmentID", currentSegID))
|
log.Info("Waiting for flush completed", zap.Int64("segmentID", currentSegID))
|
||||||
req := &datapb.SaveBinlogPathsRequest{
|
req := &datapb.SaveBinlogPathsRequest{
|
||||||
Base: &commonpb.MsgBase{},
|
Base: &commonpb.MsgBase{},
|
||||||
@ -371,8 +360,20 @@ func (node *DataNode) FlushSegments(ctx context.Context, req *datapb.FlushSegmen
|
|||||||
go waitReceive(&wg, dmlFlushedCh, req)
|
go waitReceive(&wg, dmlFlushedCh, req)
|
||||||
wg.Wait()
|
wg.Wait()
|
||||||
|
|
||||||
// TODO
|
status, err := node.dataService.SaveBinlogPaths(node.ctx, req)
|
||||||
//status := node.dataService.SaveBinlogPaths(req)
|
if err != nil {
|
||||||
|
log.Error("DataNode or DataService abnormal, restarting DataNode")
|
||||||
|
// TODO restart
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
|
if status.ErrorCode != commonpb.ErrorCode_Success {
|
||||||
|
log.Error("Save paths failed, resending request",
|
||||||
|
zap.String("error message", status.GetReason()))
|
||||||
|
// TODO resend
|
||||||
|
return
|
||||||
|
}
|
||||||
|
|
||||||
log.Info("Flush Completed", zap.Int64("segmentID", currentSegID))
|
log.Info("Flush Completed", zap.Int64("segmentID", currentSegID))
|
||||||
}()
|
}()
|
||||||
|
|
||||||
|
|||||||
@ -126,14 +126,15 @@ func TestDataNode(t *testing.T) {
|
|||||||
sync, ok := node1.vchan2SyncService[dmChannelName]
|
sync, ok := node1.vchan2SyncService[dmChannelName]
|
||||||
assert.True(t, ok)
|
assert.True(t, ok)
|
||||||
sync.replica.addSegment(0, 1, 1, dmChannelName)
|
sync.replica.addSegment(0, 1, 1, dmChannelName)
|
||||||
sync.replica.addSegment(1, 1, 1, dmChannelName)
|
// sync.replica.addSegment(1, 1, 1, dmChannelName) unable to deal with this.
|
||||||
|
|
||||||
req := &datapb.FlushSegmentsRequest{
|
req := &datapb.FlushSegmentsRequest{
|
||||||
Base: &commonpb.MsgBase{},
|
Base: &commonpb.MsgBase{},
|
||||||
DbID: 0,
|
DbID: 0,
|
||||||
CollectionID: 1,
|
CollectionID: 1,
|
||||||
SegmentIDs: []int64{0, 1},
|
SegmentIDs: []int64{0},
|
||||||
}
|
}
|
||||||
|
|
||||||
status, err := node1.FlushSegments(node.ctx, req)
|
status, err := node1.FlushSegments(node.ctx, req)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
assert.Equal(t, commonpb.ErrorCode_Success, status.ErrorCode)
|
||||||
@ -182,8 +183,13 @@ func TestDataNode(t *testing.T) {
|
|||||||
err = ddMsgStream.Broadcast(&timeTickMsgPack)
|
err = ddMsgStream.Broadcast(&timeTickMsgPack)
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
|
|
||||||
<-node1.ctx.Done()
|
_, err = sync.replica.getSegmentByID(0)
|
||||||
|
assert.NoError(t, err)
|
||||||
|
|
||||||
|
defer func() {
|
||||||
|
node1.ctx.Done()
|
||||||
node1.Stop()
|
node1.Stop()
|
||||||
|
}()
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("Test GetTimeTickChannel", func(t *testing.T) {
|
t.Run("Test GetTimeTickChannel", func(t *testing.T) {
|
||||||
@ -196,6 +202,6 @@ func TestDataNode(t *testing.T) {
|
|||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
<-node.ctx.Done()
|
// <-node.ctx.Done()
|
||||||
node.Stop()
|
node.Stop()
|
||||||
}
|
}
|
||||||
|
|||||||
@ -57,15 +57,6 @@ func newDataSyncService(ctx context.Context,
|
|||||||
return service
|
return service
|
||||||
}
|
}
|
||||||
|
|
||||||
// func (dsService *dataSyncService) init() {
|
|
||||||
// if len(Params.InsertChannelNames) == 0 {
|
|
||||||
// log.Error("InsertChannels not readly, init datasync service failed")
|
|
||||||
// return
|
|
||||||
// }
|
|
||||||
|
|
||||||
// dsService.initNodes()
|
|
||||||
// }
|
|
||||||
|
|
||||||
func (dsService *dataSyncService) start() {
|
func (dsService *dataSyncService) start() {
|
||||||
log.Debug("Data Sync Service Start Successfully")
|
log.Debug("Data Sync Service Start Successfully")
|
||||||
if dsService.fg != nil {
|
if dsService.fg != nil {
|
||||||
|
|||||||
@ -182,16 +182,15 @@ func (ddNode *ddNode) flushComplete(binlogMetaCh <-chan *datapb.DDLBinlogMeta, c
|
|||||||
ddlFlushedCh <- nil
|
ddlFlushedCh <- nil
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
//
|
||||||
|
// log.Debug(".. Saving ddl binlog meta ...")
|
||||||
|
// err := ddNode.binlogMeta.SaveDDLBinlogMetaTxn(collID, binlogMeta)
|
||||||
|
// if err != nil {
|
||||||
|
// log.Error("Save binlog meta to etcd Wrong", zap.Error(err))
|
||||||
|
// }
|
||||||
|
|
||||||
log.Debug(".. Saving ddl binlog meta ...")
|
|
||||||
err := ddNode.binlogMeta.SaveDDLBinlogMetaTxn(collID, binlogMeta)
|
|
||||||
if err != nil {
|
|
||||||
log.Error("Save binlog meta to etcd Wrong", zap.Error(err))
|
|
||||||
}
|
|
||||||
|
|
||||||
ddlFlushedCh <- []*datapb.DDLBinlogMeta{binlogMeta}
|
|
||||||
// TODO remove above
|
// TODO remove above
|
||||||
// ddlFlushCh <- binlogMetaCh
|
ddlFlushedCh <- []*datapb.DDLBinlogMeta{binlogMeta}
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|||||||
@ -16,10 +16,13 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/stretchr/testify/require"
|
"github.com/stretchr/testify/assert"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/msgstream"
|
"github.com/milvus-io/milvus/internal/msgstream"
|
||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||||
)
|
)
|
||||||
@ -41,10 +44,10 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
|
|||||||
inFlushCh := make(chan *flushMsg, 10)
|
inFlushCh := make(chan *flushMsg, 10)
|
||||||
defer close(inFlushCh)
|
defer close(inFlushCh)
|
||||||
|
|
||||||
testPath := "/test/datanode/root/meta"
|
// testPath := "/test/datanode/root/meta"
|
||||||
err := clearEtcd(testPath)
|
// err := clearEtcd(testPath)
|
||||||
require.NoError(t, err)
|
// require.NoError(t, err)
|
||||||
Params.MetaRootPath = testPath
|
// Params.MetaRootPath = testPath
|
||||||
|
|
||||||
// Params.FlushDdBufferSize = 4
|
// Params.FlushDdBufferSize = 4
|
||||||
replica := newReplica()
|
replica := newReplica()
|
||||||
@ -148,11 +151,13 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
replica.addSegment(1, collID, partitionID, "insert-01")
|
replica.addSegment(1, collID, partitionID, "insert-01")
|
||||||
|
flushCh := make(chan []*datapb.DDLBinlogMeta)
|
||||||
inFlushCh <- &flushMsg{
|
inFlushCh <- &flushMsg{
|
||||||
msgID: 5,
|
msgID: 5,
|
||||||
timestamp: 5,
|
timestamp: 5,
|
||||||
segmentID: UniqueID(1),
|
segmentID: UniqueID(1),
|
||||||
collectionID: collID,
|
collectionID: collID,
|
||||||
|
ddlFlushedCh: flushCh,
|
||||||
}
|
}
|
||||||
|
|
||||||
startPos := []*internalpb.MsgPosition{
|
startPos := []*internalpb.MsgPosition{
|
||||||
@ -172,4 +177,8 @@ func TestFlowGraphDDNode_Operate(t *testing.T) {
|
|||||||
startPos, startPos)
|
startPos, startPos)
|
||||||
var inMsg Msg = msgStream
|
var inMsg Msg = msgStream
|
||||||
ddNode.Operate([]Msg{inMsg})
|
ddNode.Operate([]Msg{inMsg})
|
||||||
|
|
||||||
|
paths := <-flushCh
|
||||||
|
log.Debug("Flushed DDL binlog paths", zap.Any("paths", paths))
|
||||||
|
assert.Equal(t, 1, len(paths))
|
||||||
}
|
}
|
||||||
|
|||||||
@ -709,13 +709,13 @@ func (ibNode *insertBufferNode) completeFlush(segID UniqueID, wait <-chan map[Un
|
|||||||
}
|
}
|
||||||
|
|
||||||
// GOOSE TODO remove the below
|
// GOOSE TODO remove the below
|
||||||
log.Debug(".. Saving binlog paths to etcd ..", zap.Int("number of fields", len(field2Path)))
|
// log.Debug(".. Saving binlog paths to etcd ..", zap.Int("number of fields", len(field2Path)))
|
||||||
err = ibNode.flushMeta.SaveSegmentBinlogMetaTxn(segID, bufferField2Paths)
|
// err = ibNode.flushMeta.SaveSegmentBinlogMetaTxn(segID, bufferField2Paths)
|
||||||
if err != nil {
|
// if err != nil {
|
||||||
log.Error("Flush failed ... cannot save binlog paths ..", zap.Error(err))
|
// log.Error("Flush failed ... cannot save binlog paths ..", zap.Error(err))
|
||||||
dmlFlushedCh <- nil
|
// dmlFlushedCh <- nil
|
||||||
return
|
// return
|
||||||
}
|
// }
|
||||||
|
|
||||||
binlogPaths := make([]*datapb.ID2PathList, 0, len(bufferField2Paths))
|
binlogPaths := make([]*datapb.ID2PathList, 0, len(bufferField2Paths))
|
||||||
for k, paths := range bufferField2Paths {
|
for k, paths := range bufferField2Paths {
|
||||||
@ -732,28 +732,28 @@ func (ibNode *insertBufferNode) completeFlush(segID UniqueID, wait <-chan map[Un
|
|||||||
ibNode.replica.setIsFlushed(segID)
|
ibNode.replica.setIsFlushed(segID)
|
||||||
ibNode.updateSegStatistics([]UniqueID{segID})
|
ibNode.updateSegStatistics([]UniqueID{segID})
|
||||||
|
|
||||||
msgPack := msgstream.MsgPack{}
|
// msgPack := msgstream.MsgPack{}
|
||||||
completeFlushMsg := internalpb.SegmentFlushCompletedMsg{
|
// completeFlushMsg := internalpb.SegmentFlushCompletedMsg{
|
||||||
Base: &commonpb.MsgBase{
|
// Base: &commonpb.MsgBase{
|
||||||
MsgType: commonpb.MsgType_SegmentFlushDone,
|
// MsgType: commonpb.MsgType_SegmentFlushDone,
|
||||||
MsgID: 0, // GOOSE TODO
|
// MsgID: 0, // GOOSE TODO
|
||||||
Timestamp: 0, // GOOSE TODO
|
// Timestamp: 0, // GOOSE TODO
|
||||||
SourceID: Params.NodeID,
|
// SourceID: Params.NodeID,
|
||||||
},
|
// },
|
||||||
SegmentID: segID,
|
// SegmentID: segID,
|
||||||
}
|
// }
|
||||||
var msg msgstream.TsMsg = &msgstream.FlushCompletedMsg{
|
// var msg msgstream.TsMsg = &msgstream.FlushCompletedMsg{
|
||||||
BaseMsg: msgstream.BaseMsg{
|
// BaseMsg: msgstream.BaseMsg{
|
||||||
HashValues: []uint32{0},
|
// HashValues: []uint32{0},
|
||||||
},
|
// },
|
||||||
SegmentFlushCompletedMsg: completeFlushMsg,
|
// SegmentFlushCompletedMsg: completeFlushMsg,
|
||||||
}
|
// }
|
||||||
|
//
|
||||||
msgPack.Msgs = append(msgPack.Msgs, msg)
|
// msgPack.Msgs = append(msgPack.Msgs, msg)
|
||||||
err = ibNode.completeFlushStream.Produce(&msgPack)
|
// err = ibNode.completeFlushStream.Produce(&msgPack)
|
||||||
if err != nil {
|
// if err != nil {
|
||||||
log.Error(".. Produce complete flush msg failed ..", zap.Error(err))
|
// log.Error(".. Produce complete flush msg failed ..", zap.Error(err))
|
||||||
}
|
// }
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp) error {
|
func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp) error {
|
||||||
|
|||||||
@ -193,6 +193,10 @@ type DataServiceFactory struct {
|
|||||||
types.DataService
|
types.DataService
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (ds *DataServiceFactory) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
|
||||||
|
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
|
||||||
|
}
|
||||||
|
|
||||||
func (ds *DataServiceFactory) RegisterNode(ctx context.Context, req *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) {
|
func (ds *DataServiceFactory) RegisterNode(ctx context.Context, req *datapb.RegisterNodeRequest) (*datapb.RegisterNodeResponse, error) {
|
||||||
ret := &datapb.RegisterNodeResponse{Status: &commonpb.Status{
|
ret := &datapb.RegisterNodeResponse{Status: &commonpb.Status{
|
||||||
ErrorCode: commonpb.ErrorCode_Success}}
|
ErrorCode: commonpb.ErrorCode_Success}}
|
||||||
@ -210,11 +214,6 @@ func (ds *DataServiceFactory) RegisterNode(ctx context.Context, req *datapb.Regi
|
|||||||
return ret, nil
|
return ret, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (ds *DataServiceFactory) WatchDmChannels(ctx context.Context, req *datapb.FlushRequest) (*commonpb.Status, error) {
|
|
||||||
ret := &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}
|
|
||||||
return ret, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func (mf *MetaFactory) CollectionMetaFactory(collectionID UniqueID, collectionName string) *etcdpb.CollectionMeta {
|
func (mf *MetaFactory) CollectionMetaFactory(collectionID UniqueID, collectionName string) *etcdpb.CollectionMeta {
|
||||||
sch := schemapb.CollectionSchema{
|
sch := schemapb.CollectionSchema{
|
||||||
Name: collectionName,
|
Name: collectionName,
|
||||||
|
|||||||
@ -321,21 +321,21 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("Failed to get collection info", zap.Int64("collectionID", req.GetCollectionID()), zap.Error(err))
|
log.Error("Failed to get collection info", zap.Int64("collectionID", req.GetCollectionID()), zap.Error(err))
|
||||||
resp.Reason = err.Error()
|
resp.Reason = err.Error()
|
||||||
return resp, err
|
return resp, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
meta, err := s.prepareBinlogAndPos(req)
|
meta, err := s.prepareBinlogAndPos(req)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("prepare binlog and pos meta failed", zap.Error(err))
|
log.Error("prepare binlog and pos meta failed", zap.Error(err))
|
||||||
resp.Reason = err.Error()
|
resp.Reason = err.Error()
|
||||||
return resp, err
|
return resp, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// set segment to SegmentState_Flushing
|
// set segment to SegmentState_Flushing
|
||||||
err = s.meta.FlushSegmentWithBinlogAndPos(req.SegmentID, meta)
|
err = s.meta.FlushSegmentWithBinlogAndPos(req.SegmentID, meta)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
resp.Reason = err.Error()
|
resp.Reason = err.Error()
|
||||||
return resp, err
|
return resp, nil
|
||||||
}
|
}
|
||||||
log.Debug("flush segment with meta", zap.Int64("id", req.SegmentID),
|
log.Debug("flush segment with meta", zap.Int64("id", req.SegmentID),
|
||||||
zap.Any("meta", meta))
|
zap.Any("meta", meta))
|
||||||
@ -345,7 +345,7 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||||||
err = s.flushMsgStream.Produce(&msgPack)
|
err = s.flushMsgStream.Produce(&msgPack)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
resp.Reason = err.Error()
|
resp.Reason = err.Error()
|
||||||
return resp, err
|
return resp, nil
|
||||||
}
|
}
|
||||||
log.Debug("send segment flush msg", zap.Int64("id", req.SegmentID))
|
log.Debug("send segment flush msg", zap.Int64("id", req.SegmentID))
|
||||||
|
|
||||||
@ -353,7 +353,7 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath
|
|||||||
if err = s.meta.FlushSegment(req.SegmentID); err != nil {
|
if err = s.meta.FlushSegment(req.SegmentID); err != nil {
|
||||||
log.Error("flush segment complete failed", zap.Error(err))
|
log.Error("flush segment complete failed", zap.Error(err))
|
||||||
resp.Reason = err.Error()
|
resp.Reason = err.Error()
|
||||||
return resp, err
|
return resp, nil
|
||||||
}
|
}
|
||||||
log.Debug("flush segment complete", zap.Int64("id", req.SegmentID))
|
log.Debug("flush segment complete", zap.Int64("id", req.SegmentID))
|
||||||
|
|
||||||
|
|||||||
@ -20,17 +20,18 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
grpcdatanodeclient "github.com/milvus-io/milvus/internal/distributed/datanode/client"
|
grpcdatanodeclient "github.com/milvus-io/milvus/internal/distributed/datanode/client"
|
||||||
"github.com/milvus-io/milvus/internal/logutil"
|
|
||||||
|
"go.etcd.io/etcd/clientv3"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
|
"github.com/milvus-io/milvus/internal/logutil"
|
||||||
"github.com/milvus-io/milvus/internal/msgstream"
|
"github.com/milvus-io/milvus/internal/msgstream"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/retry"
|
"github.com/milvus-io/milvus/internal/util/retry"
|
||||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||||
"go.etcd.io/etcd/clientv3"
|
|
||||||
"go.uber.org/zap"
|
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
|
|||||||
@ -562,7 +562,7 @@ func TestSaveBinlogPaths(t *testing.T) {
|
|||||||
},
|
},
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
assert.NotNil(t, err)
|
assert.Nil(t, err)
|
||||||
assert.EqualValues(t, resp.ErrorCode, commonpb.ErrorCode_UnexpectedError)
|
assert.EqualValues(t, resp.ErrorCode, commonpb.ErrorCode_UnexpectedError)
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|||||||
@ -272,6 +272,10 @@ func (c *Client) GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoR
|
|||||||
return ret.(*datapb.GetSegmentInfoResponse), err
|
return ret.(*datapb.GetSegmentInfoResponse), err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *Client) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error) {
|
||||||
|
return c.grpcClient.SaveBinlogPaths(ctx, req)
|
||||||
|
}
|
||||||
|
|
||||||
func (c *Client) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
|
func (c *Client) GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error) {
|
||||||
ret, err := c.recall(func() (interface{}, error) {
|
ret, err := c.recall(func() (interface{}, error) {
|
||||||
return c.grpcClient.GetRecoveryInfo(ctx, req)
|
return c.grpcClient.GetRecoveryInfo(ctx, req)
|
||||||
|
|||||||
@ -62,6 +62,7 @@ type DataService interface {
|
|||||||
GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error)
|
GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error)
|
||||||
GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error)
|
GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error)
|
||||||
GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error)
|
GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error)
|
||||||
|
SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error)
|
||||||
}
|
}
|
||||||
|
|
||||||
type IndexNode interface {
|
type IndexNode interface {
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user