2021-10-03 17:44:25 +08:00
## 9. Data Service
2020-12-29 18:02:44 +08:00
2021-10-03 17:44:25 +08:00
#### 9.1 Overview
2020-12-29 18:02:44 +08:00
2021-06-21 19:22:15 +08:00
< img src = "./figs/data_coord.png" width = 700 >
2020-12-29 18:02:44 +08:00
2021-10-03 17:44:25 +08:00
#### 9.2 Data Service Interface
2020-12-29 18:02:44 +08:00
```go
2021-06-21 16:02:11 +08:00
type DataCoord interface {
2021-04-12 12:45:38 +08:00
Component
TimeTickProvider
2021-10-27 14:46:37 +08:00
// Flush notifies DataCoord to flush all current growing segments of specified Collection
2021-09-08 17:12:11 +08:00
Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.FlushResponse, error)
2021-10-28 20:13:48 +08:00
// AssignSegmentID applies allocations for specified Coolection/Partition and related Channel Name(Virtial Channel)
2021-04-12 12:45:38 +08:00
AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error)
2021-10-28 20:13:48 +08:00
// GetSegmentStates requests segment state information
2021-04-12 12:45:38 +08:00
GetSegmentStates(ctx context.Context, req *datapb.GetSegmentStatesRequest) (*datapb.GetSegmentStatesResponse, error)
2021-10-30 18:46:33 +08:00
// GetInsertBinlogPaths requests binlog paths for specified segment
2021-04-12 12:45:38 +08:00
GetInsertBinlogPaths(ctx context.Context, req *datapb.GetInsertBinlogPathsRequest) (*datapb.GetInsertBinlogPathsResponse, error)
2021-11-09 19:20:51 +08:00
// GetSegmentInfoChannel legacy API, returns segment info statistics channel
2021-04-12 12:45:38 +08:00
GetSegmentInfoChannel(ctx context.Context) (*milvuspb.StringResponse, error)
2021-11-01 16:30:03 +08:00
// GetCollectionStatistics requests collection statistics
2021-04-12 12:45:38 +08:00
GetCollectionStatistics(ctx context.Context, req *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error)
2021-11-02 21:26:43 +08:00
// GetParititonStatistics requests partition statistics
2021-04-12 12:45:38 +08:00
GetPartitionStatistics(ctx context.Context, req *datapb.GetPartitionStatisticsRequest) (*datapb.GetPartitionStatisticsResponse, error)
2021-11-03 10:52:35 +08:00
// GetSegmentInfo requests segment info
2021-04-12 12:45:38 +08:00
GetSegmentInfo(ctx context.Context, req *datapb.GetSegmentInfoRequest) (*datapb.GetSegmentInfoResponse, error)
2021-11-04 13:30:34 +08:00
// GetRecoveryInfo request segment recovery info of collection/partition
2021-06-21 16:02:11 +08:00
GetRecoveryInfo(ctx context.Context, req *datapb.GetRecoveryInfoRequest) (*datapb.GetRecoveryInfoResponse, error)
2021-11-06 12:17:45 +08:00
// SaveBinlogPaths updates segments binlogs(including insert binlogs, stats logs and delta logs)
2021-06-21 16:02:11 +08:00
SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) (*commonpb.Status, error)
2021-11-08 09:34:58 +08:00
// GetFlushedSegments returns flushed segment list of requested collection/parition
2021-09-08 17:12:11 +08:00
GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error)
2021-11-08 13:05:44 +08:00
// GetMetrics gets the metrics about DataCoord
2021-09-08 17:12:11 +08:00
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
2021-11-15 19:07:56 +08:00
// CompleteCompaction completes a compaction with the result
CompleteCompaction(ctx context.Context, req *datapb.CompactionResult) (*commonpb.Status, error)
2020-12-29 18:02:44 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _MsgBase_
2021-01-13 11:08:03 +08:00
```go
2021-01-15 14:38:36 +08:00
type MsgBase struct {
2021-04-12 12:45:38 +08:00
MsgType MsgType
MsgID UniqueID
Timestamp Timestamp
SourceID UniqueID
2021-01-13 11:08:03 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _Flush_
2021-03-04 10:35:28 +08:00
```go
type FlushRequest struct {
2021-04-12 12:45:38 +08:00
Base *commonpb.MsgBase
DbID UniqueID
CollectionID UniqueID
2021-01-13 11:08:03 +08:00
}
2021-01-11 18:35:54 +08:00
```
2021-10-11 21:25:47 +08:00
- _AssignSegmentID_
2020-12-29 18:02:44 +08:00
```go
2021-04-12 12:45:38 +08:00
type SegmentIDRequest struct {
Count uint32
ChannelName string
CollectionID UniqueID
PartitionID UniqueID
2020-12-29 18:02:44 +08:00
}
2021-04-12 12:45:38 +08:00
type AssignSegmentIDRequest struct {
2021-10-11 21:25:47 +08:00
NodeID int64
PeerRole string
SegmentIDRequests []*SegmentIDRequest
2020-12-29 18:02:44 +08:00
}
type SegIDAssignment struct {
2021-04-12 12:45:38 +08:00
SegID UniqueID
ChannelName string
Count uint32
CollectionID UniqueID
PartitionID UniqueID
ExpireTime uint64
Status *commonpb.Status
2020-12-29 18:02:44 +08:00
}
2021-04-12 12:45:38 +08:00
type AssignSegmentIDResponse struct {
SegIDAssignments []*SegmentIDAssignment
Status *commonpb.Status
2020-12-29 18:02:44 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _GetSegmentStates_
2021-01-12 18:03:24 +08:00
```go
2021-04-12 12:45:38 +08:00
type GetSegmentStatesRequest struct {
2021-10-11 21:25:47 +08:00
Base *commonpb.MsgBase
SegmentIDs []int64
2021-01-12 18:03:24 +08:00
}
2021-04-12 12:45:38 +08:00
type SegmentState int32
const (
SegmentState_SegmentStateNone SegmentState = 0
SegmentState_NotExist SegmentState = 1
SegmentState_Growing SegmentState = 2
SegmentState_Sealed SegmentState = 3
SegmentState_Flushed SegmentState = 4
2021-06-21 16:02:11 +08:00
SegmentState_Flushing SegmentState = 5
2021-11-17 10:57:10 +08:00
SegmentState_Dropped SegmentState = 6
2021-04-12 12:45:38 +08:00
)
2021-02-03 18:55:00 +08:00
2021-03-04 10:35:28 +08:00
type SegmentStateInfo struct {
2021-04-12 12:45:38 +08:00
SegmentID UniqueID
State commonpb.SegmentState
StartPosition *internalpb.MsgPosition
EndPosition *internalpb.MsgPosition
Status *commonpb.Status
2021-02-03 18:55:00 +08:00
}
2021-01-12 18:03:24 +08:00
2021-04-12 12:45:38 +08:00
type GetSegmentStatesResponse struct {
Status *commonpb.Status
States []*SegmentStateInfo
2021-02-03 18:55:00 +08:00
}
```
2021-01-12 18:03:24 +08:00
2021-10-11 21:25:47 +08:00
- _GetInsertBinlogPaths_
2020-12-29 18:02:44 +08:00
```go
2021-04-12 12:45:38 +08:00
type GetInsertBinlogPathsRequest struct {
Base *commonpb.MsgBase
SegmentID UniqueID
2020-12-29 18:02:44 +08:00
}
2021-04-12 12:45:38 +08:00
type GetInsertBinlogPathsResponse struct {
FieldIDs []int64
Paths []*internalpb.StringList
Status *commonpb.Status
2020-12-29 18:02:44 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _GetCollectionStatistics_
2021-01-20 16:46:58 +08:00
```go
2021-04-12 12:45:38 +08:00
type GetCollectionStatisticsRequest struct {
Base *commonpb.MsgBase
DbID int64
CollectionID int64
2021-01-20 16:46:58 +08:00
}
2021-03-04 10:35:28 +08:00
2021-04-12 12:45:38 +08:00
type GetCollectionStatisticsResponse struct {
Stats []*commonpb.KeyValuePair
Status *commonpb.Status
2021-01-20 16:46:58 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _GetPartitionStatistics_
2021-01-20 16:46:58 +08:00
```go
2021-04-12 12:45:38 +08:00
type GetPartitionStatisticsRequest struct {
Base *commonpb.MsgBase
DbID UniqueID
CollectionID UniqueID
PartitionID UniqueID
2021-01-20 16:46:58 +08:00
}
2021-03-04 10:35:28 +08:00
2021-04-12 12:45:38 +08:00
type GetPartitionStatisticsResponse struct {
Stats []*commonpb.KeyValuePair
Status *commonpb.Status
2021-01-20 16:46:58 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _GetSegmentInfo_
2021-03-04 10:35:28 +08:00
```go
2021-04-12 12:45:38 +08:00
type GetSegmentInfoRequest struct{
Base *commonpb.MsgBase
SegmentIDs []UniqueID
2021-03-04 10:35:28 +08:00
}
type SegmentInfo struct {
2021-10-11 21:25:47 +08:00
ID int64
CollectionID int64
PartitionID int64
InsertChannel string
NumOfRows int64
State commonpb.SegmentState
DmlPosition *internalpb.MsgPosition
MaxRowNum int64
LastExpireTime uint64
StartPosition *internalpb.MsgPosition
2021-10-14 13:54:33 +08:00
DmlPosition *internalpb.MsgPosition
Binlogs []*FieldBinlog
2021-04-12 12:45:38 +08:00
}
type GetSegmentInfoResponse struct{
Status *commonpb.Status
infos []SegmentInfo
2021-03-04 10:35:28 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _GetRecoveryInfo_
2021-06-21 16:02:11 +08:00
```go
type GetRecoveryInfoRequest struct {
2021-10-11 21:25:47 +08:00
Base *commonpb.MsgBase
CollectionID int64
PartitionID int64
2021-06-21 16:02:11 +08:00
}
type VchannelInfo struct {
2021-10-11 21:25:47 +08:00
CollectionID int64
ChannelName string
SeekPosition *internalpb.MsgPosition
UnflushedSegments []*SegmentInfo
FlushedSegments []int64
2021-06-21 16:02:11 +08:00
}
type SegmentBinlogs struct {
2021-10-11 21:25:47 +08:00
SegmentID int64
FieldBinlogs []*FieldBinlog
2021-06-21 16:02:11 +08:00
}
type GetRecoveryInfoResponse struct {
2021-10-11 21:25:47 +08:00
Status *commonpb.Status
Channels []*VchannelInfo
Binlogs []*SegmentBinlogs
2021-06-21 16:02:11 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _SaveBinlogPaths_
2021-06-21 16:02:11 +08:00
```go
type SegmentStartPosition struct {
2021-10-11 21:25:47 +08:00
StartPosition *internalpb.MsgPosition
SegmentID int64
2021-06-21 16:02:11 +08:00
}
type SaveBinlogPathsRequest struct {
2021-10-11 21:25:47 +08:00
Base *commonpb.MsgBase
SegmentID int64
CollectionID int64
Field2BinlogPaths []*ID2PathList
CheckPoints []*CheckPoint
StartPositions []*SegmentStartPosition
Flushed bool
2021-06-21 16:02:11 +08:00
}
```
2021-03-04 10:35:28 +08:00
2021-10-03 17:44:25 +08:00
#### 9.3 Insert Channel
2021-01-15 14:38:36 +08:00
2021-10-11 21:25:47 +08:00
- _InsertMsg_
2021-04-12 12:45:38 +08:00
2021-01-15 14:38:36 +08:00
```go
type InsertRequest struct {
2021-04-12 12:45:38 +08:00
Base *commonpb.MsgBase
DbName string
CollectionName string
PartitionName string
DbID UniqueID
CollectionID UniqueID
PartitionID UniqueID
SegmentID UniqueID
ChannelID string
Timestamps []uint64
RowIDs []int64
RowData []*commonpb.Blob
}
type InsertMsg struct {
BaseMsg
InsertRequest
2021-01-15 14:02:12 +08:00
}
```
2021-10-03 17:44:25 +08:00
#### 9.4 Data Node Interface
2021-01-11 18:35:54 +08:00
```go
type DataNode interface {
2021-04-12 12:45:38 +08:00
Component
2021-11-18 22:19:49 +08:00
/// WatchDmChannels watches the channels about data manipulation.
2021-04-12 12:45:38 +08:00
WatchDmChannels(ctx context.Context, req *datapb.WatchDmChannelsRequest) (*commonpb.Status, error)
2021-11-15 09:13:17 +08:00
// FlushSegments notifies DataNode to flush the segments req provids. The flush tasks are async to this rpc, DataNode will flush the segments in the background.
2021-04-12 12:45:38 +08:00
FlushSegments(ctx context.Context, req *datapb.FlushSegmentsRequest) (*commonpb.Status, error)
2021-11-12 13:25:47 +08:00
// GetMetrics gets the metrics about DataNode.
GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error)
2021-11-16 09:43:18 +08:00
// Compaction will add a compaction task according to the request plan
2021-11-12 13:25:47 +08:00
Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error)
2021-02-04 19:34:35 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _WatchDmChannels_
2021-01-13 11:08:03 +08:00
```go
type WatchDmChannelRequest struct {
2021-04-12 12:45:38 +08:00
Base *commonpb.MsgBase
2021-06-21 18:32:12 +08:00
Vchannels []*VchannelInfo
2021-01-13 11:08:03 +08:00
}
```
2021-10-11 21:25:47 +08:00
- _FlushSegments_
2021-01-13 11:08:03 +08:00
```go
2021-04-12 12:45:38 +08:00
type FlushSegmentsRequest struct {
Base *commonpb.MsgBase
DbID UniqueID
CollectionID UniqueID
SegmentIDs []int64
2021-01-11 18:35:54 +08:00
}
```
2021-10-03 17:44:25 +08:00
#### 9.5 SegmentStatistics Update Channel
2021-03-04 10:35:28 +08:00
2021-10-11 21:25:47 +08:00
- _SegmentStatisticsMsg_
2021-01-16 15:47:33 +08:00
```go
type SegmentStatisticsUpdates struct {
2021-04-12 12:45:38 +08:00
SegmentID UniqueID
MemorySize int64
NumRows int64
CreateTime uint64
EndTime uint64
StartPosition *internalpb.MsgPosition
EndPosition *internalpb.MsgPosition
}
type SegmentStatistics struct {
Base *commonpb.MsgBase
SegStats []*SegmentStatisticsUpdates
}
type SegmentStatisticsMsg struct {
BaseMsg
SegmentStatistics
2021-01-16 15:47:33 +08:00
}
2021-06-21 18:32:12 +08:00
```
2021-10-11 21:25:47 +08:00
2021-10-03 17:44:25 +08:00
#### 9.6 DataNode Time Tick Channel
2021-06-21 18:32:12 +08:00
2021-10-11 21:25:47 +08:00
- _DataNode Tt Msg_
2021-06-21 18:32:12 +08:00
```go
message DataNodeTtMsg {
Base *commonpb.MsgBase
ChannelName string
Timestamp uint64
}
2021-01-16 15:47:33 +08:00
```