mirror of https://github.com/milvus-io/milvus.git
Support column-based insert data in message stream (#15802)
Signed-off-by: dragondriver <jiquan.long@zilliz.com>pull/15869/head
parent
a0ca4c43f2
commit
f71651e294
|
@ -17,13 +17,9 @@
|
|||
package datanode
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"strconv"
|
||||
"sync"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
@ -31,7 +27,6 @@ import (
|
|||
"go.uber.org/atomic"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
|
@ -421,7 +416,7 @@ func (ibNode *insertBufferNode) updateSegStatesInReplica(insertMsgs []*msgstream
|
|||
// 1.3 Put back into buffer
|
||||
// 1.4 Update related statistics
|
||||
func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos *internalpb.MsgPosition) error {
|
||||
if len(msg.RowIDs) != len(msg.Timestamps) || len(msg.RowIDs) != len(msg.RowData) {
|
||||
if !msg.CheckAligned() {
|
||||
return errors.New("misaligned messages detected")
|
||||
}
|
||||
currentSegID := msg.GetSegmentID()
|
||||
|
@ -440,15 +435,10 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos
|
|||
if field.DataType == schemapb.DataType_FloatVector ||
|
||||
field.DataType == schemapb.DataType_BinaryVector {
|
||||
|
||||
for _, t := range field.TypeParams {
|
||||
if t.Key == "dim" {
|
||||
dimension, err = strconv.Atoi(t.Value)
|
||||
if err != nil {
|
||||
log.Error("strconv wrong on get dim", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
dimension, err = storage.GetDimFromParams(field.TypeParams)
|
||||
if err != nil {
|
||||
log.Error("failed to get dim from field", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
@ -461,224 +451,26 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos
|
|||
bd, _ := ibNode.insertBuffer.LoadOrStore(currentSegID, newbd)
|
||||
|
||||
buffer := bd.(*BufferData)
|
||||
idata := buffer.buffer
|
||||
// idata := buffer.buffer
|
||||
|
||||
// 1.2 Get Fields
|
||||
blobReaders := make([]io.Reader, 0)
|
||||
for _, blob := range msg.RowData {
|
||||
blobReaders = append(blobReaders, bytes.NewReader(blob.GetValue()))
|
||||
addedBuffer, err := storage.InsertMsgToInsertData(msg, collSchema)
|
||||
if err != nil {
|
||||
log.Error("failed to transfer insert msg to insert data", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
for _, field := range collSchema.Fields {
|
||||
switch field.DataType {
|
||||
case schemapb.DataType_FloatVector:
|
||||
var dim int
|
||||
for _, t := range field.TypeParams {
|
||||
if t.Key == "dim" {
|
||||
dim, err = strconv.Atoi(t.Value)
|
||||
if err != nil {
|
||||
log.Error("strconv wrong on get dim", zap.Error(err))
|
||||
break
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.FloatVectorFieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]float32, 0),
|
||||
Dim: dim,
|
||||
}
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*storage.FloatVectorFieldData)
|
||||
for _, r := range blobReaders {
|
||||
var v = make([]float32, dim)
|
||||
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v...)
|
||||
}
|
||||
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
|
||||
case schemapb.DataType_BinaryVector:
|
||||
var dim int
|
||||
for _, t := range field.TypeParams {
|
||||
if t.Key == "dim" {
|
||||
dim, err = strconv.Atoi(t.Value)
|
||||
if err != nil {
|
||||
log.Error("strconv wrong on get dim", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.BinaryVectorFieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]byte, 0),
|
||||
Dim: dim,
|
||||
}
|
||||
}
|
||||
fieldData := idata.Data[field.FieldID].(*storage.BinaryVectorFieldData)
|
||||
|
||||
for _, r := range blobReaders {
|
||||
var v = make([]byte, dim/8)
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v...)
|
||||
}
|
||||
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
|
||||
case schemapb.DataType_Bool:
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.BoolFieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]bool, 0),
|
||||
}
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*storage.BoolFieldData)
|
||||
for _, r := range blobReaders {
|
||||
var v bool
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v)
|
||||
}
|
||||
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
|
||||
case schemapb.DataType_Int8:
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.Int8FieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]int8, 0),
|
||||
}
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*storage.Int8FieldData)
|
||||
for _, r := range blobReaders {
|
||||
var v int8
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v)
|
||||
}
|
||||
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
|
||||
case schemapb.DataType_Int16:
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.Int16FieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]int16, 0),
|
||||
}
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*storage.Int16FieldData)
|
||||
for _, r := range blobReaders {
|
||||
var v int16
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v)
|
||||
}
|
||||
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
|
||||
case schemapb.DataType_Int32:
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.Int32FieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]int32, 0),
|
||||
}
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*storage.Int32FieldData)
|
||||
for _, r := range blobReaders {
|
||||
var v int32
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v)
|
||||
}
|
||||
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
|
||||
case schemapb.DataType_Int64:
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.Int64FieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]int64, 0),
|
||||
}
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*storage.Int64FieldData)
|
||||
switch field.FieldID {
|
||||
case 0: // rowIDs
|
||||
fieldData.Data = append(fieldData.Data, msg.RowIDs...)
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
case 1: // Timestamps
|
||||
for _, ts := range msg.Timestamps {
|
||||
fieldData.Data = append(fieldData.Data, int64(ts))
|
||||
}
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
default:
|
||||
for _, r := range blobReaders {
|
||||
var v int64
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v)
|
||||
}
|
||||
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
}
|
||||
if field.IsPrimaryKey {
|
||||
// update segment pk filter
|
||||
ibNode.replica.updateSegmentPKRange(currentSegID, fieldData.Data)
|
||||
}
|
||||
|
||||
case schemapb.DataType_Float:
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.FloatFieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]float32, 0),
|
||||
}
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*storage.FloatFieldData)
|
||||
|
||||
for _, r := range blobReaders {
|
||||
var v float32
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v)
|
||||
}
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
|
||||
case schemapb.DataType_Double:
|
||||
if _, ok := idata.Data[field.FieldID]; !ok {
|
||||
idata.Data[field.FieldID] = &storage.DoubleFieldData{
|
||||
NumRows: make([]int64, 0, 1),
|
||||
Data: make([]float64, 0),
|
||||
}
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*storage.DoubleFieldData)
|
||||
|
||||
for _, r := range blobReaders {
|
||||
var v float64
|
||||
readBinary(r, &v, field.DataType)
|
||||
|
||||
fieldData.Data = append(fieldData.Data, v)
|
||||
}
|
||||
fieldData.NumRows = append(fieldData.NumRows, int64(len(msg.RowData)))
|
||||
}
|
||||
addedPfData, err := storage.GetPkFromInsertData(collSchema, addedBuffer)
|
||||
if err != nil {
|
||||
log.Warn("no primary field found in insert msg", zap.Error(err))
|
||||
} else {
|
||||
ibNode.replica.updateSegmentPKRange(currentSegID, addedPfData)
|
||||
}
|
||||
|
||||
// Maybe there are large write zoom if frequent insert requests are met.
|
||||
buffer.buffer = storage.MergeInsertData(buffer.buffer, addedBuffer)
|
||||
|
||||
// update buffer size
|
||||
buffer.updateSize(int64(len(msg.RowData)))
|
||||
buffer.updateSize(int64(msg.NRows()))
|
||||
metrics.DataNodeConsumeMsgRowsCount.WithLabelValues(metrics.InsertLabel, fmt.Sprint(Params.DataNodeCfg.NodeID)).Add(float64(len(msg.RowData)))
|
||||
|
||||
// store in buffer
|
||||
|
@ -690,16 +482,6 @@ func (ibNode *insertBufferNode) bufferInsertMsg(msg *msgstream.InsertMsg, endPos
|
|||
return nil
|
||||
}
|
||||
|
||||
// readBinary read data in bytes and write it into receiver.
|
||||
// The receiver can be any type in int8, int16, int32, int64, float32, float64 and bool
|
||||
// readBinary uses LittleEndian ByteOrder.
|
||||
func readBinary(reader io.Reader, receiver interface{}, dataType schemapb.DataType) {
|
||||
err := binary.Read(reader, common.Endian, receiver)
|
||||
if err != nil {
|
||||
log.Error("binary.Read failed", zap.Any("data type", dataType), zap.Error(err))
|
||||
}
|
||||
}
|
||||
|
||||
// writeHardTimeTick writes timetick once insertBufferNode operates.
|
||||
func (ibNode *insertBufferNode) writeHardTimeTick(ts Timestamp, segmentIDs []int64) error {
|
||||
ibNode.ttLogger.LogTs(ts)
|
||||
|
|
|
@ -21,6 +21,9 @@ import (
|
|||
"errors"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
@ -171,6 +174,94 @@ func (it *InsertMsg) Unmarshal(input MarshalType) (TsMsg, error) {
|
|||
return insertMsg, nil
|
||||
}
|
||||
|
||||
func (it *InsertMsg) IsRowBased() bool {
|
||||
return it.GetVersion() == internalpb.InsertDataVersion_RowBased
|
||||
}
|
||||
|
||||
func (it *InsertMsg) IsColumnBased() bool {
|
||||
return it.GetVersion() == internalpb.InsertDataVersion_ColumnBased
|
||||
}
|
||||
|
||||
func (it *InsertMsg) NRows() uint64 {
|
||||
if it.IsRowBased() {
|
||||
return uint64(len(it.RowData))
|
||||
}
|
||||
return it.InsertRequest.GetNumRows()
|
||||
}
|
||||
|
||||
func (it *InsertMsg) CheckAligned() bool {
|
||||
return len(it.GetRowIDs()) == len(it.GetTimestamps()) &&
|
||||
uint64(len(it.GetRowIDs())) == it.NRows()
|
||||
}
|
||||
|
||||
func (it *InsertMsg) rowBasedIndexRequest(index int) internalpb.InsertRequest {
|
||||
return internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: it.Base.MsgID,
|
||||
Timestamp: it.Timestamps[index],
|
||||
SourceID: it.Base.SourceID,
|
||||
},
|
||||
DbID: it.DbID,
|
||||
CollectionID: it.CollectionID,
|
||||
PartitionID: it.PartitionID,
|
||||
CollectionName: it.CollectionName,
|
||||
PartitionName: it.PartitionName,
|
||||
SegmentID: it.SegmentID,
|
||||
ShardName: it.ShardName,
|
||||
Timestamps: []uint64{it.Timestamps[index]},
|
||||
RowIDs: []int64{it.RowIDs[index]},
|
||||
RowData: []*commonpb.Blob{it.RowData[index]},
|
||||
Version: internalpb.InsertDataVersion_RowBased,
|
||||
}
|
||||
}
|
||||
|
||||
func (it *InsertMsg) columnBasedIndexRequest(index int) internalpb.InsertRequest {
|
||||
colNum := len(it.GetFieldsData())
|
||||
fieldsData := make([]*schemapb.FieldData, colNum)
|
||||
typeutil.AppendFieldData(fieldsData, it.GetFieldsData(), int64(index))
|
||||
return internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: it.Base.MsgID,
|
||||
Timestamp: it.Timestamps[index],
|
||||
SourceID: it.Base.SourceID,
|
||||
},
|
||||
DbID: it.DbID,
|
||||
CollectionID: it.CollectionID,
|
||||
PartitionID: it.PartitionID,
|
||||
CollectionName: it.CollectionName,
|
||||
PartitionName: it.PartitionName,
|
||||
SegmentID: it.SegmentID,
|
||||
ShardName: it.ShardName,
|
||||
Timestamps: []uint64{it.Timestamps[index]},
|
||||
RowIDs: []int64{it.RowIDs[index]},
|
||||
FieldsData: fieldsData,
|
||||
NumRows: 1,
|
||||
Version: internalpb.InsertDataVersion_ColumnBased,
|
||||
}
|
||||
}
|
||||
|
||||
func (it *InsertMsg) IndexRequest(index int) internalpb.InsertRequest {
|
||||
if it.IsRowBased() {
|
||||
return it.rowBasedIndexRequest(index)
|
||||
}
|
||||
return it.columnBasedIndexRequest(index)
|
||||
}
|
||||
|
||||
func (it *InsertMsg) IndexMsg(index int) *InsertMsg {
|
||||
return &InsertMsg{
|
||||
BaseMsg: BaseMsg{
|
||||
Ctx: it.TraceCtx(),
|
||||
BeginTimestamp: it.BeginTimestamp,
|
||||
EndTimestamp: it.EndTimestamp,
|
||||
HashValues: it.HashValues,
|
||||
MsgPosition: it.MsgPosition,
|
||||
},
|
||||
InsertRequest: it.IndexRequest(index),
|
||||
}
|
||||
}
|
||||
|
||||
/////////////////////////////////////////Delete//////////////////////////////////////////
|
||||
|
||||
// DeleteMsg is a message pack that contains delete request
|
||||
|
|
|
@ -136,6 +136,129 @@ func TestInsertMsg_Unmarshal_IllegalParameter(t *testing.T) {
|
|||
assert.Nil(t, tsMsg)
|
||||
}
|
||||
|
||||
func TestInsertMsg_RowBasedFormat(t *testing.T) {
|
||||
msg := &InsertMsg{
|
||||
InsertRequest: internalpb.InsertRequest{
|
||||
Version: internalpb.InsertDataVersion_RowBased,
|
||||
},
|
||||
}
|
||||
assert.True(t, msg.IsRowBased())
|
||||
}
|
||||
|
||||
func TestInsertMsg_ColumnBasedFormat(t *testing.T) {
|
||||
msg := &InsertMsg{
|
||||
InsertRequest: internalpb.InsertRequest{
|
||||
Version: internalpb.InsertDataVersion_ColumnBased,
|
||||
},
|
||||
}
|
||||
assert.True(t, msg.IsColumnBased())
|
||||
}
|
||||
|
||||
func TestInsertMsg_NRows(t *testing.T) {
|
||||
msg1 := &InsertMsg{
|
||||
InsertRequest: internalpb.InsertRequest{
|
||||
RowData: []*commonpb.Blob{
|
||||
{},
|
||||
{},
|
||||
},
|
||||
FieldsData: nil,
|
||||
Version: internalpb.InsertDataVersion_RowBased,
|
||||
},
|
||||
}
|
||||
assert.Equal(t, uint64(2), msg1.NRows())
|
||||
msg2 := &InsertMsg{
|
||||
InsertRequest: internalpb.InsertRequest{
|
||||
RowData: nil,
|
||||
FieldsData: []*schemapb.FieldData{
|
||||
{},
|
||||
},
|
||||
NumRows: 2,
|
||||
Version: internalpb.InsertDataVersion_ColumnBased,
|
||||
},
|
||||
}
|
||||
assert.Equal(t, uint64(2), msg2.NRows())
|
||||
}
|
||||
|
||||
func TestInsertMsg_CheckAligned(t *testing.T) {
|
||||
msg1 := &InsertMsg{
|
||||
InsertRequest: internalpb.InsertRequest{
|
||||
Timestamps: []uint64{1},
|
||||
RowIDs: []int64{1},
|
||||
RowData: []*commonpb.Blob{
|
||||
{},
|
||||
},
|
||||
FieldsData: nil,
|
||||
Version: internalpb.InsertDataVersion_RowBased,
|
||||
},
|
||||
}
|
||||
assert.True(t, msg1.CheckAligned())
|
||||
msg1.InsertRequest.RowData = nil
|
||||
msg1.InsertRequest.FieldsData = []*schemapb.FieldData{
|
||||
{},
|
||||
}
|
||||
msg1.InsertRequest.NumRows = 1
|
||||
msg1.Version = internalpb.InsertDataVersion_ColumnBased
|
||||
assert.True(t, msg1.CheckAligned())
|
||||
}
|
||||
|
||||
func TestInsertMsg_IndexMsg(t *testing.T) {
|
||||
msg := &InsertMsg{
|
||||
BaseMsg: BaseMsg{
|
||||
BeginTimestamp: 1,
|
||||
EndTimestamp: 2,
|
||||
},
|
||||
InsertRequest: internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: 3,
|
||||
Timestamp: 4,
|
||||
SourceID: 5,
|
||||
},
|
||||
DbID: 6,
|
||||
CollectionID: 7,
|
||||
PartitionID: 8,
|
||||
CollectionName: "test",
|
||||
PartitionName: "test",
|
||||
SegmentID: 9,
|
||||
ShardName: "test",
|
||||
Timestamps: []uint64{10},
|
||||
RowIDs: []int64{11},
|
||||
RowData: []*commonpb.Blob{
|
||||
{
|
||||
Value: []byte{1},
|
||||
},
|
||||
},
|
||||
Version: internalpb.InsertDataVersion_RowBased,
|
||||
},
|
||||
}
|
||||
indexMsg := msg.IndexMsg(0)
|
||||
assert.Equal(t, uint64(10), indexMsg.GetTimestamps()[0])
|
||||
assert.Equal(t, int64(11), indexMsg.GetRowIDs()[0])
|
||||
assert.Equal(t, []byte{1}, indexMsg.GetRowData()[0].Value)
|
||||
|
||||
msg.Version = internalpb.InsertDataVersion_ColumnBased
|
||||
msg.FieldsData = []*schemapb.FieldData{
|
||||
{
|
||||
Type: schemapb.DataType_Int64,
|
||||
FieldName: "test",
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_LongData{
|
||||
LongData: &schemapb.LongArray{
|
||||
Data: []int64{1},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: 0,
|
||||
},
|
||||
}
|
||||
indexMsg = msg.IndexMsg(0)
|
||||
assert.Equal(t, uint64(10), indexMsg.GetTimestamps()[0])
|
||||
assert.Equal(t, int64(11), indexMsg.GetRowIDs()[0])
|
||||
assert.Equal(t, int64(1), indexMsg.FieldsData[0].Field.(*schemapb.FieldData_Scalars).Scalars.Data.(*schemapb.ScalarField_LongData).LongData.Data[0])
|
||||
}
|
||||
|
||||
func TestDeleteMsg(t *testing.T) {
|
||||
deleteMsg := &DeleteMsg{
|
||||
BaseMsg: generateBaseMsg(),
|
||||
|
|
|
@ -35,12 +35,9 @@ func InsertRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e
|
|||
insertRequest := request.(*InsertMsg)
|
||||
keys := hashKeys[i]
|
||||
|
||||
timestampLen := len(insertRequest.Timestamps)
|
||||
rowIDLen := len(insertRequest.RowIDs)
|
||||
rowDataLen := len(insertRequest.RowData)
|
||||
keysLen := len(keys)
|
||||
|
||||
if keysLen != timestampLen || keysLen != rowIDLen || keysLen != rowDataLen {
|
||||
if !insertRequest.CheckAligned() || insertRequest.NRows() != uint64(keysLen) {
|
||||
return nil, errors.New("the length of hashValue, timestamps, rowIDs, RowData are not equal")
|
||||
}
|
||||
for index, key := range keys {
|
||||
|
@ -50,31 +47,7 @@ func InsertRepackFunc(tsMsgs []TsMsg, hashKeys [][]int32) (map[int32]*MsgPack, e
|
|||
result[key] = &msgPack
|
||||
}
|
||||
|
||||
sliceRequest := internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: insertRequest.Base.MsgID,
|
||||
Timestamp: insertRequest.Timestamps[index],
|
||||
SourceID: insertRequest.Base.SourceID,
|
||||
},
|
||||
DbID: insertRequest.DbID,
|
||||
CollectionID: insertRequest.CollectionID,
|
||||
PartitionID: insertRequest.PartitionID,
|
||||
CollectionName: insertRequest.CollectionName,
|
||||
PartitionName: insertRequest.PartitionName,
|
||||
SegmentID: insertRequest.SegmentID,
|
||||
ShardName: insertRequest.ShardName,
|
||||
Timestamps: []uint64{insertRequest.Timestamps[index]},
|
||||
RowIDs: []int64{insertRequest.RowIDs[index]},
|
||||
RowData: []*commonpb.Blob{insertRequest.RowData[index]},
|
||||
}
|
||||
|
||||
insertMsg := &InsertMsg{
|
||||
BaseMsg: BaseMsg{
|
||||
Ctx: request.TraceCtx(),
|
||||
},
|
||||
InsertRequest: sliceRequest,
|
||||
}
|
||||
insertMsg := insertRequest.IndexMsg(index)
|
||||
result[key].Msgs = append(result[key].Msgs, insertMsg)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -128,6 +128,12 @@ message CreateIndexRequest {
|
|||
repeated common.KeyValuePair extra_params = 8;
|
||||
}
|
||||
|
||||
enum InsertDataVersion {
|
||||
// 0 must refer to row-based format, since it's the first version in Milvus.
|
||||
RowBased = 0;
|
||||
ColumnBased = 1;
|
||||
}
|
||||
|
||||
message InsertRequest {
|
||||
common.MsgBase base = 1;
|
||||
string shardName = 2;
|
||||
|
@ -140,7 +146,11 @@ message InsertRequest {
|
|||
int64 segmentID = 9;
|
||||
repeated uint64 timestamps = 10;
|
||||
repeated int64 rowIDs = 11;
|
||||
// row_data was reserved for compatibility
|
||||
repeated common.Blob row_data = 12;
|
||||
repeated schema.FieldData fields_data = 13;
|
||||
uint64 num_rows = 14;
|
||||
InsertDataVersion version = 15;
|
||||
}
|
||||
|
||||
message SearchRequest {
|
||||
|
|
|
@ -50,6 +50,32 @@ func (StateCode) EnumDescriptor() ([]byte, []int) {
|
|||
return fileDescriptor_41f4a519b878ee3b, []int{0}
|
||||
}
|
||||
|
||||
type InsertDataVersion int32
|
||||
|
||||
const (
|
||||
// 0 must refer to row-based format, since it's the first version in Milvus.
|
||||
InsertDataVersion_RowBased InsertDataVersion = 0
|
||||
InsertDataVersion_ColumnBased InsertDataVersion = 1
|
||||
)
|
||||
|
||||
var InsertDataVersion_name = map[int32]string{
|
||||
0: "RowBased",
|
||||
1: "ColumnBased",
|
||||
}
|
||||
|
||||
var InsertDataVersion_value = map[string]int32{
|
||||
"RowBased": 0,
|
||||
"ColumnBased": 1,
|
||||
}
|
||||
|
||||
func (x InsertDataVersion) String() string {
|
||||
return proto.EnumName(InsertDataVersion_name, int32(x))
|
||||
}
|
||||
|
||||
func (InsertDataVersion) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor_41f4a519b878ee3b, []int{1}
|
||||
}
|
||||
|
||||
type ComponentInfo struct {
|
||||
NodeID int64 `protobuf:"varint,1,opt,name=nodeID,proto3" json:"nodeID,omitempty"`
|
||||
Role string `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"`
|
||||
|
@ -1106,21 +1132,25 @@ func (m *CreateIndexRequest) GetExtraParams() []*commonpb.KeyValuePair {
|
|||
}
|
||||
|
||||
type InsertRequest struct {
|
||||
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
|
||||
ShardName string `protobuf:"bytes,2,opt,name=shardName,proto3" json:"shardName,omitempty"`
|
||||
DbName string `protobuf:"bytes,3,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
|
||||
CollectionName string `protobuf:"bytes,4,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
|
||||
PartitionName string `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
|
||||
DbID int64 `protobuf:"varint,6,opt,name=dbID,proto3" json:"dbID,omitempty"`
|
||||
CollectionID int64 `protobuf:"varint,7,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
|
||||
PartitionID int64 `protobuf:"varint,8,opt,name=partitionID,proto3" json:"partitionID,omitempty"`
|
||||
SegmentID int64 `protobuf:"varint,9,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
|
||||
Timestamps []uint64 `protobuf:"varint,10,rep,packed,name=timestamps,proto3" json:"timestamps,omitempty"`
|
||||
RowIDs []int64 `protobuf:"varint,11,rep,packed,name=rowIDs,proto3" json:"rowIDs,omitempty"`
|
||||
RowData []*commonpb.Blob `protobuf:"bytes,12,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
|
||||
ShardName string `protobuf:"bytes,2,opt,name=shardName,proto3" json:"shardName,omitempty"`
|
||||
DbName string `protobuf:"bytes,3,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"`
|
||||
CollectionName string `protobuf:"bytes,4,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
|
||||
PartitionName string `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"`
|
||||
DbID int64 `protobuf:"varint,6,opt,name=dbID,proto3" json:"dbID,omitempty"`
|
||||
CollectionID int64 `protobuf:"varint,7,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
|
||||
PartitionID int64 `protobuf:"varint,8,opt,name=partitionID,proto3" json:"partitionID,omitempty"`
|
||||
SegmentID int64 `protobuf:"varint,9,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
|
||||
Timestamps []uint64 `protobuf:"varint,10,rep,packed,name=timestamps,proto3" json:"timestamps,omitempty"`
|
||||
RowIDs []int64 `protobuf:"varint,11,rep,packed,name=rowIDs,proto3" json:"rowIDs,omitempty"`
|
||||
// row_data was reserved for compatibility
|
||||
RowData []*commonpb.Blob `protobuf:"bytes,12,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"`
|
||||
FieldsData []*schemapb.FieldData `protobuf:"bytes,13,rep,name=fields_data,json=fieldsData,proto3" json:"fields_data,omitempty"`
|
||||
NumRows uint64 `protobuf:"varint,14,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"`
|
||||
Version InsertDataVersion `protobuf:"varint,15,opt,name=version,proto3,enum=milvus.proto.internal.InsertDataVersion" json:"version,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *InsertRequest) Reset() { *m = InsertRequest{} }
|
||||
|
@ -1232,6 +1262,27 @@ func (m *InsertRequest) GetRowData() []*commonpb.Blob {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (m *InsertRequest) GetFieldsData() []*schemapb.FieldData {
|
||||
if m != nil {
|
||||
return m.FieldsData
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *InsertRequest) GetNumRows() uint64 {
|
||||
if m != nil {
|
||||
return m.NumRows
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *InsertRequest) GetVersion() InsertDataVersion {
|
||||
if m != nil {
|
||||
return m.Version
|
||||
}
|
||||
return InsertDataVersion_RowBased
|
||||
}
|
||||
|
||||
type SearchRequest struct {
|
||||
Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"`
|
||||
ResultChannelID string `protobuf:"bytes,2,opt,name=result_channelID,json=resultChannelID,proto3" json:"result_channelID,omitempty"`
|
||||
|
@ -2240,6 +2291,7 @@ func (m *ChannelTimeTickMsg) GetDefaultTimestamp() uint64 {
|
|||
|
||||
func init() {
|
||||
proto.RegisterEnum("milvus.proto.internal.StateCode", StateCode_name, StateCode_value)
|
||||
proto.RegisterEnum("milvus.proto.internal.InsertDataVersion", InsertDataVersion_name, InsertDataVersion_value)
|
||||
proto.RegisterType((*ComponentInfo)(nil), "milvus.proto.internal.ComponentInfo")
|
||||
proto.RegisterType((*ComponentStates)(nil), "milvus.proto.internal.ComponentStates")
|
||||
proto.RegisterType((*GetComponentStatesRequest)(nil), "milvus.proto.internal.GetComponentStatesRequest")
|
||||
|
@ -2276,123 +2328,128 @@ func init() {
|
|||
func init() { proto.RegisterFile("internal.proto", fileDescriptor_41f4a519b878ee3b) }
|
||||
|
||||
var fileDescriptor_41f4a519b878ee3b = []byte{
|
||||
// 1886 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0x4f, 0x73, 0x1c, 0x47,
|
||||
0x15, 0x67, 0x76, 0x56, 0xda, 0xdd, 0xb7, 0x2b, 0x79, 0xdd, 0x92, 0x9d, 0xf1, 0x9f, 0xc4, 0xca,
|
||||
0x24, 0x80, 0x88, 0x0b, 0xdb, 0x28, 0x40, 0x52, 0x14, 0x85, 0x63, 0x69, 0xc1, 0x6c, 0x39, 0x36,
|
||||
0x62, 0x64, 0x52, 0x05, 0x97, 0xa9, 0xde, 0x9d, 0xd6, 0x6a, 0xf0, 0xcc, 0xf4, 0xa4, 0xbb, 0x47,
|
||||
0xf2, 0xfa, 0xc4, 0x81, 0x13, 0x14, 0x54, 0x71, 0xe0, 0x08, 0x37, 0xae, 0x70, 0xe4, 0x04, 0x54,
|
||||
0x71, 0xe2, 0x2b, 0xf0, 0x01, 0xf8, 0x12, 0x9c, 0xa8, 0x7e, 0xdd, 0x33, 0xfb, 0x47, 0x2b, 0x59,
|
||||
0x52, 0x2a, 0xc4, 0xa9, 0xca, 0xad, 0xfb, 0xf7, 0x5e, 0x77, 0xbf, 0xf7, 0x7b, 0xef, 0x75, 0xbf,
|
||||
0x9d, 0x85, 0xd5, 0x38, 0x53, 0x4c, 0x64, 0x34, 0xb9, 0x93, 0x0b, 0xae, 0x38, 0xb9, 0x92, 0xc6,
|
||||
0xc9, 0x61, 0x21, 0xcd, 0xec, 0x4e, 0x29, 0xbc, 0xde, 0x19, 0xf2, 0x34, 0xe5, 0x99, 0x81, 0xaf,
|
||||
0x77, 0xe4, 0xf0, 0x80, 0xa5, 0xd4, 0xcc, 0xfc, 0xbf, 0x3b, 0xb0, 0xb2, 0xc3, 0xd3, 0x9c, 0x67,
|
||||
0x2c, 0x53, 0xfd, 0x6c, 0x9f, 0x93, 0xab, 0xb0, 0x9c, 0xf1, 0x88, 0xf5, 0x7b, 0x9e, 0xb3, 0xe1,
|
||||
0x6c, 0xba, 0x81, 0x9d, 0x11, 0x02, 0x75, 0xc1, 0x13, 0xe6, 0xd5, 0x36, 0x9c, 0xcd, 0x56, 0x80,
|
||||
0x63, 0x72, 0x1f, 0x40, 0x2a, 0xaa, 0x58, 0x38, 0xe4, 0x11, 0xf3, 0xdc, 0x0d, 0x67, 0x73, 0x75,
|
||||
0x6b, 0xe3, 0xce, 0x42, 0x2b, 0xee, 0xec, 0x69, 0xc5, 0x1d, 0x1e, 0xb1, 0xa0, 0x25, 0xcb, 0x21,
|
||||
0xf9, 0x00, 0x80, 0x3d, 0x57, 0x82, 0x86, 0x71, 0xb6, 0xcf, 0xbd, 0xfa, 0x86, 0xbb, 0xd9, 0xde,
|
||||
0x7a, 0x73, 0x76, 0x03, 0x6b, 0xfc, 0x23, 0x36, 0xfe, 0x88, 0x26, 0x05, 0xdb, 0xa5, 0xb1, 0x08,
|
||||
0x5a, 0xb8, 0x48, 0x9b, 0xeb, 0xff, 0xdb, 0x81, 0x4b, 0x95, 0x03, 0x78, 0x86, 0x24, 0xdf, 0x81,
|
||||
0x25, 0x3c, 0x02, 0x3d, 0x68, 0x6f, 0xbd, 0x7d, 0x82, 0x45, 0x33, 0x7e, 0x07, 0x66, 0x09, 0xf9,
|
||||
0x09, 0xac, 0xc9, 0x62, 0x30, 0x2c, 0x45, 0x21, 0xa2, 0xd2, 0xab, 0xa1, 0x69, 0x67, 0xdb, 0x89,
|
||||
0x4c, 0x6f, 0x60, 0x4d, 0x7a, 0x17, 0x96, 0xf5, 0x4e, 0x85, 0x44, 0x96, 0xda, 0x5b, 0x37, 0x16,
|
||||
0x3a, 0xb9, 0x87, 0x2a, 0x81, 0x55, 0xf5, 0x6f, 0xc0, 0xb5, 0x87, 0x4c, 0xcd, 0x79, 0x17, 0xb0,
|
||||
0x8f, 0x0b, 0x26, 0x95, 0x15, 0x3e, 0x8d, 0x53, 0xf6, 0x34, 0x1e, 0x3e, 0xdb, 0x39, 0xa0, 0x59,
|
||||
0xc6, 0x92, 0x52, 0xf8, 0x3a, 0xdc, 0x78, 0xc8, 0x70, 0x41, 0x2c, 0x55, 0x3c, 0x94, 0x73, 0xe2,
|
||||
0x2b, 0xb0, 0xf6, 0x90, 0xa9, 0x5e, 0x34, 0x07, 0x7f, 0x04, 0xcd, 0x27, 0x3a, 0xd8, 0x3a, 0x0d,
|
||||
0xbe, 0x0d, 0x0d, 0x1a, 0x45, 0x82, 0x49, 0x69, 0x59, 0xbc, 0xb9, 0xd0, 0xe2, 0x07, 0x46, 0x27,
|
||||
0x28, 0x95, 0x17, 0xa5, 0x89, 0xff, 0x73, 0x80, 0x7e, 0x16, 0xab, 0x5d, 0x2a, 0x68, 0x2a, 0x4f,
|
||||
0x4c, 0xb0, 0x1e, 0x74, 0xa4, 0xa2, 0x42, 0x85, 0x39, 0xea, 0x59, 0xca, 0xcf, 0x90, 0x0d, 0x6d,
|
||||
0x5c, 0x66, 0x76, 0xf7, 0x7f, 0x0a, 0xb0, 0xa7, 0x44, 0x9c, 0x8d, 0x3e, 0x8c, 0xa5, 0xd2, 0x67,
|
||||
0x1d, 0x6a, 0x3d, 0xed, 0x84, 0xbb, 0xd9, 0x0a, 0xec, 0x6c, 0x2a, 0x1c, 0xb5, 0xb3, 0x87, 0xe3,
|
||||
0x3e, 0xb4, 0x4b, 0xba, 0x1f, 0xcb, 0x11, 0xb9, 0x07, 0xf5, 0x01, 0x95, 0xec, 0x54, 0x7a, 0x1e,
|
||||
0xcb, 0xd1, 0x36, 0x95, 0x2c, 0x40, 0x4d, 0xff, 0x57, 0x2e, 0xbc, 0xb6, 0x23, 0x18, 0x26, 0x7f,
|
||||
0x92, 0xb0, 0xa1, 0x8a, 0x79, 0x66, 0xb9, 0x3f, 0xff, 0x6e, 0xe4, 0x35, 0x68, 0x44, 0x83, 0x30,
|
||||
0xa3, 0x69, 0x49, 0xf6, 0x72, 0x34, 0x78, 0x42, 0x53, 0x46, 0xbe, 0x02, 0xab, 0xc3, 0x6a, 0x7f,
|
||||
0x8d, 0x60, 0xce, 0xb5, 0x82, 0x39, 0x94, 0xbc, 0x0d, 0x2b, 0x39, 0x15, 0x2a, 0xae, 0xd4, 0xea,
|
||||
0xa8, 0x36, 0x0b, 0xea, 0x80, 0x46, 0x83, 0x7e, 0xcf, 0x5b, 0xc2, 0x60, 0xe1, 0x98, 0xf8, 0xd0,
|
||||
0x99, 0xec, 0xd5, 0xef, 0x79, 0xcb, 0x28, 0x9b, 0xc1, 0xc8, 0x06, 0xb4, 0xab, 0x8d, 0xfa, 0x3d,
|
||||
0xaf, 0x81, 0x2a, 0xd3, 0x90, 0x0e, 0x8e, 0xb9, 0x8b, 0xbc, 0xe6, 0x86, 0xb3, 0xd9, 0x09, 0xec,
|
||||
0x8c, 0xdc, 0x83, 0xb5, 0xc3, 0x58, 0xa8, 0x82, 0x26, 0x36, 0x3f, 0xb5, 0x1d, 0xd2, 0x6b, 0x61,
|
||||
0x04, 0x17, 0x89, 0xc8, 0x16, 0xac, 0xe7, 0x07, 0x63, 0x19, 0x0f, 0xe7, 0x96, 0x00, 0x2e, 0x59,
|
||||
0x28, 0xf3, 0xff, 0xe9, 0xc0, 0x95, 0x9e, 0xe0, 0xf9, 0x2b, 0x11, 0x8a, 0x92, 0xe4, 0xfa, 0x29,
|
||||
0x24, 0x2f, 0x1d, 0x27, 0xd9, 0xff, 0x4d, 0x0d, 0xae, 0x9a, 0x8c, 0xda, 0x2d, 0x89, 0xfd, 0x14,
|
||||
0xbc, 0xf8, 0x2a, 0x5c, 0x9a, 0x9c, 0x6a, 0x14, 0x16, 0xbb, 0xf1, 0x65, 0x58, 0xad, 0x02, 0x6c,
|
||||
0xf4, 0xfe, 0xbf, 0x29, 0xe5, 0xff, 0xba, 0x06, 0xeb, 0x3a, 0xa8, 0x5f, 0xb0, 0xa1, 0xd9, 0xf8,
|
||||
0xa3, 0x03, 0xc4, 0x64, 0xc7, 0x83, 0x24, 0xa6, 0xf2, 0xb3, 0xe4, 0x62, 0x1d, 0x96, 0xa8, 0xb6,
|
||||
0xc1, 0x52, 0x60, 0x26, 0xbe, 0x84, 0xae, 0x8e, 0xd6, 0xa7, 0x65, 0x5d, 0x75, 0xa8, 0x3b, 0x7d,
|
||||
0xe8, 0x1f, 0x1c, 0xb8, 0xfc, 0x20, 0x51, 0x4c, 0xbc, 0xa2, 0xa4, 0xfc, 0xa3, 0x56, 0x46, 0xad,
|
||||
0x9f, 0x45, 0xec, 0xf9, 0x67, 0x69, 0xe0, 0xeb, 0x00, 0xfb, 0x31, 0x4b, 0xa2, 0xe9, 0xec, 0x6d,
|
||||
0x21, 0xf2, 0x89, 0x32, 0xd7, 0x83, 0x06, 0x6e, 0x52, 0x65, 0x6d, 0x39, 0xd5, 0x3d, 0x80, 0xe9,
|
||||
0x07, 0x6d, 0x0f, 0xd0, 0x3c, 0x73, 0x0f, 0x80, 0xcb, 0x6c, 0x0f, 0xf0, 0x67, 0x17, 0x56, 0xfa,
|
||||
0x99, 0x64, 0x42, 0x5d, 0x9c, 0xbc, 0x9b, 0xd0, 0x92, 0x07, 0x54, 0xa0, 0xa3, 0x96, 0xbe, 0x09,
|
||||
0x30, 0x4d, 0xad, 0xfb, 0x32, 0x6a, 0xeb, 0x67, 0xbc, 0x1c, 0x96, 0x4e, 0xbb, 0x1c, 0x96, 0x4f,
|
||||
0xa1, 0xb8, 0xf1, 0xf2, 0xcb, 0xa1, 0x79, 0xfc, 0xf5, 0xd5, 0x0e, 0xb2, 0x51, 0xaa, 0x9b, 0xd6,
|
||||
0x9e, 0xd7, 0x42, 0xf9, 0x04, 0x20, 0x6f, 0x00, 0xa8, 0x38, 0x65, 0x52, 0xd1, 0x34, 0x37, 0xef,
|
||||
0x68, 0x3d, 0x98, 0x42, 0xf4, 0xdb, 0x2d, 0xf8, 0x51, 0xbf, 0x27, 0xbd, 0xf6, 0x86, 0xab, 0x9b,
|
||||
0x38, 0x33, 0x23, 0xdf, 0x84, 0xa6, 0xe0, 0x47, 0x61, 0x44, 0x15, 0xf5, 0x3a, 0x18, 0xbc, 0x6b,
|
||||
0x0b, 0xc9, 0xde, 0x4e, 0xf8, 0x20, 0x68, 0x08, 0x7e, 0xd4, 0xa3, 0x8a, 0xfa, 0x7f, 0xaa, 0xc3,
|
||||
0xca, 0x1e, 0xa3, 0x62, 0x78, 0x70, 0xf1, 0x80, 0x7d, 0x0d, 0xba, 0x82, 0xc9, 0x22, 0x51, 0xe1,
|
||||
0xd0, 0x3c, 0xf3, 0xfd, 0x9e, 0x8d, 0xdb, 0x25, 0x83, 0xef, 0x94, 0x70, 0x45, 0xaa, 0x7b, 0x0a,
|
||||
0xa9, 0xf5, 0x05, 0xa4, 0xfa, 0xd0, 0x99, 0x62, 0x50, 0x7a, 0x4b, 0xe8, 0xfa, 0x0c, 0x46, 0xba,
|
||||
0xe0, 0x46, 0x32, 0xc1, 0x78, 0xb5, 0x02, 0x3d, 0x24, 0xb7, 0xe1, 0x72, 0x9e, 0xd0, 0x21, 0x3b,
|
||||
0xe0, 0x49, 0xc4, 0x44, 0x38, 0x12, 0xbc, 0xc8, 0x31, 0x66, 0x9d, 0xa0, 0x3b, 0x25, 0x78, 0xa8,
|
||||
0x71, 0xf2, 0x1e, 0x34, 0x23, 0x99, 0x84, 0x6a, 0x9c, 0x33, 0x0c, 0xda, 0xea, 0x09, 0xbe, 0xf7,
|
||||
0x64, 0xf2, 0x74, 0x9c, 0xb3, 0xa0, 0x11, 0x99, 0x01, 0xb9, 0x07, 0xeb, 0x92, 0x89, 0x98, 0x26,
|
||||
0xf1, 0x0b, 0x16, 0x85, 0xec, 0x79, 0x2e, 0xc2, 0x3c, 0xa1, 0x19, 0x46, 0xb6, 0x13, 0x90, 0x89,
|
||||
0xec, 0xfb, 0xcf, 0x73, 0xb1, 0x9b, 0xd0, 0x8c, 0x6c, 0x42, 0x97, 0x17, 0x2a, 0x2f, 0x54, 0x88,
|
||||
0xd5, 0x27, 0xc3, 0x38, 0xc2, 0x40, 0xbb, 0xc1, 0xaa, 0xc1, 0x7f, 0x80, 0x70, 0x3f, 0xd2, 0xd4,
|
||||
0x2a, 0x41, 0x0f, 0x59, 0x12, 0x56, 0x19, 0xe0, 0xb5, 0x37, 0x9c, 0xcd, 0x7a, 0x70, 0xc9, 0xe0,
|
||||
0x4f, 0x4b, 0x98, 0xdc, 0x85, 0xb5, 0x51, 0x41, 0x05, 0xcd, 0x14, 0x63, 0x53, 0xda, 0x1d, 0xd4,
|
||||
0x26, 0x95, 0x68, 0xb2, 0xe0, 0x36, 0x5c, 0xd6, 0x6a, 0xbc, 0x50, 0x53, 0xea, 0x2b, 0xa8, 0xde,
|
||||
0xb5, 0x82, 0x4a, 0xd9, 0xff, 0xdd, 0x54, 0x9e, 0xe8, 0x90, 0xca, 0x0b, 0xe4, 0xc9, 0x45, 0x5a,
|
||||
0xff, 0x85, 0xc9, 0xe5, 0x2e, 0x4e, 0xae, 0x5b, 0xd0, 0x4e, 0x99, 0x12, 0xf1, 0xd0, 0x04, 0xd1,
|
||||
0x54, 0x3f, 0x18, 0x08, 0x23, 0x75, 0x0b, 0xda, 0x59, 0x91, 0x86, 0x1f, 0x17, 0x4c, 0xc4, 0x4c,
|
||||
0xda, 0xcb, 0x13, 0xb2, 0x22, 0xfd, 0xb1, 0x41, 0xc8, 0x1a, 0x2c, 0x29, 0x9e, 0x87, 0xcf, 0xca,
|
||||
0xa2, 0x57, 0x3c, 0x7f, 0x44, 0xbe, 0x0b, 0xd7, 0x25, 0xa3, 0x09, 0x8b, 0xc2, 0xaa, 0x48, 0x65,
|
||||
0x28, 0x91, 0x0b, 0x16, 0x79, 0x0d, 0x8c, 0x9b, 0x67, 0x34, 0xf6, 0x2a, 0x85, 0x3d, 0x2b, 0xd7,
|
||||
0x61, 0xa9, 0x0c, 0x9f, 0x5a, 0xd6, 0xc4, 0xfe, 0x98, 0x4c, 0x44, 0xd5, 0x82, 0xf7, 0xc1, 0x1b,
|
||||
0x25, 0x7c, 0x40, 0x93, 0xf0, 0xd8, 0xa9, 0xd8, 0x88, 0xbb, 0xc1, 0x55, 0x23, 0xdf, 0x9b, 0x3b,
|
||||
0x52, 0xbb, 0x27, 0x93, 0x78, 0xc8, 0xa2, 0x70, 0x90, 0xf0, 0x81, 0x07, 0x98, 0x7f, 0x60, 0x20,
|
||||
0x5d, 0xf5, 0x3a, 0xef, 0xac, 0x82, 0xa6, 0x61, 0xc8, 0x8b, 0x4c, 0x61, 0x36, 0xb9, 0xc1, 0xaa,
|
||||
0xc1, 0x9f, 0x14, 0xe9, 0x8e, 0x46, 0xc9, 0x5b, 0xb0, 0x62, 0x35, 0xf9, 0xfe, 0xbe, 0x64, 0x0a,
|
||||
0xd3, 0xc8, 0x0d, 0x3a, 0x06, 0xfc, 0x11, 0x62, 0xfe, 0x5f, 0x5c, 0xb8, 0x14, 0x68, 0x76, 0xd9,
|
||||
0x21, 0xfb, 0xdc, 0xdf, 0x1e, 0x27, 0x55, 0xf1, 0xf2, 0xb9, 0xaa, 0xb8, 0x71, 0xe6, 0x2a, 0x6e,
|
||||
0x9e, 0xab, 0x8a, 0x5b, 0xe7, 0xab, 0x62, 0x38, 0xa1, 0x8a, 0xff, 0x36, 0x13, 0xb1, 0x57, 0xb5,
|
||||
0x8e, 0xdf, 0x01, 0x37, 0x8e, 0x4c, 0x6b, 0xd6, 0xde, 0xf2, 0x66, 0x37, 0xb7, 0x9f, 0xd0, 0xfa,
|
||||
0x3d, 0x19, 0x68, 0x25, 0x72, 0x1f, 0xda, 0x96, 0x7d, 0x7c, 0xf8, 0x96, 0xf0, 0xe1, 0x7b, 0x63,
|
||||
0xe1, 0x1a, 0x0c, 0x87, 0x7e, 0xf4, 0x02, 0xd3, 0x5a, 0x49, 0x3d, 0x26, 0xdf, 0x83, 0x1b, 0xc7,
|
||||
0xab, 0x5b, 0x58, 0x8e, 0x22, 0x6f, 0x19, 0x03, 0x7a, 0x6d, 0xbe, 0xbc, 0x4b, 0x12, 0x23, 0xf2,
|
||||
0x0d, 0x58, 0x9f, 0xaa, 0xef, 0xc9, 0xc2, 0x86, 0xf9, 0xcd, 0x3c, 0x91, 0x4d, 0x96, 0x9c, 0x56,
|
||||
0xe1, 0xcd, 0xd3, 0x2a, 0xdc, 0xff, 0x4f, 0x0d, 0x56, 0x7a, 0x2c, 0x61, 0x8a, 0x7d, 0xd1, 0x5e,
|
||||
0x9d, 0xd8, 0x5e, 0xbd, 0x09, 0x9d, 0x5c, 0xc4, 0x29, 0x15, 0xe3, 0xf0, 0x19, 0x1b, 0x97, 0x97,
|
||||
0x66, 0xdb, 0x62, 0x8f, 0xd8, 0x58, 0xbe, 0xac, 0xc7, 0xf2, 0xff, 0xeb, 0x40, 0xeb, 0x43, 0x4e,
|
||||
0x23, 0xfc, 0x19, 0x70, 0x41, 0x8e, 0xab, 0x0e, 0xaf, 0x36, 0xdf, 0xe1, 0xdd, 0x84, 0x49, 0x27,
|
||||
0x6f, 0x59, 0x9e, 0x6a, 0xed, 0xa7, 0x5a, 0xf4, 0xfa, 0x6c, 0x8b, 0x7e, 0x0b, 0xda, 0xb1, 0x36,
|
||||
0x28, 0xcc, 0xa9, 0x3a, 0x30, 0xb7, 0x58, 0x2b, 0x00, 0x84, 0x76, 0x35, 0xa2, 0x7b, 0xf8, 0x52,
|
||||
0x01, 0x7b, 0xf8, 0xe5, 0x33, 0xf7, 0xf0, 0x76, 0x13, 0xec, 0xe1, 0x7f, 0xe9, 0x00, 0xa0, 0xe3,
|
||||
0xba, 0x82, 0x8f, 0x6f, 0xea, 0x5c, 0x64, 0x53, 0x7d, 0xbd, 0xea, 0x37, 0x47, 0xb0, 0x84, 0xaa,
|
||||
0x49, 0xc6, 0x4b, 0x4b, 0x0e, 0xc9, 0x8a, 0x34, 0x30, 0x22, 0x9b, 0xed, 0xd2, 0xff, 0xad, 0x03,
|
||||
0x80, 0x25, 0x6b, 0xcc, 0x98, 0xcf, 0x0d, 0xe7, 0xf4, 0x5f, 0x37, 0xb5, 0x59, 0xea, 0xb6, 0x4b,
|
||||
0xea, 0xf4, 0x6d, 0xa4, 0x7f, 0x95, 0x2e, 0xf0, 0xa1, 0xfa, 0xa6, 0x3c, 0x71, 0xde, 0xb2, 0x8b,
|
||||
0x63, 0xff, 0xf7, 0x0e, 0x74, 0xac, 0x75, 0xc6, 0xa4, 0x99, 0x28, 0x3b, 0xf3, 0x51, 0xc6, 0x6e,
|
||||
0x24, 0xe5, 0x62, 0x1c, 0xca, 0xf8, 0x05, 0xb3, 0x06, 0x81, 0x81, 0xf6, 0xe2, 0x17, 0x8c, 0x5c,
|
||||
0x83, 0x26, 0x52, 0xc2, 0x8f, 0xa4, 0x7d, 0xd1, 0x1a, 0x9a, 0x06, 0x7e, 0x24, 0xf5, 0xa5, 0x2e,
|
||||
0xd8, 0x90, 0x65, 0x2a, 0x19, 0x87, 0x29, 0x8f, 0xe2, 0xfd, 0x98, 0x45, 0x98, 0x0d, 0xcd, 0xa0,
|
||||
0x5b, 0x0a, 0x1e, 0x5b, 0xdc, 0xff, 0x97, 0x03, 0xab, 0xba, 0x81, 0x19, 0x3f, 0xe1, 0x11, 0x33,
|
||||
0x96, 0x9d, 0x3f, 0x63, 0x3f, 0x40, 0x5f, 0x2c, 0x3d, 0xe6, 0xfb, 0xef, 0x5b, 0x27, 0xfd, 0x9d,
|
||||
0x30, 0xc5, 0x41, 0xd0, 0x94, 0x6c, 0x64, 0xce, 0xdc, 0xb6, 0x37, 0xf1, 0x99, 0x28, 0x9e, 0x04,
|
||||
0xd6, 0x5e, 0xc6, 0x86, 0xe2, 0x5f, 0x38, 0xd0, 0x7e, 0x2c, 0x47, 0xbb, 0x5c, 0x62, 0x31, 0xeb,
|
||||
0x52, 0xb6, 0x17, 0xa8, 0xb9, 0x49, 0x1c, 0x2c, 0x96, 0xf6, 0x70, 0xf2, 0x35, 0x51, 0xff, 0x92,
|
||||
0x4f, 0xe5, 0xc8, 0x46, 0xbc, 0x13, 0x98, 0x09, 0xb9, 0x0e, 0xcd, 0x54, 0x8e, 0xb0, 0xb1, 0xb7,
|
||||
0x15, 0x56, 0xcd, 0x75, 0xd8, 0x26, 0x2f, 0x65, 0x1d, 0x5f, 0xca, 0x09, 0xe0, 0xff, 0xd5, 0x01,
|
||||
0x62, 0x9f, 0xa2, 0x4f, 0xf4, 0xc9, 0x19, 0x13, 0x76, 0xfa, 0x8b, 0x68, 0x0d, 0xcb, 0x75, 0x06,
|
||||
0x9b, 0xbb, 0x87, 0xdc, 0x63, 0xbf, 0xf5, 0x6e, 0xc3, 0xe5, 0x88, 0xed, 0x53, 0xfd, 0x6a, 0xce,
|
||||
0x9b, 0xdc, 0xb5, 0x82, 0xea, 0x71, 0x7f, 0xe7, 0x7d, 0x68, 0x55, 0xff, 0xf4, 0x90, 0x2e, 0x74,
|
||||
0xfa, 0x59, 0xac, 0xb0, 0x67, 0x89, 0xb3, 0x51, 0xf7, 0x4b, 0xa4, 0x0d, 0x8d, 0x1f, 0x32, 0x9a,
|
||||
0xa8, 0x83, 0x71, 0xd7, 0x21, 0x1d, 0x68, 0x3e, 0x18, 0x64, 0x5c, 0xa4, 0x34, 0xe9, 0xd6, 0xb6,
|
||||
0xdf, 0xfb, 0xd9, 0xb7, 0x46, 0xb1, 0x3a, 0x28, 0x06, 0xda, 0x93, 0xbb, 0xc6, 0xb5, 0xaf, 0xc7,
|
||||
0xdc, 0x8e, 0xee, 0x96, 0x51, 0xbb, 0x8b, 0xde, 0x56, 0xd3, 0x7c, 0x30, 0x58, 0x46, 0xe4, 0xdd,
|
||||
0xff, 0x05, 0x00, 0x00, 0xff, 0xff, 0x48, 0xeb, 0xcc, 0x3e, 0x0f, 0x1b, 0x00, 0x00,
|
||||
// 1954 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x59, 0x4f, 0x73, 0x1b, 0x49,
|
||||
0x15, 0xdf, 0x91, 0x64, 0x4b, 0x7a, 0x23, 0xdb, 0x72, 0xc7, 0xc9, 0x4e, 0xfe, 0xec, 0xc6, 0x3b,
|
||||
0xbb, 0x80, 0x49, 0x8a, 0x24, 0x78, 0x81, 0xdd, 0xa2, 0x28, 0xb2, 0xb1, 0x05, 0x41, 0x95, 0x4d,
|
||||
0x30, 0xe3, 0x90, 0x2a, 0xb8, 0x4c, 0xb5, 0x34, 0x6d, 0xa9, 0xc9, 0xcc, 0xf4, 0x6c, 0x77, 0x8f,
|
||||
0x1d, 0xe5, 0xc4, 0x81, 0x13, 0x14, 0x54, 0x71, 0xe0, 0x08, 0x37, 0xce, 0x1c, 0x39, 0x01, 0x55,
|
||||
0x9c, 0x38, 0x71, 0xe7, 0x03, 0xf0, 0x25, 0x38, 0x51, 0xfd, 0x67, 0x46, 0x23, 0x59, 0x76, 0x6c,
|
||||
0x6f, 0x2d, 0x1b, 0xaa, 0xf6, 0x36, 0xfd, 0xde, 0xeb, 0xee, 0xd7, 0xbf, 0xf7, 0x7b, 0xaf, 0x5f,
|
||||
0x4b, 0xb0, 0x4a, 0x53, 0x49, 0x78, 0x8a, 0xe3, 0x3b, 0x19, 0x67, 0x92, 0xa1, 0xcb, 0x09, 0x8d,
|
||||
0x0f, 0x73, 0x61, 0x46, 0x77, 0x0a, 0xe5, 0xb5, 0xce, 0x90, 0x25, 0x09, 0x4b, 0x8d, 0xf8, 0x5a,
|
||||
0x47, 0x0c, 0xc7, 0x24, 0xc1, 0x66, 0xe4, 0xff, 0xd5, 0x81, 0x95, 0x5d, 0x96, 0x64, 0x2c, 0x25,
|
||||
0xa9, 0xec, 0xa7, 0x07, 0x0c, 0x5d, 0x81, 0xe5, 0x94, 0x45, 0xa4, 0xdf, 0xf3, 0x9c, 0x4d, 0x67,
|
||||
0xab, 0x1e, 0xd8, 0x11, 0x42, 0xd0, 0xe0, 0x2c, 0x26, 0x5e, 0x6d, 0xd3, 0xd9, 0x6a, 0x07, 0xfa,
|
||||
0x1b, 0xdd, 0x07, 0x10, 0x12, 0x4b, 0x12, 0x0e, 0x59, 0x44, 0xbc, 0xfa, 0xa6, 0xb3, 0xb5, 0xba,
|
||||
0xbd, 0x79, 0x67, 0xa1, 0x17, 0x77, 0xf6, 0x95, 0xe1, 0x2e, 0x8b, 0x48, 0xd0, 0x16, 0xc5, 0x27,
|
||||
0xfa, 0x08, 0x80, 0xbc, 0x90, 0x1c, 0x87, 0x34, 0x3d, 0x60, 0x5e, 0x63, 0xb3, 0xbe, 0xe5, 0x6e,
|
||||
0xbf, 0x33, 0xbb, 0x80, 0x75, 0xfe, 0x11, 0x99, 0x3c, 0xc3, 0x71, 0x4e, 0xf6, 0x30, 0xe5, 0x41,
|
||||
0x5b, 0x4f, 0x52, 0xee, 0xfa, 0xff, 0x72, 0x60, 0xad, 0x3c, 0x80, 0xde, 0x43, 0xa0, 0x6f, 0xc3,
|
||||
0x92, 0xde, 0x42, 0x9f, 0xc0, 0xdd, 0x7e, 0xef, 0x04, 0x8f, 0x66, 0xce, 0x1d, 0x98, 0x29, 0xe8,
|
||||
0xc7, 0x70, 0x49, 0xe4, 0x83, 0x61, 0xa1, 0x0a, 0xb5, 0x54, 0x78, 0x35, 0xed, 0xda, 0xd9, 0x56,
|
||||
0x42, 0xd5, 0x05, 0xac, 0x4b, 0xef, 0xc3, 0xb2, 0x5a, 0x29, 0x17, 0x1a, 0x25, 0x77, 0xfb, 0xfa,
|
||||
0xc2, 0x43, 0xee, 0x6b, 0x93, 0xc0, 0x9a, 0xfa, 0xd7, 0xe1, 0xea, 0x43, 0x22, 0xe7, 0x4e, 0x17,
|
||||
0x90, 0x4f, 0x72, 0x22, 0xa4, 0x55, 0x3e, 0xa5, 0x09, 0x79, 0x4a, 0x87, 0xcf, 0x77, 0xc7, 0x38,
|
||||
0x4d, 0x49, 0x5c, 0x28, 0xdf, 0x82, 0xeb, 0x0f, 0x89, 0x9e, 0x40, 0x85, 0xa4, 0x43, 0x31, 0xa7,
|
||||
0xbe, 0x0c, 0x97, 0x1e, 0x12, 0xd9, 0x8b, 0xe6, 0xc4, 0xcf, 0xa0, 0xf5, 0x44, 0x05, 0x5b, 0xd1,
|
||||
0xe0, 0x5b, 0xd0, 0xc4, 0x51, 0xc4, 0x89, 0x10, 0x16, 0xc5, 0x1b, 0x0b, 0x3d, 0x7e, 0x60, 0x6c,
|
||||
0x82, 0xc2, 0x78, 0x11, 0x4d, 0xfc, 0x9f, 0x01, 0xf4, 0x53, 0x2a, 0xf7, 0x30, 0xc7, 0x89, 0x38,
|
||||
0x91, 0x60, 0x3d, 0xe8, 0x08, 0x89, 0xb9, 0x0c, 0x33, 0x6d, 0x67, 0x21, 0x3f, 0x03, 0x1b, 0x5c,
|
||||
0x3d, 0xcd, 0xac, 0xee, 0xff, 0x04, 0x60, 0x5f, 0x72, 0x9a, 0x8e, 0x3e, 0xa6, 0x42, 0xaa, 0xbd,
|
||||
0x0e, 0x95, 0x9d, 0x3a, 0x44, 0x7d, 0xab, 0x1d, 0xd8, 0x51, 0x25, 0x1c, 0xb5, 0xb3, 0x87, 0xe3,
|
||||
0x3e, 0xb8, 0x05, 0xdc, 0x8f, 0xc5, 0x08, 0xdd, 0x83, 0xc6, 0x00, 0x0b, 0x72, 0x2a, 0x3c, 0x8f,
|
||||
0xc5, 0x68, 0x07, 0x0b, 0x12, 0x68, 0x4b, 0xff, 0x97, 0x75, 0x78, 0x73, 0x97, 0x13, 0x4d, 0xfe,
|
||||
0x38, 0x26, 0x43, 0x49, 0x59, 0x6a, 0xb1, 0x3f, 0xff, 0x6a, 0xe8, 0x4d, 0x68, 0x46, 0x83, 0x30,
|
||||
0xc5, 0x49, 0x01, 0xf6, 0x72, 0x34, 0x78, 0x82, 0x13, 0x82, 0xbe, 0x0c, 0xab, 0xc3, 0x72, 0x7d,
|
||||
0x25, 0xd1, 0x9c, 0x6b, 0x07, 0x73, 0x52, 0xf4, 0x1e, 0xac, 0x64, 0x98, 0x4b, 0x5a, 0x9a, 0x35,
|
||||
0xb4, 0xd9, 0xac, 0x50, 0x05, 0x34, 0x1a, 0xf4, 0x7b, 0xde, 0x92, 0x0e, 0x96, 0xfe, 0x46, 0x3e,
|
||||
0x74, 0xa6, 0x6b, 0xf5, 0x7b, 0xde, 0xb2, 0xd6, 0xcd, 0xc8, 0xd0, 0x26, 0xb8, 0xe5, 0x42, 0xfd,
|
||||
0x9e, 0xd7, 0xd4, 0x26, 0x55, 0x91, 0x0a, 0x8e, 0xa9, 0x45, 0x5e, 0x6b, 0xd3, 0xd9, 0xea, 0x04,
|
||||
0x76, 0x84, 0xee, 0xc1, 0xa5, 0x43, 0xca, 0x65, 0x8e, 0x63, 0xcb, 0x4f, 0xe5, 0x87, 0xf0, 0xda,
|
||||
0x3a, 0x82, 0x8b, 0x54, 0x68, 0x1b, 0x36, 0xb2, 0xf1, 0x44, 0xd0, 0xe1, 0xdc, 0x14, 0xd0, 0x53,
|
||||
0x16, 0xea, 0xfc, 0xbf, 0x3b, 0x70, 0xb9, 0xc7, 0x59, 0xf6, 0x5a, 0x84, 0xa2, 0x00, 0xb9, 0x71,
|
||||
0x0a, 0xc8, 0x4b, 0xc7, 0x41, 0xf6, 0x7f, 0x5d, 0x83, 0x2b, 0x86, 0x51, 0x7b, 0x05, 0xb0, 0x9f,
|
||||
0xc1, 0x29, 0xbe, 0x02, 0x6b, 0xd3, 0x5d, 0x8d, 0xc1, 0xe2, 0x63, 0x7c, 0x09, 0x56, 0xcb, 0x00,
|
||||
0x1b, 0xbb, 0xff, 0x2d, 0xa5, 0xfc, 0x5f, 0xd5, 0x60, 0x43, 0x05, 0xf5, 0x0b, 0x34, 0x14, 0x1a,
|
||||
0x7f, 0x70, 0x00, 0x19, 0x76, 0x3c, 0x88, 0x29, 0x16, 0x9f, 0x27, 0x16, 0x1b, 0xb0, 0x84, 0x95,
|
||||
0x0f, 0x16, 0x02, 0x33, 0xf0, 0x05, 0x74, 0x55, 0xb4, 0x3e, 0x2b, 0xef, 0xca, 0x4d, 0xeb, 0xd5,
|
||||
0x4d, 0x7f, 0xef, 0xc0, 0xfa, 0x83, 0x58, 0x12, 0xfe, 0x9a, 0x82, 0xf2, 0xb7, 0x5a, 0x11, 0xb5,
|
||||
0x7e, 0x1a, 0x91, 0x17, 0x9f, 0xa7, 0x83, 0x6f, 0x01, 0x1c, 0x50, 0x12, 0x47, 0x55, 0xf6, 0xb6,
|
||||
0xb5, 0xe4, 0x53, 0x31, 0xd7, 0x83, 0xa6, 0x5e, 0xa4, 0x64, 0x6d, 0x31, 0x54, 0x3d, 0x80, 0xe9,
|
||||
0x07, 0x6d, 0x0f, 0xd0, 0x3a, 0x73, 0x0f, 0xa0, 0xa7, 0xd9, 0x1e, 0xe0, 0x9f, 0x0d, 0x58, 0xe9,
|
||||
0xa7, 0x82, 0x70, 0x79, 0x71, 0xf0, 0x6e, 0x40, 0x5b, 0x8c, 0x31, 0xd7, 0x07, 0xb5, 0xf0, 0x4d,
|
||||
0x05, 0x55, 0x68, 0xeb, 0xaf, 0x82, 0xb6, 0x71, 0xc6, 0xe2, 0xb0, 0x74, 0x5a, 0x71, 0x58, 0x3e,
|
||||
0x05, 0xe2, 0xe6, 0xab, 0x8b, 0x43, 0xeb, 0xf8, 0xed, 0xab, 0x0e, 0x48, 0x46, 0x89, 0x6a, 0x5a,
|
||||
0x7b, 0x5e, 0x5b, 0xeb, 0xa7, 0x02, 0xf4, 0x36, 0x80, 0xa4, 0x09, 0x11, 0x12, 0x27, 0x99, 0xb9,
|
||||
0x47, 0x1b, 0x41, 0x45, 0xa2, 0xee, 0x6e, 0xce, 0x8e, 0xfa, 0x3d, 0xe1, 0xb9, 0x9b, 0x75, 0xd5,
|
||||
0xc4, 0x99, 0x11, 0xfa, 0x06, 0xb4, 0x38, 0x3b, 0x0a, 0x23, 0x2c, 0xb1, 0xd7, 0xd1, 0xc1, 0xbb,
|
||||
0xba, 0x10, 0xec, 0x9d, 0x98, 0x0d, 0x82, 0x26, 0x67, 0x47, 0x3d, 0x2c, 0x31, 0xba, 0x0f, 0xae,
|
||||
0x66, 0x80, 0x30, 0x13, 0x57, 0xf4, 0xc4, 0xb7, 0x67, 0x27, 0xda, 0x67, 0xcb, 0xf7, 0x95, 0x9d,
|
||||
0x9a, 0x14, 0x18, 0x6a, 0x0a, 0xbd, 0xc0, 0x55, 0x68, 0xa5, 0x79, 0x12, 0x72, 0x76, 0x24, 0xbc,
|
||||
0xd5, 0x4d, 0x67, 0xab, 0x11, 0x34, 0xd3, 0x3c, 0x09, 0xd8, 0x91, 0x40, 0x3b, 0xd0, 0x3c, 0x24,
|
||||
0x5c, 0x50, 0x96, 0x7a, 0x6b, 0xfa, 0x81, 0xb2, 0x75, 0x42, 0x13, 0x6f, 0x18, 0xa3, 0x96, 0x7b,
|
||||
0x66, 0xec, 0x83, 0x62, 0xa2, 0xff, 0xc7, 0x06, 0xac, 0xec, 0x13, 0xcc, 0x87, 0xe3, 0x8b, 0x13,
|
||||
0xea, 0xab, 0xd0, 0xe5, 0x44, 0xe4, 0xb1, 0x0c, 0x87, 0xa6, 0x0d, 0xe9, 0xf7, 0x2c, 0xaf, 0xd6,
|
||||
0x8c, 0x7c, 0xb7, 0x10, 0x97, 0x41, 0xaf, 0x9f, 0x12, 0xf4, 0xc6, 0x82, 0xa0, 0xfb, 0xd0, 0xa9,
|
||||
0x44, 0x58, 0x78, 0x4b, 0x3a, 0x34, 0x33, 0x32, 0xd4, 0x85, 0x7a, 0x24, 0x62, 0xcd, 0xa7, 0x76,
|
||||
0xa0, 0x3e, 0xd1, 0x6d, 0x58, 0xcf, 0x62, 0x3c, 0x24, 0x63, 0x16, 0x47, 0x84, 0x87, 0x23, 0xce,
|
||||
0xf2, 0x4c, 0x73, 0xaa, 0x13, 0x74, 0x2b, 0x8a, 0x87, 0x4a, 0x8e, 0x3e, 0x80, 0x56, 0x24, 0xe2,
|
||||
0x50, 0x4e, 0x32, 0xa2, 0x49, 0xb5, 0x7a, 0xc2, 0xd9, 0x7b, 0x22, 0x7e, 0x3a, 0xc9, 0x48, 0xd0,
|
||||
0x8c, 0xcc, 0x07, 0xba, 0x07, 0x1b, 0x82, 0x70, 0x8a, 0x63, 0xfa, 0x92, 0x44, 0x21, 0x79, 0x91,
|
||||
0xf1, 0x30, 0x8b, 0x71, 0xaa, 0x99, 0xd7, 0x09, 0xd0, 0x54, 0xf7, 0xbd, 0x17, 0x19, 0xdf, 0x8b,
|
||||
0x71, 0x8a, 0xb6, 0xa0, 0xcb, 0x72, 0x99, 0xe5, 0x32, 0xb4, 0xdc, 0xa0, 0x91, 0x26, 0x62, 0x3d,
|
||||
0x58, 0x35, 0x72, 0x4d, 0x05, 0xd1, 0x8f, 0x14, 0xb4, 0x92, 0xe3, 0x43, 0x12, 0x87, 0x25, 0x43,
|
||||
0x3d, 0x57, 0xb3, 0x60, 0xcd, 0xc8, 0x9f, 0x16, 0x62, 0x74, 0x17, 0x2e, 0x8d, 0x72, 0xcc, 0x71,
|
||||
0x2a, 0x09, 0xa9, 0x58, 0x77, 0xb4, 0x35, 0x2a, 0x55, 0xd3, 0x09, 0xb7, 0x61, 0x5d, 0x99, 0xb1,
|
||||
0x5c, 0x56, 0xcc, 0x57, 0xb4, 0x79, 0xd7, 0x2a, 0x4a, 0x63, 0xff, 0xb7, 0x15, 0x9e, 0xa8, 0x90,
|
||||
0x8a, 0x0b, 0xf0, 0xe4, 0x22, 0x4f, 0x93, 0x85, 0xe4, 0xaa, 0x2f, 0x26, 0xd7, 0x4d, 0x70, 0x13,
|
||||
0x22, 0x39, 0x1d, 0x9a, 0x20, 0x9a, 0xea, 0x04, 0x46, 0xa4, 0x23, 0x75, 0x13, 0x5c, 0x95, 0x4b,
|
||||
0x9f, 0xe4, 0x84, 0x53, 0x22, 0x6c, 0x71, 0x87, 0x34, 0x4f, 0x7e, 0x64, 0x24, 0xe8, 0x12, 0x2c,
|
||||
0x49, 0x96, 0x85, 0xcf, 0x8b, 0xa2, 0x24, 0x59, 0xf6, 0x08, 0x7d, 0x07, 0xae, 0x09, 0x82, 0x63,
|
||||
0x12, 0x85, 0x65, 0x11, 0x11, 0xa1, 0xd0, 0x58, 0x90, 0xc8, 0x6b, 0xea, 0xb8, 0x79, 0xc6, 0x62,
|
||||
0xbf, 0x34, 0xd8, 0xb7, 0x7a, 0x15, 0x96, 0xd2, 0xf1, 0xca, 0xb4, 0x96, 0xee, 0xdf, 0xd1, 0x54,
|
||||
0x55, 0x4e, 0xf8, 0x10, 0xbc, 0x51, 0xcc, 0x06, 0x38, 0x0e, 0x8f, 0xed, 0xaa, 0x1f, 0x0a, 0xf5,
|
||||
0xe0, 0x8a, 0xd1, 0xef, 0xcf, 0x6d, 0xa9, 0x8e, 0x27, 0x62, 0x3a, 0x24, 0x51, 0x38, 0x88, 0xd9,
|
||||
0xc0, 0x03, 0xcd, 0x3f, 0x30, 0x22, 0x55, 0x95, 0x14, 0xef, 0xac, 0x81, 0x82, 0x61, 0xc8, 0xf2,
|
||||
0x54, 0x6a, 0x36, 0xd5, 0x83, 0x55, 0x23, 0x7f, 0x92, 0x27, 0xbb, 0x4a, 0x8a, 0xde, 0x85, 0x15,
|
||||
0x6b, 0xc9, 0x0e, 0x0e, 0x04, 0x91, 0x9a, 0x46, 0xf5, 0xa0, 0x63, 0x84, 0x3f, 0xd4, 0x32, 0xff,
|
||||
0x4f, 0x75, 0x58, 0x0b, 0x14, 0xba, 0xe4, 0x90, 0xfc, 0xdf, 0x57, 0x8f, 0x93, 0xb2, 0x78, 0xf9,
|
||||
0x5c, 0x59, 0xdc, 0x3c, 0x73, 0x16, 0xb7, 0xce, 0x95, 0xc5, 0xed, 0xf3, 0x65, 0x31, 0x9c, 0x90,
|
||||
0xc5, 0x7f, 0x99, 0x89, 0xd8, 0xeb, 0x9a, 0xc7, 0xb7, 0xa0, 0x4e, 0x23, 0xd3, 0x3a, 0xba, 0xdb,
|
||||
0xde, 0xc2, 0xbb, 0xb2, 0xdf, 0x13, 0x81, 0x32, 0x9a, 0xbf, 0x5f, 0x97, 0xce, 0x7d, 0xbf, 0x7e,
|
||||
0x17, 0xae, 0x1f, 0xcf, 0x6e, 0x6e, 0x31, 0x8a, 0xbc, 0x65, 0x1d, 0xd0, 0xab, 0xf3, 0xe9, 0x5d,
|
||||
0x80, 0x18, 0xa1, 0xaf, 0xc3, 0x46, 0x25, 0xbf, 0xa7, 0x13, 0x9b, 0xe6, 0x4d, 0x3f, 0xd5, 0x4d,
|
||||
0xa7, 0x9c, 0x96, 0xe1, 0xad, 0xd3, 0x32, 0xdc, 0xff, 0x77, 0x0d, 0x56, 0x7a, 0x24, 0x26, 0x92,
|
||||
0x7c, 0xd1, 0xfe, 0x9d, 0xd8, 0xfe, 0xbd, 0x03, 0x9d, 0x8c, 0xd3, 0x04, 0xf3, 0x49, 0xf8, 0x9c,
|
||||
0x4c, 0x8a, 0xa2, 0xe9, 0x5a, 0xd9, 0x23, 0x32, 0x11, 0xaf, 0xea, 0x01, 0xfd, 0xff, 0x38, 0xd0,
|
||||
0xfe, 0x98, 0xe1, 0x48, 0x3f, 0x53, 0x2e, 0x88, 0x71, 0xd9, 0x81, 0xd6, 0xe6, 0x3b, 0xd0, 0x1b,
|
||||
0x30, 0x7d, 0x69, 0x58, 0x94, 0x2b, 0x4f, 0x8f, 0xca, 0x13, 0xa2, 0x31, 0xfb, 0x84, 0xb8, 0x09,
|
||||
0x2e, 0x55, 0x0e, 0x85, 0x19, 0x96, 0x63, 0x53, 0xc5, 0xda, 0x01, 0x68, 0xd1, 0x9e, 0x92, 0xa8,
|
||||
0x37, 0x46, 0x61, 0xa0, 0xdf, 0x18, 0xcb, 0x67, 0x7e, 0x63, 0xd8, 0x45, 0xf4, 0x1b, 0xe3, 0x17,
|
||||
0x0e, 0x80, 0x3e, 0xb8, 0xca, 0xe0, 0xe3, 0x8b, 0x3a, 0x17, 0x59, 0x54, 0x95, 0x57, 0xdd, 0xc6,
|
||||
0x92, 0x18, 0xcb, 0x29, 0xe3, 0x85, 0x05, 0x07, 0xa9, 0x96, 0xd6, 0xa8, 0x2c, 0xdb, 0x85, 0xff,
|
||||
0x1b, 0x07, 0x40, 0xa7, 0xac, 0x71, 0x63, 0x9e, 0x1b, 0xce, 0xe9, 0xaf, 0xaf, 0xda, 0x2c, 0x74,
|
||||
0x3b, 0x05, 0x74, 0xaa, 0x1a, 0xa9, 0x57, 0xf3, 0x82, 0x33, 0x54, 0xda, 0xe5, 0xe2, 0xf0, 0x16,
|
||||
0x5d, 0xfd, 0xed, 0xff, 0xce, 0x81, 0x8e, 0xf5, 0xce, 0xb8, 0x34, 0x13, 0x65, 0x67, 0x3e, 0xca,
|
||||
0xba, 0x1b, 0x49, 0x18, 0x9f, 0x84, 0x82, 0xbe, 0x24, 0xd6, 0x21, 0x30, 0xa2, 0x7d, 0xfa, 0x92,
|
||||
0xcc, 0x74, 0xf6, 0xe6, 0x46, 0x2b, 0x3b, 0xfb, 0xdb, 0xb0, 0xce, 0xc9, 0x90, 0xa4, 0x32, 0x9e,
|
||||
0x84, 0x09, 0x8b, 0xe8, 0x01, 0x25, 0x91, 0x66, 0x43, 0x2b, 0xe8, 0x16, 0x8a, 0xc7, 0x56, 0xee,
|
||||
0xff, 0xc3, 0x81, 0x55, 0xd5, 0xc0, 0x4c, 0x9e, 0xb0, 0x88, 0x18, 0xcf, 0xce, 0xcf, 0xd8, 0x8f,
|
||||
0xf4, 0x59, 0x2c, 0x3c, 0xe6, 0xf7, 0xe9, 0x77, 0x4f, 0xfa, 0xbb, 0xa3, 0x82, 0x41, 0xd0, 0x12,
|
||||
0x64, 0x64, 0xf6, 0xdc, 0xb1, 0x95, 0xf8, 0x4c, 0x10, 0x4f, 0x03, 0x6b, 0x8b, 0xb1, 0x81, 0xf8,
|
||||
0xe7, 0x0e, 0xb8, 0x8f, 0xc5, 0x68, 0x8f, 0x09, 0x9d, 0xcc, 0x2a, 0x95, 0x6d, 0x01, 0x35, 0x95,
|
||||
0xc4, 0xd1, 0xc9, 0xe2, 0x0e, 0xa7, 0xbf, 0x76, 0xa2, 0x0d, 0x58, 0x4a, 0xc4, 0xc8, 0x46, 0xbc,
|
||||
0x13, 0x98, 0x01, 0xba, 0x06, 0xad, 0x44, 0x8c, 0x74, 0x63, 0x6f, 0x33, 0xac, 0x1c, 0xab, 0xb0,
|
||||
0x4d, 0x6f, 0xca, 0x86, 0xbe, 0x29, 0xa7, 0x02, 0xff, 0xcf, 0x0e, 0x20, 0x7b, 0x15, 0x7d, 0xaa,
|
||||
0x9f, 0xc4, 0x35, 0x61, 0xab, 0xbf, 0xd8, 0xd6, 0x74, 0xba, 0xce, 0xc8, 0xe6, 0xea, 0x50, 0xfd,
|
||||
0xd8, 0x5b, 0xf4, 0x36, 0xac, 0x47, 0xe4, 0x00, 0xab, 0x5b, 0x73, 0xde, 0xe5, 0xae, 0x55, 0x94,
|
||||
0x97, 0xfb, 0xad, 0x0f, 0xa1, 0x5d, 0xfe, 0x13, 0x85, 0xba, 0xd0, 0xe9, 0xa7, 0x54, 0xea, 0x9e,
|
||||
0x85, 0xa6, 0xa3, 0xee, 0x1b, 0xc8, 0x85, 0xe6, 0x0f, 0x08, 0x8e, 0xe5, 0x78, 0xd2, 0x75, 0x50,
|
||||
0x07, 0x5a, 0x0f, 0x06, 0x29, 0xe3, 0x09, 0x8e, 0xbb, 0xb5, 0x5b, 0xdb, 0xb0, 0x7e, 0xec, 0x89,
|
||||
0xa8, 0x4c, 0x02, 0x76, 0xa4, 0x0e, 0x14, 0x75, 0xdf, 0x40, 0x6b, 0xe0, 0xee, 0xb2, 0x38, 0x4f,
|
||||
0x52, 0x23, 0x70, 0x76, 0x3e, 0xf8, 0xe9, 0x37, 0x47, 0x54, 0x8e, 0xf3, 0x81, 0x3a, 0xfd, 0x5d,
|
||||
0x03, 0xc7, 0xd7, 0x28, 0xb3, 0x5f, 0x77, 0x8b, 0x48, 0xdf, 0xd5, 0x08, 0x95, 0xc3, 0x6c, 0x30,
|
||||
0x58, 0xd6, 0x92, 0xf7, 0xff, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x52, 0x88, 0x12, 0x39, 0xe3, 0x1b,
|
||||
0x00, 0x00,
|
||||
}
|
||||
|
|
|
@ -17,9 +17,7 @@
|
|||
package proxy
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
|
@ -341,248 +339,6 @@ func (it *insertTask) checkRowNums() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// TODO(dragondriver): ignore the order of fields in request, use the order of CollectionSchema to reorganize data
|
||||
func (it *insertTask) transferColumnBasedRequestToRowBasedData() error {
|
||||
dTypes := make([]schemapb.DataType, 0, len(it.req.FieldsData))
|
||||
datas := make([][]interface{}, 0, len(it.req.FieldsData))
|
||||
rowNum := 0
|
||||
|
||||
appendScalarField := func(getDataFunc func() interface{}) error {
|
||||
fieldDatas := reflect.ValueOf(getDataFunc())
|
||||
if rowNum != 0 && rowNum != fieldDatas.Len() {
|
||||
return errors.New("the row num of different column is not equal")
|
||||
}
|
||||
rowNum = fieldDatas.Len()
|
||||
datas = append(datas, make([]interface{}, 0, rowNum))
|
||||
idx := len(datas) - 1
|
||||
for i := 0; i < rowNum; i++ {
|
||||
datas[idx] = append(datas[idx], fieldDatas.Index(i).Interface())
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
appendFloatVectorField := func(fDatas []float32, dim int64) error {
|
||||
l := len(fDatas)
|
||||
if int64(l)%dim != 0 {
|
||||
return errors.New("invalid vectors")
|
||||
}
|
||||
r := int64(l) / dim
|
||||
if rowNum != 0 && rowNum != int(r) {
|
||||
return errors.New("the row num of different column is not equal")
|
||||
}
|
||||
rowNum = int(r)
|
||||
datas = append(datas, make([]interface{}, 0, rowNum))
|
||||
idx := len(datas) - 1
|
||||
vector := make([]float32, 0, dim)
|
||||
for i := 0; i < l; i++ {
|
||||
vector = append(vector, fDatas[i])
|
||||
if int64(i+1)%dim == 0 {
|
||||
datas[idx] = append(datas[idx], vector)
|
||||
vector = make([]float32, 0, dim)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
appendBinaryVectorField := func(bDatas []byte, dim int64) error {
|
||||
l := len(bDatas)
|
||||
if dim%8 != 0 {
|
||||
return errors.New("invalid dim")
|
||||
}
|
||||
if (8*int64(l))%dim != 0 {
|
||||
return errors.New("invalid vectors")
|
||||
}
|
||||
r := (8 * int64(l)) / dim
|
||||
if rowNum != 0 && rowNum != int(r) {
|
||||
return errors.New("the row num of different column is not equal")
|
||||
}
|
||||
rowNum = int(r)
|
||||
datas = append(datas, make([]interface{}, 0, rowNum))
|
||||
idx := len(datas) - 1
|
||||
vector := make([]byte, 0, dim)
|
||||
for i := 0; i < l; i++ {
|
||||
vector = append(vector, bDatas[i])
|
||||
if (8*int64(i+1))%dim == 0 {
|
||||
datas[idx] = append(datas[idx], vector)
|
||||
vector = make([]byte, 0, dim)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
for _, field := range it.req.FieldsData {
|
||||
switch field.Field.(type) {
|
||||
case *schemapb.FieldData_Scalars:
|
||||
scalarField := field.GetScalars()
|
||||
switch scalarField.Data.(type) {
|
||||
case *schemapb.ScalarField_BoolData:
|
||||
err := appendScalarField(func() interface{} {
|
||||
return scalarField.GetBoolData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
case *schemapb.ScalarField_IntData:
|
||||
err := appendScalarField(func() interface{} {
|
||||
return scalarField.GetIntData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
case *schemapb.ScalarField_LongData:
|
||||
err := appendScalarField(func() interface{} {
|
||||
return scalarField.GetLongData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
case *schemapb.ScalarField_FloatData:
|
||||
err := appendScalarField(func() interface{} {
|
||||
return scalarField.GetFloatData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
case *schemapb.ScalarField_DoubleData:
|
||||
err := appendScalarField(func() interface{} {
|
||||
return scalarField.GetDoubleData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
case *schemapb.ScalarField_BytesData:
|
||||
return errors.New("bytes field is not supported now")
|
||||
case *schemapb.ScalarField_StringData:
|
||||
return errors.New("string field is not supported now")
|
||||
case nil:
|
||||
continue
|
||||
default:
|
||||
continue
|
||||
}
|
||||
case *schemapb.FieldData_Vectors:
|
||||
vectorField := field.GetVectors()
|
||||
switch vectorField.Data.(type) {
|
||||
case *schemapb.VectorField_FloatVector:
|
||||
floatVectorFieldData := vectorField.GetFloatVector().Data
|
||||
dim := vectorField.GetDim()
|
||||
err := appendFloatVectorField(floatVectorFieldData, dim)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
case *schemapb.VectorField_BinaryVector:
|
||||
binaryVectorFieldData := vectorField.GetBinaryVector()
|
||||
dim := vectorField.GetDim()
|
||||
err := appendBinaryVectorField(binaryVectorFieldData, dim)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
case nil:
|
||||
continue
|
||||
default:
|
||||
continue
|
||||
}
|
||||
case nil:
|
||||
continue
|
||||
default:
|
||||
continue
|
||||
}
|
||||
|
||||
dTypes = append(dTypes, field.Type)
|
||||
}
|
||||
|
||||
it.RowData = make([]*commonpb.Blob, 0, rowNum)
|
||||
l := len(dTypes)
|
||||
// TODO(dragondriver): big endian or little endian?
|
||||
endian := common.Endian
|
||||
printed := false
|
||||
for i := 0; i < rowNum; i++ {
|
||||
blob := &commonpb.Blob{
|
||||
Value: make([]byte, 0),
|
||||
}
|
||||
|
||||
for j := 0; j < l; j++ {
|
||||
var buffer bytes.Buffer
|
||||
switch dTypes[j] {
|
||||
case schemapb.DataType_Bool:
|
||||
d := datas[j][i].(bool)
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
case schemapb.DataType_Int8:
|
||||
d := int8(datas[j][i].(int32))
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
case schemapb.DataType_Int16:
|
||||
d := int16(datas[j][i].(int32))
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
case schemapb.DataType_Int32:
|
||||
d := datas[j][i].(int32)
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
case schemapb.DataType_Int64:
|
||||
d := datas[j][i].(int64)
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
case schemapb.DataType_Float:
|
||||
d := datas[j][i].(float32)
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
case schemapb.DataType_Double:
|
||||
d := datas[j][i].(float64)
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
case schemapb.DataType_FloatVector:
|
||||
d := datas[j][i].([]float32)
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
case schemapb.DataType_BinaryVector:
|
||||
d := datas[j][i].([]byte)
|
||||
err := binary.Write(&buffer, endian, d)
|
||||
if err != nil {
|
||||
log.Warn("ConvertData", zap.Error(err))
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
default:
|
||||
log.Warn("unsupported data type")
|
||||
}
|
||||
}
|
||||
if !printed {
|
||||
log.Debug("Proxy, transform", zap.Any("ID", it.ID()), zap.Any("BlobLen", len(blob.Value)), zap.Any("dTypes", dTypes))
|
||||
printed = true
|
||||
}
|
||||
it.RowData = append(it.RowData, blob)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (it *insertTask) checkFieldAutoIDAndHashPK() error {
|
||||
// TODO(dragondriver): in fact, NumRows is not trustable, we should check all input fields
|
||||
if it.req.NumRows <= 0 {
|
||||
|
@ -762,14 +518,15 @@ func (it *insertTask) PreExecute(ctx context.Context) error {
|
|||
return err
|
||||
}
|
||||
|
||||
tr := timerecord.NewTimeRecorder("ColumnToRow")
|
||||
err = it.transferColumnBasedRequestToRowBasedData()
|
||||
it.BaseInsertTask.InsertRequest.Version = internalpb.InsertDataVersion_ColumnBased
|
||||
it.BaseInsertTask.InsertRequest.FieldsData = it.req.GetFieldsData()
|
||||
it.BaseInsertTask.InsertRequest.NumRows = uint64(it.req.GetNumRows())
|
||||
err = typeutil.FillFieldBySchema(it.BaseInsertTask.InsertRequest.GetFieldsData(), collSchema)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
metrics.ProxyInsertColToRowLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.ProxyID, 10), collectionName).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||
|
||||
rowNum := len(it.RowData)
|
||||
rowNum := it.req.NumRows
|
||||
it.Timestamps = make([]uint64, rowNum)
|
||||
for index := range it.Timestamps {
|
||||
it.Timestamps[index] = it.BeginTimestamp
|
||||
|
@ -810,13 +567,20 @@ func (it *insertTask) _assignSegmentID(stream msgstream.MsgStream, pack *msgstre
|
|||
}
|
||||
|
||||
keys := hashKeys[i]
|
||||
timestampLen := len(insertRequest.Timestamps)
|
||||
rowIDLen := len(insertRequest.RowIDs)
|
||||
rowDataLen := len(insertRequest.RowData)
|
||||
keysLen := len(keys)
|
||||
|
||||
if keysLen != timestampLen || keysLen != rowIDLen || keysLen != rowDataLen {
|
||||
return nil, fmt.Errorf("the length of hashValue, timestamps, rowIDs, RowData are not equal")
|
||||
if !insertRequest.CheckAligned() {
|
||||
return nil,
|
||||
fmt.Errorf("the length of timestamps(%d), rowIDs(%d) and num_rows(%d) are not equal",
|
||||
len(insertRequest.GetTimestamps()),
|
||||
len(insertRequest.GetRowIDs()),
|
||||
insertRequest.NRows())
|
||||
}
|
||||
if uint64(keysLen) != insertRequest.NRows() {
|
||||
return nil,
|
||||
fmt.Errorf(
|
||||
"the length of hashValue(%d), num_rows(%d) are not equal",
|
||||
keysLen, insertRequest.NRows())
|
||||
}
|
||||
for idx, channelID := range keys {
|
||||
channelCountMap[channelID]++
|
||||
|
@ -919,6 +683,7 @@ func (it *insertTask) _assignSegmentID(stream msgstream.MsgStream, pack *msgstre
|
|||
return size
|
||||
}
|
||||
|
||||
sizePerRow, _ := typeutil.EstimateSizePerRecord(it.schema)
|
||||
result := make(map[int32]msgstream.TsMsg)
|
||||
curMsgSizeMap := make(map[int32]int)
|
||||
|
||||
|
@ -933,7 +698,6 @@ func (it *insertTask) _assignSegmentID(stream msgstream.MsgStream, pack *msgstre
|
|||
for index, key := range keys {
|
||||
ts := insertRequest.Timestamps[index]
|
||||
rowID := insertRequest.RowIDs[index]
|
||||
row := insertRequest.RowData[index]
|
||||
segmentID := getSegmentID(key)
|
||||
if segmentID == 0 {
|
||||
return nil, fmt.Errorf("get SegmentID failed, segmentID is zero")
|
||||
|
@ -954,6 +718,11 @@ func (it *insertTask) _assignSegmentID(stream msgstream.MsgStream, pack *msgstre
|
|||
SegmentID: segmentID,
|
||||
ShardName: channelNames[key],
|
||||
}
|
||||
|
||||
sliceRequest.Version = internalpb.InsertDataVersion_ColumnBased
|
||||
sliceRequest.NumRows = 0
|
||||
sliceRequest.FieldsData = make([]*schemapb.FieldData, len(it.BaseInsertTask.InsertRequest.GetFieldsData()))
|
||||
|
||||
insertMsg := &msgstream.InsertMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
Ctx: request.TraceCtx(),
|
||||
|
@ -963,15 +732,17 @@ func (it *insertTask) _assignSegmentID(stream msgstream.MsgStream, pack *msgstre
|
|||
result[key] = insertMsg
|
||||
curMsgSizeMap[key] = getFixedSizeOfInsertMsg(insertMsg)
|
||||
}
|
||||
|
||||
curMsg := result[key].(*msgstream.InsertMsg)
|
||||
curMsgSize := curMsgSizeMap[key]
|
||||
|
||||
curMsg.HashValues = append(curMsg.HashValues, insertRequest.HashValues[index])
|
||||
curMsg.Timestamps = append(curMsg.Timestamps, ts)
|
||||
curMsg.RowIDs = append(curMsg.RowIDs, rowID)
|
||||
curMsg.RowData = append(curMsg.RowData, row)
|
||||
/* #nosec G103 */
|
||||
curMsgSize += 4 + 8 + int(unsafe.Sizeof(row.Value))
|
||||
curMsgSize += len(row.Value)
|
||||
|
||||
typeutil.AppendFieldData(curMsg.FieldsData, it.BaseInsertTask.InsertRequest.GetFieldsData(), int64(index))
|
||||
curMsg.NumRows++
|
||||
curMsgSize += sizePerRow
|
||||
|
||||
if curMsgSize >= threshold {
|
||||
newPack.Msgs = append(newPack.Msgs, curMsg)
|
||||
|
|
|
@ -3537,7 +3537,11 @@ func TestTask_all(t *testing.T) {
|
|||
doubleField := "double"
|
||||
floatVecField := "fvec"
|
||||
binaryVecField := "bvec"
|
||||
fieldsLen := len([]string{boolField, int32Field, int64Field, floatField, doubleField, floatVecField, binaryVecField})
|
||||
var fieldsLen int
|
||||
fieldsLen = len([]string{boolField, int32Field, int64Field, floatField, doubleField, floatVecField})
|
||||
if enableMultipleVectorFields {
|
||||
fieldsLen = len([]string{boolField, int32Field, int64Field, floatField, doubleField, floatVecField, binaryVecField})
|
||||
}
|
||||
dim := 128
|
||||
nb := 10
|
||||
|
||||
|
@ -3746,7 +3750,7 @@ func TestTask_all(t *testing.T) {
|
|||
|
||||
task.req.FieldsData[5] = &schemapb.FieldData{
|
||||
Type: schemapb.DataType_FloatVector,
|
||||
FieldName: doubleField,
|
||||
FieldName: floatVecField,
|
||||
Field: &schemapb.FieldData_Vectors{
|
||||
Vectors: &schemapb.VectorField{
|
||||
Dim: int64(dim),
|
||||
|
@ -3760,18 +3764,20 @@ func TestTask_all(t *testing.T) {
|
|||
FieldId: common.StartOfUserFieldID + 5,
|
||||
}
|
||||
|
||||
task.req.FieldsData[6] = &schemapb.FieldData{
|
||||
Type: schemapb.DataType_BinaryVector,
|
||||
FieldName: doubleField,
|
||||
Field: &schemapb.FieldData_Vectors{
|
||||
Vectors: &schemapb.VectorField{
|
||||
Dim: int64(dim),
|
||||
Data: &schemapb.VectorField_BinaryVector{
|
||||
BinaryVector: generateBinaryVectors(nb, dim),
|
||||
if enableMultipleVectorFields {
|
||||
task.req.FieldsData[6] = &schemapb.FieldData{
|
||||
Type: schemapb.DataType_BinaryVector,
|
||||
FieldName: binaryVecField,
|
||||
Field: &schemapb.FieldData_Vectors{
|
||||
Vectors: &schemapb.VectorField{
|
||||
Dim: int64(dim),
|
||||
Data: &schemapb.VectorField_BinaryVector{
|
||||
BinaryVector: generateBinaryVectors(nb, dim),
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: common.StartOfUserFieldID + 6,
|
||||
FieldId: common.StartOfUserFieldID + 6,
|
||||
}
|
||||
}
|
||||
|
||||
assert.NoError(t, task.OnEnqueue())
|
||||
|
|
|
@ -158,6 +158,12 @@ func (fdmNode *filterDmNode) filterInvalidDeleteMessage(msg *msgstream.DeleteMsg
|
|||
|
||||
// filterInvalidInsertMessage would filter out invalid insert messages
|
||||
func (fdmNode *filterDmNode) filterInvalidInsertMessage(msg *msgstream.InsertMsg) *msgstream.InsertMsg {
|
||||
if !msg.CheckAligned() {
|
||||
// TODO: what if the messages are misaligned? Here, we ignore those messages and print error
|
||||
log.Warn("Error, misaligned messages detected")
|
||||
return nil
|
||||
}
|
||||
|
||||
sp, ctx := trace.StartSpanFromContext(msg.TraceCtx())
|
||||
msg.SetTraceCtx(ctx)
|
||||
defer sp.Finish()
|
||||
|
@ -219,12 +225,6 @@ func (fdmNode *filterDmNode) filterInvalidInsertMessage(msg *msgstream.InsertMsg
|
|||
}
|
||||
}
|
||||
|
||||
if len(msg.RowIDs) != len(msg.Timestamps) || len(msg.RowIDs) != len(msg.RowData) {
|
||||
// TODO: what if the messages are misaligned? Here, we ignore those messages and print error
|
||||
log.Warn("Error, misaligned messages detected")
|
||||
return nil
|
||||
}
|
||||
|
||||
if len(msg.Timestamps) <= 0 {
|
||||
log.Debug("filter invalid insert message, no message",
|
||||
zap.Any("collectionID", msg.CollectionID),
|
||||
|
|
|
@ -25,6 +25,8 @@ import (
|
|||
"strconv"
|
||||
"sync"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
||||
"github.com/opentracing/opentracing-go"
|
||||
"go.uber.org/zap"
|
||||
|
||||
|
@ -99,15 +101,24 @@ func (iNode *insertNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
}
|
||||
|
||||
// 1. hash insertMessages to insertData
|
||||
for _, task := range iMsg.insertMessages {
|
||||
for _, insertMsg := range iMsg.insertMessages {
|
||||
if insertMsg.IsColumnBased() {
|
||||
var err error
|
||||
insertMsg.RowData, err = typeutil.TransferColumnBasedDataToRowBasedData(insertMsg.FieldsData)
|
||||
if err != nil {
|
||||
log.Error("failed to transfer column-based data to row-based data", zap.Error(err))
|
||||
return []Msg{}
|
||||
}
|
||||
}
|
||||
|
||||
// if loadType is loadCollection, check if partition exists, if not, create partition
|
||||
col, err := iNode.streamingReplica.getCollectionByID(task.CollectionID)
|
||||
col, err := iNode.streamingReplica.getCollectionByID(insertMsg.CollectionID)
|
||||
if err != nil {
|
||||
log.Error(err.Error())
|
||||
continue
|
||||
}
|
||||
if col.getLoadType() == loadTypeCollection {
|
||||
err = iNode.streamingReplica.addPartition(task.CollectionID, task.PartitionID)
|
||||
err = iNode.streamingReplica.addPartition(insertMsg.CollectionID, insertMsg.PartitionID)
|
||||
if err != nil {
|
||||
log.Error(err.Error())
|
||||
continue
|
||||
|
@ -115,23 +126,24 @@ func (iNode *insertNode) Operate(in []flowgraph.Msg) []flowgraph.Msg {
|
|||
}
|
||||
|
||||
// check if segment exists, if not, create this segment
|
||||
if !iNode.streamingReplica.hasSegment(task.SegmentID) {
|
||||
err := iNode.streamingReplica.addSegment(task.SegmentID, task.PartitionID, task.CollectionID, task.ShardName, segmentTypeGrowing, true)
|
||||
if !iNode.streamingReplica.hasSegment(insertMsg.SegmentID) {
|
||||
err := iNode.streamingReplica.addSegment(insertMsg.SegmentID, insertMsg.PartitionID, insertMsg.CollectionID, insertMsg.ShardName, segmentTypeGrowing, true)
|
||||
if err != nil {
|
||||
log.Warn(err.Error())
|
||||
continue
|
||||
}
|
||||
}
|
||||
|
||||
iData.insertIDs[task.SegmentID] = append(iData.insertIDs[task.SegmentID], task.RowIDs...)
|
||||
iData.insertTimestamps[task.SegmentID] = append(iData.insertTimestamps[task.SegmentID], task.Timestamps...)
|
||||
iData.insertRecords[task.SegmentID] = append(iData.insertRecords[task.SegmentID], task.RowData...)
|
||||
pks, err := getPrimaryKeys(task, iNode.streamingReplica)
|
||||
iData.insertIDs[insertMsg.SegmentID] = append(iData.insertIDs[insertMsg.SegmentID], insertMsg.RowIDs...)
|
||||
iData.insertTimestamps[insertMsg.SegmentID] = append(iData.insertTimestamps[insertMsg.SegmentID], insertMsg.Timestamps...)
|
||||
// using insertMsg.RowData is valid here, since we have already transferred the column-based data.
|
||||
iData.insertRecords[insertMsg.SegmentID] = append(iData.insertRecords[insertMsg.SegmentID], insertMsg.RowData...)
|
||||
pks, err := getPrimaryKeys(insertMsg, iNode.streamingReplica)
|
||||
if err != nil {
|
||||
log.Warn(err.Error())
|
||||
continue
|
||||
}
|
||||
iData.insertPKs[task.SegmentID] = append(iData.insertPKs[task.SegmentID], pks...)
|
||||
iData.insertPKs[insertMsg.SegmentID] = append(iData.insertPKs[insertMsg.SegmentID], pks...)
|
||||
}
|
||||
|
||||
// 2. do preInsert
|
||||
|
@ -333,7 +345,7 @@ func (iNode *insertNode) delete(deleteData *deleteData, segmentID UniqueID, wg *
|
|||
// TODO: remove this function to proper file
|
||||
// getPrimaryKeys would get primary keys by insert messages
|
||||
func getPrimaryKeys(msg *msgstream.InsertMsg, streamingReplica ReplicaInterface) ([]int64, error) {
|
||||
if len(msg.RowIDs) != len(msg.Timestamps) || len(msg.RowIDs) != len(msg.RowData) {
|
||||
if !msg.CheckAligned() {
|
||||
log.Warn("misaligned messages detected")
|
||||
return nil, errors.New("misaligned messages detected")
|
||||
}
|
||||
|
@ -344,8 +356,20 @@ func getPrimaryKeys(msg *msgstream.InsertMsg, streamingReplica ReplicaInterface)
|
|||
log.Warn(err.Error())
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return getPKs(msg, collection.schema)
|
||||
}
|
||||
|
||||
func getPKs(msg *msgstream.InsertMsg, schema *schemapb.CollectionSchema) ([]int64, error) {
|
||||
if msg.IsRowBased() {
|
||||
return getPKsFromRowBasedInsertMsg(msg, schema)
|
||||
}
|
||||
return getPKsFromColumnBasedInsertMsg(msg, schema)
|
||||
}
|
||||
|
||||
func getPKsFromRowBasedInsertMsg(msg *msgstream.InsertMsg, schema *schemapb.CollectionSchema) ([]int64, error) {
|
||||
offset := 0
|
||||
for _, field := range collection.schema.Fields {
|
||||
for _, field := range schema.Fields {
|
||||
if field.IsPrimaryKey {
|
||||
break
|
||||
}
|
||||
|
@ -408,6 +432,25 @@ func getPrimaryKeys(msg *msgstream.InsertMsg, streamingReplica ReplicaInterface)
|
|||
return pks, nil
|
||||
}
|
||||
|
||||
func getPKsFromColumnBasedInsertMsg(msg *msgstream.InsertMsg, schema *schemapb.CollectionSchema) ([]int64, error) {
|
||||
loc := -1
|
||||
for idx, field := range schema.Fields {
|
||||
if field.IsPrimaryKey {
|
||||
loc = idx
|
||||
break
|
||||
}
|
||||
}
|
||||
if loc == -1 {
|
||||
return nil, errors.New("no primary field found")
|
||||
}
|
||||
|
||||
if len(msg.GetFieldsData()) <= loc {
|
||||
return nil, errors.New("insert msg mismatch the schema")
|
||||
}
|
||||
|
||||
return msg.GetFieldsData()[loc].GetScalars().GetLongData().GetData(), nil
|
||||
}
|
||||
|
||||
// newInsertNode returns a new insertNode
|
||||
func newInsertNode(streamingReplica ReplicaInterface) *insertNode {
|
||||
maxQueueLength := Params.QueryNodeCfg.FlowGraphMaxQueueLength
|
||||
|
|
|
@ -21,9 +21,18 @@ import (
|
|||
"encoding/binary"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"sort"
|
||||
"strconv"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
@ -234,3 +243,597 @@ func TransferColumnBasedInsertDataToRowBased(data *InsertData) (
|
|||
|
||||
return utss, rowIds.Data, rows, nil
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
// TODO: remove these functions to proper file.
|
||||
|
||||
// GetDimFromParams get dim from params.
|
||||
func GetDimFromParams(params []*commonpb.KeyValuePair) (int, error) {
|
||||
var dim int
|
||||
var err error
|
||||
for _, t := range params {
|
||||
if t.Key == "dim" {
|
||||
dim, err = strconv.Atoi(t.Value)
|
||||
if err != nil {
|
||||
return -1, err
|
||||
}
|
||||
return dim, nil
|
||||
}
|
||||
}
|
||||
return -1, errors.New("dim not found in params")
|
||||
}
|
||||
|
||||
// ReadBinary read data in bytes and write it into receiver.
|
||||
// The receiver can be any type in int8, int16, int32, int64, float32, float64 and bool
|
||||
// ReadBinary uses LittleEndian ByteOrder.
|
||||
func ReadBinary(reader io.Reader, receiver interface{}, dataType schemapb.DataType) {
|
||||
err := binary.Read(reader, common.Endian, receiver)
|
||||
if err != nil {
|
||||
log.Error("binary.Read failed", zap.Any("data type", dataType), zap.Error(err))
|
||||
}
|
||||
}
|
||||
|
||||
// It will save my life if golang support generic programming.
|
||||
|
||||
// TODO: string type.
|
||||
|
||||
func readFloatVectors(blobReaders []io.Reader, dim int) []float32 {
|
||||
ret := make([]float32, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v = make([]float32, dim)
|
||||
ReadBinary(r, &v, schemapb.DataType_FloatVector)
|
||||
ret = append(ret, v...)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func readBinaryVectors(blobReaders []io.Reader, dim int) []byte {
|
||||
ret := make([]byte, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v = make([]byte, dim/8)
|
||||
ReadBinary(r, &v, schemapb.DataType_BinaryVector)
|
||||
ret = append(ret, v...)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func readBoolArray(blobReaders []io.Reader) []bool {
|
||||
ret := make([]bool, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v bool
|
||||
ReadBinary(r, &v, schemapb.DataType_Bool)
|
||||
ret = append(ret, v)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func readInt8Array(blobReaders []io.Reader) []int8 {
|
||||
ret := make([]int8, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v int8
|
||||
ReadBinary(r, &v, schemapb.DataType_Int8)
|
||||
ret = append(ret, v)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func readInt16Array(blobReaders []io.Reader) []int16 {
|
||||
ret := make([]int16, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v int16
|
||||
ReadBinary(r, &v, schemapb.DataType_Int16)
|
||||
ret = append(ret, v)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func readInt32Array(blobReaders []io.Reader) []int32 {
|
||||
ret := make([]int32, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v int32
|
||||
ReadBinary(r, &v, schemapb.DataType_Int32)
|
||||
ret = append(ret, v)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func readInt64Array(blobReaders []io.Reader) []int64 {
|
||||
ret := make([]int64, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v int64
|
||||
ReadBinary(r, &v, schemapb.DataType_Int64)
|
||||
ret = append(ret, v)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func readFloatArray(blobReaders []io.Reader) []float32 {
|
||||
ret := make([]float32, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v float32
|
||||
ReadBinary(r, &v, schemapb.DataType_Float)
|
||||
ret = append(ret, v)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func readDoubleArray(blobReaders []io.Reader) []float64 {
|
||||
ret := make([]float64, 0)
|
||||
for _, r := range blobReaders {
|
||||
var v float64
|
||||
ReadBinary(r, &v, schemapb.DataType_Double)
|
||||
ret = append(ret, v)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func RowBasedInsertMsgToInsertData(msg *msgstream.InsertMsg, collSchema *schemapb.CollectionSchema) (idata *InsertData, err error) {
|
||||
blobReaders := make([]io.Reader, 0)
|
||||
for _, blob := range msg.RowData {
|
||||
blobReaders = append(blobReaders, bytes.NewReader(blob.GetValue()))
|
||||
}
|
||||
|
||||
idata = &InsertData{
|
||||
Data: make(map[FieldID]FieldData),
|
||||
// TODO: handle Infos.
|
||||
Infos: nil,
|
||||
}
|
||||
|
||||
for _, field := range collSchema.Fields {
|
||||
switch field.DataType {
|
||||
case schemapb.DataType_FloatVector:
|
||||
dim, err := GetDimFromParams(field.TypeParams)
|
||||
if err != nil {
|
||||
log.Error("failed to get dim", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
vecs := readFloatVectors(blobReaders, dim)
|
||||
idata.Data[field.FieldID] = &FloatVectorFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: vecs,
|
||||
Dim: dim,
|
||||
}
|
||||
|
||||
case schemapb.DataType_BinaryVector:
|
||||
var dim int
|
||||
dim, err := GetDimFromParams(field.TypeParams)
|
||||
if err != nil {
|
||||
log.Error("failed to get dim", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
vecs := readBinaryVectors(blobReaders, dim)
|
||||
idata.Data[field.FieldID] = &BinaryVectorFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: vecs,
|
||||
Dim: dim,
|
||||
}
|
||||
|
||||
case schemapb.DataType_Bool:
|
||||
idata.Data[field.FieldID] = &BoolFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: readBoolArray(blobReaders),
|
||||
}
|
||||
|
||||
case schemapb.DataType_Int8:
|
||||
idata.Data[field.FieldID] = &Int8FieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: readInt8Array(blobReaders),
|
||||
}
|
||||
|
||||
case schemapb.DataType_Int16:
|
||||
idata.Data[field.FieldID] = &Int16FieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: readInt16Array(blobReaders),
|
||||
}
|
||||
|
||||
case schemapb.DataType_Int32:
|
||||
idata.Data[field.FieldID] = &Int32FieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: readInt32Array(blobReaders),
|
||||
}
|
||||
|
||||
case schemapb.DataType_Int64:
|
||||
idata.Data[field.FieldID] = &Int64FieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: nil,
|
||||
}
|
||||
|
||||
fieldData := idata.Data[field.FieldID].(*Int64FieldData)
|
||||
switch field.FieldID {
|
||||
case 0: // rowIDs
|
||||
fieldData.Data = append(fieldData.Data, msg.RowIDs...)
|
||||
case 1: // Timestamps
|
||||
for _, ts := range msg.Timestamps {
|
||||
fieldData.Data = append(fieldData.Data, int64(ts))
|
||||
}
|
||||
default:
|
||||
fieldData.Data = readInt64Array(blobReaders)
|
||||
}
|
||||
|
||||
case schemapb.DataType_Float:
|
||||
idata.Data[field.FieldID] = &FloatFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: readFloatArray(blobReaders),
|
||||
}
|
||||
|
||||
case schemapb.DataType_Double:
|
||||
idata.Data[field.FieldID] = &DoubleFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: readDoubleArray(blobReaders),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return idata, nil
|
||||
}
|
||||
|
||||
func ColumnBasedInsertMsgToInsertData(msg *msgstream.InsertMsg, collSchema *schemapb.CollectionSchema) (idata *InsertData, err error) {
|
||||
srcFields := make(map[FieldID]*schemapb.FieldData)
|
||||
for _, field := range msg.FieldsData {
|
||||
srcFields[field.FieldId] = field
|
||||
}
|
||||
|
||||
idata = &InsertData{
|
||||
Data: make(map[FieldID]FieldData),
|
||||
// TODO: handle Infos.
|
||||
Infos: nil,
|
||||
}
|
||||
|
||||
for _, field := range collSchema.Fields {
|
||||
switch field.DataType {
|
||||
case schemapb.DataType_FloatVector:
|
||||
dim, err := GetDimFromParams(field.TypeParams)
|
||||
if err != nil {
|
||||
log.Error("failed to get dim", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
srcData := srcFields[field.FieldID].GetVectors().GetFloatVector().GetData()
|
||||
|
||||
fieldData := &FloatVectorFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]float32, 0, len(srcData)),
|
||||
Dim: dim,
|
||||
}
|
||||
fieldData.Data = append(fieldData.Data, srcData...)
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
|
||||
case schemapb.DataType_BinaryVector:
|
||||
dim, err := GetDimFromParams(field.TypeParams)
|
||||
if err != nil {
|
||||
log.Error("failed to get dim", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
srcData := srcFields[field.FieldID].GetVectors().GetBinaryVector()
|
||||
|
||||
fieldData := &BinaryVectorFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]byte, 0, len(srcData)),
|
||||
Dim: dim,
|
||||
}
|
||||
fieldData.Data = append(fieldData.Data, srcData...)
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
|
||||
case schemapb.DataType_Bool:
|
||||
srcData := srcFields[field.FieldID].GetScalars().GetBoolData().GetData()
|
||||
|
||||
fieldData := &BoolFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]bool, 0, len(srcData)),
|
||||
}
|
||||
fieldData.Data = append(fieldData.Data, srcData...)
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
|
||||
case schemapb.DataType_Int8:
|
||||
srcData := srcFields[field.FieldID].GetScalars().GetIntData().GetData()
|
||||
|
||||
fieldData := &Int8FieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]int8, 0, len(srcData)),
|
||||
}
|
||||
int8SrcData := make([]int8, len(srcData))
|
||||
for i := 0; i < len(srcData); i++ {
|
||||
int8SrcData[i] = int8(srcData[i])
|
||||
}
|
||||
fieldData.Data = append(fieldData.Data, int8SrcData...)
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
|
||||
case schemapb.DataType_Int16:
|
||||
srcData := srcFields[field.FieldID].GetScalars().GetIntData().GetData()
|
||||
|
||||
fieldData := &Int16FieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]int16, 0, len(srcData)),
|
||||
}
|
||||
int16SrcData := make([]int16, len(srcData))
|
||||
for i := 0; i < len(srcData); i++ {
|
||||
int16SrcData[i] = int16(srcData[i])
|
||||
}
|
||||
fieldData.Data = append(fieldData.Data, int16SrcData...)
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
|
||||
case schemapb.DataType_Int32:
|
||||
srcData := srcFields[field.FieldID].GetScalars().GetIntData().GetData()
|
||||
|
||||
fieldData := &Int32FieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]int32, 0, len(srcData)),
|
||||
}
|
||||
fieldData.Data = append(fieldData.Data, srcData...)
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
|
||||
case schemapb.DataType_Int64:
|
||||
fieldData := &Int64FieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]int64, 0),
|
||||
}
|
||||
|
||||
switch field.FieldID {
|
||||
case 0: // rowIDs
|
||||
fieldData.Data = make([]int64, 0, len(msg.RowIDs))
|
||||
fieldData.Data = append(fieldData.Data, msg.RowIDs...)
|
||||
case 1: // Timestamps
|
||||
fieldData.Data = make([]int64, 0, len(msg.Timestamps))
|
||||
for _, ts := range msg.Timestamps {
|
||||
fieldData.Data = append(fieldData.Data, int64(ts))
|
||||
}
|
||||
default:
|
||||
srcData := srcFields[field.FieldID].GetScalars().GetLongData().GetData()
|
||||
fieldData.Data = make([]int64, 0, len(srcData))
|
||||
fieldData.Data = append(fieldData.Data, srcData...)
|
||||
}
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
|
||||
case schemapb.DataType_Float:
|
||||
srcData := srcFields[field.FieldID].GetScalars().GetFloatData().GetData()
|
||||
|
||||
fieldData := &FloatFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]float32, 0, len(srcData)),
|
||||
}
|
||||
fieldData.Data = append(fieldData.Data, srcData...)
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
|
||||
case schemapb.DataType_Double:
|
||||
srcData := srcFields[field.FieldID].GetScalars().GetDoubleData().GetData()
|
||||
|
||||
fieldData := &DoubleFieldData{
|
||||
NumRows: []int64{int64(msg.NRows())},
|
||||
Data: make([]float64, 0, len(srcData)),
|
||||
}
|
||||
fieldData.Data = append(fieldData.Data, srcData...)
|
||||
|
||||
idata.Data[field.FieldID] = fieldData
|
||||
}
|
||||
}
|
||||
|
||||
return idata, nil
|
||||
}
|
||||
|
||||
func InsertMsgToInsertData(msg *msgstream.InsertMsg, schema *schemapb.CollectionSchema) (idata *InsertData, err error) {
|
||||
if msg.IsRowBased() {
|
||||
return RowBasedInsertMsgToInsertData(msg, schema)
|
||||
}
|
||||
return ColumnBasedInsertMsgToInsertData(msg, schema)
|
||||
}
|
||||
|
||||
func mergeBoolField(data *InsertData, fid FieldID, field *BoolFieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &BoolFieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*BoolFieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeInt8Field(data *InsertData, fid FieldID, field *Int8FieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &Int8FieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*Int8FieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeInt16Field(data *InsertData, fid FieldID, field *Int16FieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &Int16FieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*Int16FieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeInt32Field(data *InsertData, fid FieldID, field *Int32FieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &Int32FieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*Int32FieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeInt64Field(data *InsertData, fid FieldID, field *Int64FieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &Int64FieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*Int64FieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeFloatField(data *InsertData, fid FieldID, field *FloatFieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &FloatFieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*FloatFieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeDoubleField(data *InsertData, fid FieldID, field *DoubleFieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &DoubleFieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*DoubleFieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeStringField(data *InsertData, fid FieldID, field *StringFieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &StringFieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*StringFieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeBinaryVectorField(data *InsertData, fid FieldID, field *BinaryVectorFieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &BinaryVectorFieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
Dim: field.Dim,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*BinaryVectorFieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
func mergeFloatVectorField(data *InsertData, fid FieldID, field *FloatVectorFieldData) {
|
||||
if _, ok := data.Data[fid]; !ok {
|
||||
fieldData := &FloatVectorFieldData{
|
||||
NumRows: []int64{0},
|
||||
Data: nil,
|
||||
Dim: field.Dim,
|
||||
}
|
||||
data.Data[fid] = fieldData
|
||||
}
|
||||
fieldData := data.Data[fid].(*FloatVectorFieldData)
|
||||
fieldData.Data = append(fieldData.Data, field.Data...)
|
||||
fieldData.NumRows[0] += int64(field.RowNum())
|
||||
}
|
||||
|
||||
// MergeFieldData merge field into data.
|
||||
func MergeFieldData(data *InsertData, fid FieldID, field FieldData) {
|
||||
if field == nil {
|
||||
return
|
||||
}
|
||||
switch field := field.(type) {
|
||||
case *BoolFieldData:
|
||||
mergeBoolField(data, fid, field)
|
||||
case *Int8FieldData:
|
||||
mergeInt8Field(data, fid, field)
|
||||
case *Int16FieldData:
|
||||
mergeInt16Field(data, fid, field)
|
||||
case *Int32FieldData:
|
||||
mergeInt32Field(data, fid, field)
|
||||
case *Int64FieldData:
|
||||
mergeInt64Field(data, fid, field)
|
||||
case *FloatFieldData:
|
||||
mergeFloatField(data, fid, field)
|
||||
case *DoubleFieldData:
|
||||
mergeDoubleField(data, fid, field)
|
||||
case *StringFieldData:
|
||||
mergeStringField(data, fid, field)
|
||||
case *BinaryVectorFieldData:
|
||||
mergeBinaryVectorField(data, fid, field)
|
||||
case *FloatVectorFieldData:
|
||||
mergeFloatVectorField(data, fid, field)
|
||||
}
|
||||
}
|
||||
|
||||
// MergeInsertData merge insert datas. Maybe there are large write zoom if frequent inserts are met.
|
||||
func MergeInsertData(datas ...*InsertData) *InsertData {
|
||||
ret := &InsertData{
|
||||
Data: make(map[FieldID]FieldData),
|
||||
Infos: nil,
|
||||
}
|
||||
for _, data := range datas {
|
||||
if data != nil {
|
||||
for fid, field := range data.Data {
|
||||
MergeFieldData(ret, fid, field)
|
||||
}
|
||||
|
||||
// TODO: handle storage.InsertData.Infos
|
||||
ret.Infos = append(ret.Infos, data.Infos...)
|
||||
}
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
// TODO: string type.
|
||||
func GetPkFromInsertData(collSchema *schemapb.CollectionSchema, data *InsertData) ([]int64, error) {
|
||||
helper, err := typeutil.CreateSchemaHelper(collSchema)
|
||||
if err != nil {
|
||||
log.Error("failed to create schema helper", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
pf, err := helper.GetPrimaryKeyField()
|
||||
if err != nil {
|
||||
log.Warn("no primary field found", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
pfData, ok := data.Data[pf.FieldID]
|
||||
if !ok {
|
||||
log.Warn("no primary field found in insert msg", zap.Int64("fieldID", pf.FieldID))
|
||||
return nil, errors.New("no primary field found in insert msg")
|
||||
}
|
||||
|
||||
realPfData, ok := pfData.(*Int64FieldData)
|
||||
if !ok {
|
||||
log.Warn("primary field not in int64 format", zap.Int64("fieldID", pf.FieldID))
|
||||
return nil, errors.New("primary field not in int64 format")
|
||||
}
|
||||
|
||||
return realPfData.Data, nil
|
||||
}
|
||||
|
|
|
@ -18,13 +18,20 @@ package storage
|
|||
|
||||
import (
|
||||
"bytes"
|
||||
"crypto/rand"
|
||||
"encoding/binary"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/uniquegenerator"
|
||||
|
@ -573,7 +580,7 @@ func TestTransferColumnBasedInsertDataToRowBased(t *testing.T) {
|
|||
rows[1].Value)
|
||||
assert.ElementsMatch(t,
|
||||
[]byte{
|
||||
1, // false
|
||||
1, // true
|
||||
0x1f, // 0x1f
|
||||
0xff, 0x1f, // 0x1fff
|
||||
0xff, 0xff, 0xff, 0x1f, // 0x1fffffff
|
||||
|
@ -587,3 +594,832 @@ func TestTransferColumnBasedInsertDataToRowBased(t *testing.T) {
|
|||
rows[2].Value)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGetDimFromParams(t *testing.T) {
|
||||
dim := 8
|
||||
params1 := []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "dim",
|
||||
Value: strconv.Itoa(dim),
|
||||
},
|
||||
}
|
||||
got, err := GetDimFromParams(params1)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, dim, got)
|
||||
|
||||
params2 := []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "dim",
|
||||
Value: "not in int format",
|
||||
},
|
||||
}
|
||||
_, err = GetDimFromParams(params2)
|
||||
assert.Error(t, err)
|
||||
|
||||
params3 := []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "not dim",
|
||||
Value: strconv.Itoa(dim),
|
||||
},
|
||||
}
|
||||
_, err = GetDimFromParams(params3)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func TestReadBinary(t *testing.T) {
|
||||
reader := bytes.NewReader(
|
||||
[]byte{
|
||||
1, // true
|
||||
0x1f, // 0x1f
|
||||
0xff, 0x1f, // 0x1fff
|
||||
0xff, 0xff, 0xff, 0x1f, // 0x1fffffff
|
||||
0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x1f, // 0x1fffffffffffffff
|
||||
// hard to compare float value
|
||||
// 0, 0, 0, 0, // 0
|
||||
// 0, 0, 0, 0, 0, 0, 0, 0, // 0
|
||||
// b + 3, // "3"
|
||||
3, // 3
|
||||
// 0, 0, 0, 0, // 0
|
||||
},
|
||||
)
|
||||
|
||||
if common.Endian == binary.LittleEndian {
|
||||
var b bool
|
||||
ReadBinary(reader, &b, schemapb.DataType_Bool)
|
||||
assert.True(t, b)
|
||||
|
||||
var i8 int8
|
||||
ReadBinary(reader, &i8, schemapb.DataType_Int8)
|
||||
assert.Equal(t, int8(0x1f), i8)
|
||||
|
||||
var i16 int16
|
||||
ReadBinary(reader, &i16, schemapb.DataType_Int16)
|
||||
assert.Equal(t, int16(0x1fff), i16)
|
||||
|
||||
var i32 int32
|
||||
ReadBinary(reader, &i32, schemapb.DataType_Int32)
|
||||
assert.Equal(t, int32(0x1fffffff), i32)
|
||||
|
||||
var i64 int64
|
||||
ReadBinary(reader, &i64, schemapb.DataType_Int64)
|
||||
assert.Equal(t, int64(0x1fffffffffffffff), i64)
|
||||
|
||||
bvec := make([]byte, 1)
|
||||
ReadBinary(reader, &bvec, schemapb.DataType_BinaryVector)
|
||||
assert.Equal(t, []byte{3}, bvec)
|
||||
|
||||
// should print error here, no content in reader.
|
||||
ReadBinary(reader, &bvec, schemapb.DataType_BinaryVector)
|
||||
}
|
||||
}
|
||||
|
||||
func genAllFieldsSchema(fVecDim, bVecDim int) (schema *schemapb.CollectionSchema, pkFieldID UniqueID, fieldIDs []UniqueID) {
|
||||
schema = &schemapb.CollectionSchema{
|
||||
Name: "all_fields_schema",
|
||||
Description: "all_fields_schema",
|
||||
AutoID: false,
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
DataType: schemapb.DataType_Int64,
|
||||
IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
DataType: schemapb.DataType_Bool,
|
||||
},
|
||||
{
|
||||
DataType: schemapb.DataType_Int8,
|
||||
},
|
||||
{
|
||||
DataType: schemapb.DataType_Int16,
|
||||
},
|
||||
{
|
||||
DataType: schemapb.DataType_Int32,
|
||||
},
|
||||
{
|
||||
DataType: schemapb.DataType_Float,
|
||||
},
|
||||
{
|
||||
DataType: schemapb.DataType_Double,
|
||||
},
|
||||
{
|
||||
DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "dim",
|
||||
Value: strconv.Itoa(fVecDim),
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
DataType: schemapb.DataType_BinaryVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "dim",
|
||||
Value: strconv.Itoa(bVecDim),
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
fieldIDs = make([]UniqueID, 0)
|
||||
for idx := range schema.Fields {
|
||||
fID := int64(common.StartOfUserFieldID + idx)
|
||||
schema.Fields[idx].FieldID = fID
|
||||
if schema.Fields[idx].IsPrimaryKey {
|
||||
pkFieldID = fID
|
||||
}
|
||||
fieldIDs = append(fieldIDs, fID)
|
||||
}
|
||||
schema.Fields = append(schema.Fields, &schemapb.FieldSchema{
|
||||
FieldID: common.RowIDField,
|
||||
Name: common.RowIDFieldName,
|
||||
IsPrimaryKey: false,
|
||||
Description: "",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
})
|
||||
schema.Fields = append(schema.Fields, &schemapb.FieldSchema{
|
||||
FieldID: common.TimeStampField,
|
||||
Name: common.TimeStampFieldName,
|
||||
IsPrimaryKey: false,
|
||||
Description: "",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
})
|
||||
return schema, pkFieldID, fieldIDs
|
||||
}
|
||||
|
||||
func generateFloatVectors(numRows, dim int) []float32 {
|
||||
total := numRows * dim
|
||||
ret := make([]float32, 0, total)
|
||||
for i := 0; i < total; i++ {
|
||||
ret = append(ret, rand.Float32())
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func generateBinaryVectors(numRows, dim int) []byte {
|
||||
total := (numRows * dim) / 8
|
||||
ret := make([]byte, total)
|
||||
_, err := rand.Read(ret)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func generateBoolArray(numRows int) []bool {
|
||||
ret := make([]bool, 0, numRows)
|
||||
for i := 0; i < numRows; i++ {
|
||||
ret = append(ret, rand.Int()%2 == 0)
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func generateInt32Array(numRows int) []int32 {
|
||||
ret := make([]int32, 0, numRows)
|
||||
for i := 0; i < numRows; i++ {
|
||||
ret = append(ret, int32(rand.Int()))
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func generateInt64Array(numRows int) []int64 {
|
||||
ret := make([]int64, 0, numRows)
|
||||
for i := 0; i < numRows; i++ {
|
||||
ret = append(ret, int64(rand.Int()))
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func generateFloat32Array(numRows int) []float32 {
|
||||
ret := make([]float32, 0, numRows)
|
||||
for i := 0; i < numRows; i++ {
|
||||
ret = append(ret, rand.Float32())
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func generateFloat64Array(numRows int) []float64 {
|
||||
ret := make([]float64, 0, numRows)
|
||||
for i := 0; i < numRows; i++ {
|
||||
ret = append(ret, rand.Float64())
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func genRowWithAllFields(fVecDim, bVecDim int) (blob *commonpb.Blob, pk int64, row []interface{}) {
|
||||
schema, _, _ := genAllFieldsSchema(fVecDim, bVecDim)
|
||||
ret := &commonpb.Blob{
|
||||
Value: nil,
|
||||
}
|
||||
row = make([]interface{}, 0)
|
||||
for _, field := range schema.Fields {
|
||||
var buffer bytes.Buffer
|
||||
switch field.DataType {
|
||||
case schemapb.DataType_FloatVector:
|
||||
fVec := generateFloatVectors(1, fVecDim)
|
||||
_ = binary.Write(&buffer, common.Endian, fVec)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, fVec)
|
||||
case schemapb.DataType_BinaryVector:
|
||||
bVec := generateBinaryVectors(1, bVecDim)
|
||||
_ = binary.Write(&buffer, common.Endian, bVec)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, bVec)
|
||||
case schemapb.DataType_Bool:
|
||||
data := rand.Int()%2 == 0
|
||||
_ = binary.Write(&buffer, common.Endian, data)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, data)
|
||||
case schemapb.DataType_Int8:
|
||||
data := int8(rand.Int())
|
||||
_ = binary.Write(&buffer, common.Endian, data)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, data)
|
||||
case schemapb.DataType_Int16:
|
||||
data := int16(rand.Int())
|
||||
_ = binary.Write(&buffer, common.Endian, data)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, data)
|
||||
case schemapb.DataType_Int32:
|
||||
data := int32(rand.Int())
|
||||
_ = binary.Write(&buffer, common.Endian, data)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, data)
|
||||
case schemapb.DataType_Int64:
|
||||
pk = int64(rand.Int())
|
||||
_ = binary.Write(&buffer, common.Endian, pk)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, pk)
|
||||
case schemapb.DataType_Float:
|
||||
data := rand.Float32()
|
||||
_ = binary.Write(&buffer, common.Endian, data)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, data)
|
||||
case schemapb.DataType_Double:
|
||||
data := rand.Float64()
|
||||
_ = binary.Write(&buffer, common.Endian, data)
|
||||
ret.Value = append(ret.Value, buffer.Bytes()...)
|
||||
row = append(row, data)
|
||||
}
|
||||
}
|
||||
return ret, pk, row
|
||||
}
|
||||
|
||||
func genRowBasedInsertMsg(numRows, fVecDim, bVecDim int) (msg *msgstream.InsertMsg, pks []int64, columns [][]interface{}) {
|
||||
msg = &msgstream.InsertMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
Ctx: nil,
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: nil,
|
||||
MsgPosition: nil,
|
||||
},
|
||||
InsertRequest: internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: 0,
|
||||
},
|
||||
RowData: nil,
|
||||
Version: internalpb.InsertDataVersion_RowBased,
|
||||
},
|
||||
}
|
||||
pks = make([]int64, 0)
|
||||
raws := make([][]interface{}, 0)
|
||||
for i := 0; i < numRows; i++ {
|
||||
row, pk, raw := genRowWithAllFields(fVecDim, bVecDim)
|
||||
msg.InsertRequest.RowData = append(msg.InsertRequest.RowData, row)
|
||||
pks = append(pks, pk)
|
||||
raws = append(raws, raw)
|
||||
}
|
||||
numColumns := len(raws[0])
|
||||
columns = make([][]interface{}, numColumns)
|
||||
for _, raw := range raws {
|
||||
for j, data := range raw {
|
||||
columns[j] = append(columns[j], data)
|
||||
}
|
||||
}
|
||||
return msg, pks, columns
|
||||
}
|
||||
|
||||
func genColumnBasedInsertMsg(schema *schemapb.CollectionSchema, numRows, fVecDim, bVecDim int) (msg *msgstream.InsertMsg, pks []int64, columns [][]interface{}) {
|
||||
msg = &msgstream.InsertMsg{
|
||||
BaseMsg: msgstream.BaseMsg{
|
||||
Ctx: nil,
|
||||
BeginTimestamp: 0,
|
||||
EndTimestamp: 0,
|
||||
HashValues: nil,
|
||||
MsgPosition: nil,
|
||||
},
|
||||
InsertRequest: internalpb.InsertRequest{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_Insert,
|
||||
MsgID: 0,
|
||||
Timestamp: 0,
|
||||
SourceID: 0,
|
||||
},
|
||||
FieldsData: nil,
|
||||
NumRows: uint64(numRows),
|
||||
Version: internalpb.InsertDataVersion_ColumnBased,
|
||||
},
|
||||
}
|
||||
pks = make([]int64, 0)
|
||||
columns = make([][]interface{}, len(schema.Fields))
|
||||
|
||||
for idx, field := range schema.Fields {
|
||||
switch field.DataType {
|
||||
case schemapb.DataType_Bool:
|
||||
data := generateBoolArray(numRows)
|
||||
f := &schemapb.FieldData{
|
||||
Type: field.DataType,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_BoolData{
|
||||
BoolData: &schemapb.BoolArray{
|
||||
Data: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for _, d := range data {
|
||||
columns[idx] = append(columns[idx], d)
|
||||
}
|
||||
case schemapb.DataType_Int8:
|
||||
data := generateInt32Array(numRows)
|
||||
f := &schemapb.FieldData{
|
||||
Type: field.DataType,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for _, d := range data {
|
||||
columns[idx] = append(columns[idx], int8(d))
|
||||
}
|
||||
case schemapb.DataType_Int16:
|
||||
data := generateInt32Array(numRows)
|
||||
f := &schemapb.FieldData{
|
||||
Type: field.DataType,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for _, d := range data {
|
||||
columns[idx] = append(columns[idx], int16(d))
|
||||
}
|
||||
case schemapb.DataType_Int32:
|
||||
data := generateInt32Array(numRows)
|
||||
f := &schemapb.FieldData{
|
||||
Type: field.DataType,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for _, d := range data {
|
||||
columns[idx] = append(columns[idx], d)
|
||||
}
|
||||
case schemapb.DataType_Int64:
|
||||
data := generateInt64Array(numRows)
|
||||
f := &schemapb.FieldData{
|
||||
Type: field.DataType,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_LongData{
|
||||
LongData: &schemapb.LongArray{
|
||||
Data: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for _, d := range data {
|
||||
columns[idx] = append(columns[idx], d)
|
||||
}
|
||||
pks = data
|
||||
case schemapb.DataType_Float:
|
||||
data := generateFloat32Array(numRows)
|
||||
f := &schemapb.FieldData{
|
||||
Type: field.DataType,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_FloatData{
|
||||
FloatData: &schemapb.FloatArray{
|
||||
Data: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for _, d := range data {
|
||||
columns[idx] = append(columns[idx], d)
|
||||
}
|
||||
case schemapb.DataType_Double:
|
||||
data := generateFloat64Array(numRows)
|
||||
f := &schemapb.FieldData{
|
||||
Type: field.DataType,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_DoubleData{
|
||||
DoubleData: &schemapb.DoubleArray{
|
||||
Data: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for _, d := range data {
|
||||
columns[idx] = append(columns[idx], d)
|
||||
}
|
||||
case schemapb.DataType_FloatVector:
|
||||
data := generateFloatVectors(numRows, fVecDim)
|
||||
f := &schemapb.FieldData{
|
||||
Type: schemapb.DataType_FloatVector,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Vectors{
|
||||
Vectors: &schemapb.VectorField{
|
||||
Dim: int64(fVecDim),
|
||||
Data: &schemapb.VectorField_FloatVector{
|
||||
FloatVector: &schemapb.FloatArray{
|
||||
Data: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for nrows := 0; nrows < numRows; nrows++ {
|
||||
columns[idx] = append(columns[idx], data[nrows*fVecDim:(nrows+1)*fVecDim])
|
||||
}
|
||||
case schemapb.DataType_BinaryVector:
|
||||
data := generateBinaryVectors(numRows, bVecDim)
|
||||
f := &schemapb.FieldData{
|
||||
Type: schemapb.DataType_BinaryVector,
|
||||
FieldName: field.Name,
|
||||
Field: &schemapb.FieldData_Vectors{
|
||||
Vectors: &schemapb.VectorField{
|
||||
Dim: int64(bVecDim),
|
||||
Data: &schemapb.VectorField_BinaryVector{
|
||||
BinaryVector: data,
|
||||
},
|
||||
},
|
||||
},
|
||||
FieldId: field.FieldID,
|
||||
}
|
||||
msg.FieldsData = append(msg.FieldsData, f)
|
||||
for nrows := 0; nrows < numRows; nrows++ {
|
||||
columns[idx] = append(columns[idx], data[nrows*bVecDim/8:(nrows+1)*bVecDim/8])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return msg, pks, columns
|
||||
}
|
||||
|
||||
func TestRowBasedInsertMsgToInsertData(t *testing.T) {
|
||||
numRows, fVecDim, bVecDim := 10, 8, 8
|
||||
schema, _, fieldIDs := genAllFieldsSchema(fVecDim, bVecDim)
|
||||
msg, _, columns := genRowBasedInsertMsg(numRows, fVecDim, bVecDim)
|
||||
|
||||
idata, err := RowBasedInsertMsgToInsertData(msg, schema)
|
||||
assert.Nil(t, err)
|
||||
for idx, fID := range fieldIDs {
|
||||
column := columns[idx]
|
||||
fData, ok := idata.Data[fID]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(column), fData.RowNum())
|
||||
for j := range column {
|
||||
assert.Equal(t, fData.GetRow(j), column[j])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestColumnBasedInsertMsgToInsertData(t *testing.T) {
|
||||
numRows, fVecDim, bVecDim := 2, 2, 8
|
||||
schema, _, fieldIDs := genAllFieldsSchema(fVecDim, bVecDim)
|
||||
msg, _, columns := genColumnBasedInsertMsg(schema, numRows, fVecDim, bVecDim)
|
||||
|
||||
idata, err := ColumnBasedInsertMsgToInsertData(msg, schema)
|
||||
assert.Nil(t, err)
|
||||
for idx, fID := range fieldIDs {
|
||||
column := columns[idx]
|
||||
fData, ok := idata.Data[fID]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(column), fData.RowNum())
|
||||
for j := range column {
|
||||
assert.Equal(t, fData.GetRow(j), column[j])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestInsertMsgToInsertData(t *testing.T) {
|
||||
numRows, fVecDim, bVecDim := 10, 8, 8
|
||||
schema, _, fieldIDs := genAllFieldsSchema(fVecDim, bVecDim)
|
||||
msg, _, columns := genRowBasedInsertMsg(numRows, fVecDim, bVecDim)
|
||||
|
||||
idata, err := InsertMsgToInsertData(msg, schema)
|
||||
assert.Nil(t, err)
|
||||
for idx, fID := range fieldIDs {
|
||||
column := columns[idx]
|
||||
fData, ok := idata.Data[fID]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(column), fData.RowNum())
|
||||
for j := range column {
|
||||
assert.Equal(t, fData.GetRow(j), column[j])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestInsertMsgToInsertData2(t *testing.T) {
|
||||
numRows, fVecDim, bVecDim := 2, 2, 8
|
||||
schema, _, fieldIDs := genAllFieldsSchema(fVecDim, bVecDim)
|
||||
msg, _, columns := genColumnBasedInsertMsg(schema, numRows, fVecDim, bVecDim)
|
||||
|
||||
idata, err := InsertMsgToInsertData(msg, schema)
|
||||
assert.Nil(t, err)
|
||||
for idx, fID := range fieldIDs {
|
||||
column := columns[idx]
|
||||
fData, ok := idata.Data[fID]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, len(column), fData.RowNum())
|
||||
for j := range column {
|
||||
assert.Equal(t, fData.GetRow(j), column[j])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func TestMergeInsertData(t *testing.T) {
|
||||
d1 := &InsertData{
|
||||
Data: map[int64]FieldData{
|
||||
common.RowIDField: &Int64FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int64{1},
|
||||
},
|
||||
common.TimeStampField: &Int64FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int64{1},
|
||||
},
|
||||
BoolField: &BoolFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []bool{true},
|
||||
},
|
||||
Int8Field: &Int8FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int8{1},
|
||||
},
|
||||
Int16Field: &Int16FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int16{1},
|
||||
},
|
||||
Int32Field: &Int32FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int32{1},
|
||||
},
|
||||
Int64Field: &Int64FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int64{1},
|
||||
},
|
||||
FloatField: &FloatFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []float32{0},
|
||||
},
|
||||
DoubleField: &DoubleFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []float64{0},
|
||||
},
|
||||
StringField: &StringFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []string{"1"},
|
||||
},
|
||||
BinaryVectorField: &BinaryVectorFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []byte{0},
|
||||
Dim: 8,
|
||||
},
|
||||
FloatVectorField: &FloatVectorFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []float32{0},
|
||||
Dim: 1,
|
||||
},
|
||||
},
|
||||
Infos: nil,
|
||||
}
|
||||
d2 := &InsertData{
|
||||
Data: map[int64]FieldData{
|
||||
common.RowIDField: &Int64FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int64{2},
|
||||
},
|
||||
common.TimeStampField: &Int64FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int64{2},
|
||||
},
|
||||
BoolField: &BoolFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []bool{false},
|
||||
},
|
||||
Int8Field: &Int8FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int8{2},
|
||||
},
|
||||
Int16Field: &Int16FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int16{2},
|
||||
},
|
||||
Int32Field: &Int32FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int32{2},
|
||||
},
|
||||
Int64Field: &Int64FieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []int64{2},
|
||||
},
|
||||
FloatField: &FloatFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []float32{0},
|
||||
},
|
||||
DoubleField: &DoubleFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []float64{0},
|
||||
},
|
||||
StringField: &StringFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []string{"2"},
|
||||
},
|
||||
BinaryVectorField: &BinaryVectorFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []byte{0},
|
||||
Dim: 8,
|
||||
},
|
||||
FloatVectorField: &FloatVectorFieldData{
|
||||
NumRows: []int64{1},
|
||||
Data: []float32{0},
|
||||
Dim: 1,
|
||||
},
|
||||
},
|
||||
Infos: nil,
|
||||
}
|
||||
|
||||
merged := MergeInsertData(d1, d2)
|
||||
|
||||
f, ok := merged.Data[common.RowIDField]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*Int64FieldData).NumRows)
|
||||
assert.Equal(t, []int64{1, 2}, f.(*Int64FieldData).Data)
|
||||
|
||||
f, ok = merged.Data[common.TimeStampField]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*Int64FieldData).NumRows)
|
||||
assert.Equal(t, []int64{1, 2}, f.(*Int64FieldData).Data)
|
||||
|
||||
f, ok = merged.Data[BoolField]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*BoolFieldData).NumRows)
|
||||
assert.Equal(t, []bool{true, false}, f.(*BoolFieldData).Data)
|
||||
|
||||
f, ok = merged.Data[Int8Field]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*Int8FieldData).NumRows)
|
||||
assert.Equal(t, []int8{1, 2}, f.(*Int8FieldData).Data)
|
||||
|
||||
f, ok = merged.Data[Int16Field]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*Int16FieldData).NumRows)
|
||||
assert.Equal(t, []int16{1, 2}, f.(*Int16FieldData).Data)
|
||||
|
||||
f, ok = merged.Data[Int32Field]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*Int32FieldData).NumRows)
|
||||
assert.Equal(t, []int32{1, 2}, f.(*Int32FieldData).Data)
|
||||
|
||||
f, ok = merged.Data[Int64Field]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*Int64FieldData).NumRows)
|
||||
assert.Equal(t, []int64{1, 2}, f.(*Int64FieldData).Data)
|
||||
|
||||
f, ok = merged.Data[FloatField]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*FloatFieldData).NumRows)
|
||||
assert.Equal(t, []float32{0, 0}, f.(*FloatFieldData).Data)
|
||||
|
||||
f, ok = merged.Data[DoubleField]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*DoubleFieldData).NumRows)
|
||||
assert.Equal(t, []float64{0, 0}, f.(*DoubleFieldData).Data)
|
||||
|
||||
f, ok = merged.Data[StringField]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*StringFieldData).NumRows)
|
||||
assert.Equal(t, []string{"1", "2"}, f.(*StringFieldData).Data)
|
||||
|
||||
f, ok = merged.Data[BinaryVectorField]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*BinaryVectorFieldData).NumRows)
|
||||
assert.Equal(t, []byte{0, 0}, f.(*BinaryVectorFieldData).Data)
|
||||
|
||||
f, ok = merged.Data[FloatVectorField]
|
||||
assert.True(t, ok)
|
||||
assert.Equal(t, []int64{2}, f.(*FloatVectorFieldData).NumRows)
|
||||
assert.Equal(t, []float32{0, 0}, f.(*FloatVectorFieldData).Data)
|
||||
}
|
||||
|
||||
func TestGetPkFromInsertData(t *testing.T) {
|
||||
var nilSchema *schemapb.CollectionSchema
|
||||
_, err := GetPkFromInsertData(nilSchema, nil)
|
||||
assert.Error(t, err)
|
||||
|
||||
noPfSchema := &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.StartOfUserFieldID,
|
||||
Name: "no_pf_schema",
|
||||
IsPrimaryKey: false,
|
||||
Description: "no pf schema",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
AutoID: false,
|
||||
},
|
||||
},
|
||||
}
|
||||
_, err = GetPkFromInsertData(noPfSchema, nil)
|
||||
assert.Error(t, err)
|
||||
|
||||
pfSchema := &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.StartOfUserFieldID,
|
||||
Name: "pf_schema",
|
||||
IsPrimaryKey: true,
|
||||
Description: "pf schema",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
AutoID: false,
|
||||
},
|
||||
},
|
||||
}
|
||||
noPfData := &InsertData{
|
||||
Data: map[FieldID]FieldData{},
|
||||
Infos: nil,
|
||||
}
|
||||
_, err = GetPkFromInsertData(pfSchema, noPfData)
|
||||
assert.Error(t, err)
|
||||
|
||||
notInt64Data := &InsertData{
|
||||
Data: map[FieldID]FieldData{
|
||||
common.StartOfUserFieldID: &BoolFieldData{},
|
||||
},
|
||||
Infos: nil,
|
||||
}
|
||||
_, err = GetPkFromInsertData(pfSchema, notInt64Data)
|
||||
assert.Error(t, err)
|
||||
|
||||
realInt64Data := &InsertData{
|
||||
Data: map[FieldID]FieldData{
|
||||
common.StartOfUserFieldID: &Int64FieldData{
|
||||
Data: []int64{1, 2, 3},
|
||||
},
|
||||
},
|
||||
Infos: nil,
|
||||
}
|
||||
d, err := GetPkFromInsertData(pfSchema, realInt64Data)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, []int64{1, 2, 3}, d)
|
||||
}
|
||||
|
|
|
@ -0,0 +1,244 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package typeutil
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"errors"
|
||||
"io"
|
||||
"reflect"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type rowsHelper = [][]interface{}
|
||||
|
||||
func appendScalarField(datas *rowsHelper, rowNum *int, getDataFunc func() interface{}) error {
|
||||
fieldDatas := reflect.ValueOf(getDataFunc())
|
||||
if *rowNum != 0 && *rowNum != fieldDatas.Len() {
|
||||
return errors.New("the row num of different column is not equal")
|
||||
}
|
||||
*rowNum = fieldDatas.Len()
|
||||
*datas = append(*datas, make([]interface{}, 0, *rowNum))
|
||||
idx := len(*datas) - 1
|
||||
for i := 0; i < *rowNum; i++ {
|
||||
(*datas)[idx] = append((*datas)[idx], fieldDatas.Index(i).Interface())
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func appendFloatVectorField(datas *rowsHelper, rowNum *int, fDatas []float32, dim int64) error {
|
||||
l := len(fDatas)
|
||||
if int64(l)%dim != 0 {
|
||||
return errors.New("invalid vectors")
|
||||
}
|
||||
r := int64(l) / dim
|
||||
if *rowNum != 0 && *rowNum != int(r) {
|
||||
return errors.New("the row num of different column is not equal")
|
||||
}
|
||||
*rowNum = int(r)
|
||||
*datas = append(*datas, make([]interface{}, 0, *rowNum))
|
||||
idx := len(*datas) - 1
|
||||
vector := make([]float32, 0, dim)
|
||||
for i := 0; i < l; i++ {
|
||||
vector = append(vector, fDatas[i])
|
||||
if int64(i+1)%dim == 0 {
|
||||
(*datas)[idx] = append((*datas)[idx], vector)
|
||||
vector = make([]float32, 0, dim)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func appendBinaryVectorField(datas *rowsHelper, rowNum *int, bDatas []byte, dim int64) error {
|
||||
l := len(bDatas)
|
||||
if dim%8 != 0 {
|
||||
return errors.New("invalid dim")
|
||||
}
|
||||
if (8*int64(l))%dim != 0 {
|
||||
return errors.New("invalid vectors")
|
||||
}
|
||||
r := (8 * int64(l)) / dim
|
||||
if *rowNum != 0 && *rowNum != int(r) {
|
||||
return errors.New("the row num of different column is not equal")
|
||||
}
|
||||
*rowNum = int(r)
|
||||
*datas = append(*datas, make([]interface{}, 0, *rowNum))
|
||||
idx := len(*datas) - 1
|
||||
vector := make([]byte, 0, dim)
|
||||
for i := 0; i < l; i++ {
|
||||
vector = append(vector, bDatas[i])
|
||||
if (8*int64(i+1))%dim == 0 {
|
||||
(*datas)[idx] = append((*datas)[idx], vector)
|
||||
vector = make([]byte, 0, dim)
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func writeToBuffer(w io.Writer, endian binary.ByteOrder, d interface{}) error {
|
||||
return binary.Write(w, endian, d)
|
||||
}
|
||||
|
||||
func TransferColumnBasedDataToRowBasedData(columns []*schemapb.FieldData) (rows []*commonpb.Blob, err error) {
|
||||
dTypes := make([]schemapb.DataType, 0, len(columns))
|
||||
datas := make([][]interface{}, 0, len(columns))
|
||||
rowNum := 0
|
||||
|
||||
for _, field := range columns {
|
||||
switch field.Field.(type) {
|
||||
case *schemapb.FieldData_Scalars:
|
||||
scalarField := field.GetScalars()
|
||||
switch scalarField.Data.(type) {
|
||||
case *schemapb.ScalarField_BoolData:
|
||||
err := appendScalarField(&datas, &rowNum, func() interface{} {
|
||||
return scalarField.GetBoolData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case *schemapb.ScalarField_IntData:
|
||||
err := appendScalarField(&datas, &rowNum, func() interface{} {
|
||||
return scalarField.GetIntData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case *schemapb.ScalarField_LongData:
|
||||
err := appendScalarField(&datas, &rowNum, func() interface{} {
|
||||
return scalarField.GetLongData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case *schemapb.ScalarField_FloatData:
|
||||
err := appendScalarField(&datas, &rowNum, func() interface{} {
|
||||
return scalarField.GetFloatData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case *schemapb.ScalarField_DoubleData:
|
||||
err := appendScalarField(&datas, &rowNum, func() interface{} {
|
||||
return scalarField.GetDoubleData().Data
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case *schemapb.ScalarField_BytesData:
|
||||
return nil, errors.New("bytes field is not supported now")
|
||||
case *schemapb.ScalarField_StringData:
|
||||
return nil, errors.New("string field is not supported now")
|
||||
case nil:
|
||||
continue
|
||||
default:
|
||||
continue
|
||||
}
|
||||
case *schemapb.FieldData_Vectors:
|
||||
vectorField := field.GetVectors()
|
||||
switch vectorField.Data.(type) {
|
||||
case *schemapb.VectorField_FloatVector:
|
||||
floatVectorFieldData := vectorField.GetFloatVector().Data
|
||||
dim := vectorField.GetDim()
|
||||
err := appendFloatVectorField(&datas, &rowNum, floatVectorFieldData, dim)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case *schemapb.VectorField_BinaryVector:
|
||||
binaryVectorFieldData := vectorField.GetBinaryVector()
|
||||
dim := vectorField.GetDim()
|
||||
err := appendBinaryVectorField(&datas, &rowNum, binaryVectorFieldData, dim)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
case nil:
|
||||
continue
|
||||
default:
|
||||
continue
|
||||
}
|
||||
case nil:
|
||||
continue
|
||||
default:
|
||||
continue
|
||||
}
|
||||
|
||||
dTypes = append(dTypes, field.Type)
|
||||
|
||||
}
|
||||
|
||||
rows = make([]*commonpb.Blob, 0, rowNum)
|
||||
l := len(dTypes)
|
||||
// TODO(dragondriver): big endian or little endian?
|
||||
endian := common.Endian
|
||||
for i := 0; i < rowNum; i++ {
|
||||
blob := &commonpb.Blob{
|
||||
Value: make([]byte, 0),
|
||||
}
|
||||
|
||||
for j := 0; j < l; j++ {
|
||||
var buffer bytes.Buffer
|
||||
var err error
|
||||
switch dTypes[j] {
|
||||
case schemapb.DataType_Bool:
|
||||
d := datas[j][i].(bool)
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
case schemapb.DataType_Int8:
|
||||
d := int8(datas[j][i].(int32))
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
case schemapb.DataType_Int16:
|
||||
d := int16(datas[j][i].(int32))
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
case schemapb.DataType_Int32:
|
||||
d := datas[j][i].(int32)
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
case schemapb.DataType_Int64:
|
||||
d := datas[j][i].(int64)
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
case schemapb.DataType_Float:
|
||||
d := datas[j][i].(float32)
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
case schemapb.DataType_Double:
|
||||
d := datas[j][i].(float64)
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
case schemapb.DataType_FloatVector:
|
||||
d := datas[j][i].([]float32)
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
case schemapb.DataType_BinaryVector:
|
||||
d := datas[j][i].([]byte)
|
||||
err = writeToBuffer(&buffer, endian, d)
|
||||
default:
|
||||
log.Warn("unsupported data type", zap.String("type", dTypes[j].String()))
|
||||
}
|
||||
if err != nil {
|
||||
log.Error("failed to write to buffer", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
blob.Value = append(blob.Value, buffer.Bytes()...)
|
||||
}
|
||||
rows = append(rows, blob)
|
||||
}
|
||||
|
||||
return rows, nil
|
||||
}
|
|
@ -0,0 +1,190 @@
|
|||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package typeutil
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
"testing"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/schemapb"
|
||||
)
|
||||
|
||||
func TestTransferColumnBasedDataToRowBasedData(t *testing.T) {
|
||||
columns := []*schemapb.FieldData{
|
||||
{
|
||||
Type: schemapb.DataType_Bool,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_BoolData{
|
||||
BoolData: &schemapb.BoolArray{
|
||||
Data: []bool{true, false, true},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: schemapb.DataType_Int8,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: []int32{0, 0xf, 0x1f},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: schemapb.DataType_Int16,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: []int32{0, 0xff, 0x1fff},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: schemapb.DataType_Int32,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_IntData{
|
||||
IntData: &schemapb.IntArray{
|
||||
Data: []int32{0, 0xffff, 0x1fffffff},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: schemapb.DataType_Int64,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_LongData{
|
||||
LongData: &schemapb.LongArray{
|
||||
Data: []int64{0, 0xffffffff, 0x1fffffffffffffff},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: schemapb.DataType_Float,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_FloatData{
|
||||
FloatData: &schemapb.FloatArray{
|
||||
Data: []float32{0, 0, 0},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: schemapb.DataType_Double,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
Scalars: &schemapb.ScalarField{
|
||||
Data: &schemapb.ScalarField_DoubleData{
|
||||
DoubleData: &schemapb.DoubleArray{
|
||||
Data: []float64{0, 0, 0},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: schemapb.DataType_FloatVector,
|
||||
Field: &schemapb.FieldData_Vectors{
|
||||
Vectors: &schemapb.VectorField{
|
||||
Dim: 1,
|
||||
Data: &schemapb.VectorField_FloatVector{
|
||||
FloatVector: &schemapb.FloatArray{
|
||||
Data: []float32{0, 0, 0},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Type: schemapb.DataType_BinaryVector,
|
||||
Field: &schemapb.FieldData_Vectors{
|
||||
Vectors: &schemapb.VectorField{
|
||||
Dim: 8,
|
||||
Data: &schemapb.VectorField_BinaryVector{
|
||||
BinaryVector: []byte{1, 2, 3},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
rows, err := TransferColumnBasedDataToRowBasedData(columns)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 3, len(rows))
|
||||
if common.Endian == binary.LittleEndian {
|
||||
// low byte in high address
|
||||
|
||||
assert.ElementsMatch(t,
|
||||
[]byte{
|
||||
1, // true
|
||||
0, // 0
|
||||
0, 0, // 0
|
||||
0, 0, 0, 0, // 0
|
||||
0, 0, 0, 0, 0, 0, 0, 0, // 0
|
||||
0, 0, 0, 0, // 0
|
||||
0, 0, 0, 0, 0, 0, 0, 0, // 0
|
||||
// b + 1, // "1"
|
||||
1, // 1
|
||||
0, 0, 0, 0, // 0
|
||||
},
|
||||
rows[0].Value)
|
||||
assert.ElementsMatch(t,
|
||||
[]byte{
|
||||
0, // false
|
||||
0xf, // 0xf
|
||||
0, 0xff, // 0xff
|
||||
0, 0, 0xff, 0xff, // 0xffff
|
||||
0, 0, 0, 0, 0xff, 0xff, 0xff, 0xff, // 0xffffffff
|
||||
0, 0, 0, 0, // 0
|
||||
0, 0, 0, 0, 0, 0, 0, 0, // 0
|
||||
// b + 2, // "2"
|
||||
2, // 2
|
||||
0, 0, 0, 0, // 0
|
||||
},
|
||||
rows[1].Value)
|
||||
assert.ElementsMatch(t,
|
||||
[]byte{
|
||||
1, // false
|
||||
0x1f, // 0x1f
|
||||
0xff, 0x1f, // 0x1fff
|
||||
0xff, 0xff, 0xff, 0x1f, // 0x1fffffff
|
||||
0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0x1f, // 0x1fffffffffffffff
|
||||
0, 0, 0, 0, // 0
|
||||
0, 0, 0, 0, 0, 0, 0, 0, // 0
|
||||
// b + 3, // "3"
|
||||
3, // 3
|
||||
0, 0, 0, 0, // 0
|
||||
},
|
||||
rows[2].Value)
|
||||
}
|
||||
}
|
|
@ -196,6 +196,7 @@ func AppendFieldData(dst []*schemapb.FieldData, src []*schemapb.FieldData, idx i
|
|||
case *schemapb.FieldData_Scalars:
|
||||
if dst[i] == nil || dst[i].GetScalars() == nil {
|
||||
dst[i] = &schemapb.FieldData{
|
||||
Type: fieldData.Type,
|
||||
FieldName: fieldData.FieldName,
|
||||
FieldId: fieldData.FieldId,
|
||||
Field: &schemapb.FieldData_Scalars{
|
||||
|
@ -262,6 +263,7 @@ func AppendFieldData(dst []*schemapb.FieldData, src []*schemapb.FieldData, idx i
|
|||
dim := fieldType.Vectors.Dim
|
||||
if dst[i] == nil || dst[i].GetVectors() == nil {
|
||||
dst[i] = &schemapb.FieldData{
|
||||
Type: fieldData.Type,
|
||||
FieldName: fieldData.FieldName,
|
||||
FieldId: fieldData.FieldId,
|
||||
Field: &schemapb.FieldData_Vectors{
|
||||
|
@ -302,3 +304,18 @@ func AppendFieldData(dst []*schemapb.FieldData, src []*schemapb.FieldData, idx i
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
func FillFieldBySchema(columns []*schemapb.FieldData, schema *schemapb.CollectionSchema) error {
|
||||
if len(columns) != len(schema.GetFields()) {
|
||||
return fmt.Errorf("len(columns) mismatch the len(fields), len(columns): %d, len(fields): %d",
|
||||
len(columns), len(schema.GetFields()))
|
||||
}
|
||||
|
||||
for idx, f := range schema.GetFields() {
|
||||
columns[idx].FieldName = f.Name
|
||||
columns[idx].Type = f.DataType
|
||||
columns[idx].FieldId = f.FieldID
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -505,3 +505,30 @@ func TestAppendFieldData(t *testing.T) {
|
|||
assert.Equal(t, BinaryVector, result[5].GetVectors().Data.(*schemapb.VectorField_BinaryVector).BinaryVector)
|
||||
assert.Equal(t, FloatVector, result[6].GetVectors().GetFloatVector().Data)
|
||||
}
|
||||
|
||||
func TestFillFieldBySchema(t *testing.T) {
|
||||
columns := []*schemapb.FieldData{
|
||||
{},
|
||||
}
|
||||
schema := &schemapb.CollectionSchema{}
|
||||
// length mismatch
|
||||
assert.Error(t, FillFieldBySchema(columns, schema))
|
||||
columns = []*schemapb.FieldData{
|
||||
{
|
||||
FieldId: 0,
|
||||
},
|
||||
}
|
||||
schema = &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
Name: "TestFillFieldIDBySchema",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
FieldID: 1,
|
||||
},
|
||||
},
|
||||
}
|
||||
assert.NoError(t, FillFieldBySchema(columns, schema))
|
||||
assert.Equal(t, "TestFillFieldIDBySchema", columns[0].FieldName)
|
||||
assert.Equal(t, schemapb.DataType_Int64, columns[0].Type)
|
||||
assert.Equal(t, int64(1), columns[0].FieldId)
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue