mirror of https://github.com/milvus-io/milvus.git
parent
21b5973253
commit
68b1b82faf
|
@ -50,13 +50,6 @@ type BaseKV interface {
|
|||
Close()
|
||||
}
|
||||
|
||||
// DataKV persists the data.
|
||||
type DataKV interface {
|
||||
BaseKV
|
||||
LoadPartial(key string, start, end int64) ([]byte, error)
|
||||
GetSize(key string) (int64, error)
|
||||
}
|
||||
|
||||
// TxnKV contains extra txn operations of kv. The extra operations is transactional.
|
||||
type TxnKV interface {
|
||||
BaseKV
|
||||
|
|
|
@ -24,7 +24,7 @@ import (
|
|||
"github.com/google/btree"
|
||||
)
|
||||
|
||||
// MemoryKV implements DataKV interface and relies on underling btree.BTree.
|
||||
// MemoryKV implements BaseKv interface and relies on underling btree.BTree.
|
||||
// As its name implies, all data is stored in memory.
|
||||
type MemoryKV struct {
|
||||
sync.RWMutex
|
||||
|
@ -380,27 +380,3 @@ func (kv *MemoryKV) RemoveWithPrefix(key string) error {
|
|||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// LoadPartial item already in memory, just slice the value.
|
||||
func (kv *MemoryKV) LoadPartial(key string, start, end int64) ([]byte, error) {
|
||||
value, err := kv.Load(key)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
switch {
|
||||
case 0 <= start && start < end && end <= int64(len(value)):
|
||||
return []byte(value[start:end]), nil
|
||||
default:
|
||||
return nil, fmt.Errorf("invalid range specified: start=%d end=%d",
|
||||
start, end)
|
||||
}
|
||||
}
|
||||
|
||||
func (kv *MemoryKV) GetSize(key string) (int64, error) {
|
||||
value, err := kv.Load(key)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
|
||||
return int64(len(value)), nil
|
||||
}
|
||||
|
|
|
@ -198,63 +198,3 @@ func TestMemoryKV_MultiSaveBytesAndRemoveWithPrefix(t *testing.T) {
|
|||
assert.ElementsMatch(t, values, _values)
|
||||
assert.NoError(t, err)
|
||||
}
|
||||
|
||||
func TestMemoryKV_LoadPartial(t *testing.T) {
|
||||
memKV := NewMemoryKV()
|
||||
|
||||
key := "TestMemoryKV_LoadPartial_key"
|
||||
value := "TestMemoryKV_LoadPartial_value"
|
||||
|
||||
err := memKV.Save(key, value)
|
||||
assert.NoError(t, err)
|
||||
|
||||
var start, end int64
|
||||
var partial []byte
|
||||
|
||||
// case 0 <= start && start = end && end <= int64(len(value))
|
||||
|
||||
start, end = 1, 2
|
||||
partial, err = memKV.LoadPartial(key, start, end)
|
||||
assert.NoError(t, err)
|
||||
assert.ElementsMatch(t, partial, []byte(value[start:end]))
|
||||
|
||||
start, end = int64(len(value)-2), int64(len(value)-1)
|
||||
partial, err = memKV.LoadPartial(key, start, end)
|
||||
assert.NoError(t, err)
|
||||
assert.ElementsMatch(t, partial, []byte(value[start:end]))
|
||||
|
||||
// error case
|
||||
start, end = 5, 3
|
||||
_, err = memKV.LoadPartial(key, start, end)
|
||||
assert.Error(t, err)
|
||||
|
||||
start, end = 1, 1
|
||||
_, err = memKV.LoadPartial(key, start, end)
|
||||
assert.Error(t, err)
|
||||
|
||||
err = memKV.Remove(key)
|
||||
assert.NoError(t, err)
|
||||
start, end = 1, 2
|
||||
_, err = memKV.LoadPartial(key, start, end)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func TestMemoryKV_GetSize(t *testing.T) {
|
||||
memKV := NewMemoryKV()
|
||||
|
||||
key := "TestMemoryKV_GetSize_key"
|
||||
value := "TestMemoryKV_GetSize_value"
|
||||
|
||||
err := memKV.Save(key, value)
|
||||
assert.NoError(t, err)
|
||||
|
||||
size, err := memKV.GetSize(key)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, size, int64(len(value)))
|
||||
|
||||
key2 := "TestMemoryKV_GetSize_key2"
|
||||
|
||||
size, err = memKV.GetSize(key2)
|
||||
assert.Error(t, err)
|
||||
assert.Equal(t, int64(0), size)
|
||||
}
|
||||
|
|
|
@ -30,10 +30,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
@ -46,6 +42,7 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/util/etcd"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
const (
|
||||
|
@ -227,47 +224,6 @@ func genStorageBlob(collectionID UniqueID,
|
|||
return binLogs, err
|
||||
}
|
||||
|
||||
func saveSimpleBinLog(ctx context.Context, schema *schemapb.CollectionSchema, dataKV kv.DataKV) ([]*datapb.FieldBinlog, error) {
|
||||
return saveBinLog(ctx, defaultCollectionID, defaultPartitionID, defaultSegmentID, defaultNumRowPerSegment, schema, dataKV)
|
||||
}
|
||||
|
||||
func saveBinLog(ctx context.Context,
|
||||
collectionID UniqueID,
|
||||
partitionID UniqueID,
|
||||
segmentID UniqueID,
|
||||
msgLength int,
|
||||
schema *schemapb.CollectionSchema,
|
||||
dataKV kv.DataKV) ([]*datapb.FieldBinlog, error) {
|
||||
binLogs, err := genStorageBlob(collectionID, partitionID, segmentID, msgLength, schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
log.Debug(".. [query coord unittest] Saving bin logs to MinIO ..", zap.Int("number", len(binLogs)))
|
||||
kvs := make(map[string]string, len(binLogs))
|
||||
|
||||
// write insert binlog
|
||||
fieldBinlog := make([]*datapb.FieldBinlog, 0)
|
||||
for _, blob := range binLogs {
|
||||
fieldID, err := strconv.ParseInt(blob.GetKey(), 10, 64)
|
||||
log.Debug("[QueryCoord unittest] save binlog", zap.Int64("fieldID", fieldID))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
key := genKey(collectionID, partitionID, segmentID, fieldID)
|
||||
kvs[key] = string(blob.Value[:])
|
||||
fieldBinlog = append(fieldBinlog, &datapb.FieldBinlog{
|
||||
FieldID: fieldID,
|
||||
Binlogs: []*datapb.Binlog{{LogPath: key}},
|
||||
})
|
||||
}
|
||||
log.Debug("[QueryCoord unittest] save binlog file to MinIO/S3")
|
||||
|
||||
err = dataKV.MultiSave(kvs)
|
||||
return fieldBinlog, err
|
||||
}
|
||||
|
||||
func genKey(collectionID, partitionID, segmentID UniqueID, fieldID int64) string {
|
||||
ids := []string{
|
||||
defaultKVRootPath,
|
||||
|
|
|
@ -351,7 +351,7 @@ func (loader *segmentLoader) loadGrowingFields(segment *Segment, fieldBinlogs []
|
|||
return loader.loadGrowingSegments(segment, ids, timestamps, rowData)
|
||||
}
|
||||
|
||||
// Load binlogs concurrently into memory from DataKV
|
||||
// Load binlogs concurrently into memory from KV storage
|
||||
func (loader *segmentLoader) loadFieldBinlogs(field *datapb.FieldBinlog) ([]*storage.Blob, error) {
|
||||
log.Debug("load field binlogs",
|
||||
zap.Int64("fieldID", field.FieldID),
|
||||
|
|
|
@ -38,83 +38,8 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/common"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
)
|
||||
|
||||
// GetBinlogSize get size of a binlog file.
|
||||
// normal binlog file, error = nil;
|
||||
// key not exist, size = 0, error = nil;
|
||||
// key not in binlog format, size = (a not accurate number), error != nil;
|
||||
// failed to read event reader, size = (a not accurate number), error != nil;
|
||||
func GetBinlogSize(kv kv.DataKV, key string) (int64, error) {
|
||||
|
||||
return kv.GetSize(key)
|
||||
}
|
||||
|
||||
// EstimateMemorySize get approximate memory size of a binlog file.
|
||||
// 1, key not exist, size = 0, error != nil;
|
||||
// 2, failed to read event header, size = 0, error != nil;
|
||||
// 3, invalid event length, size = 0, error != nil;
|
||||
// 4, failed to read descriptor event, size = 0, error != nil;
|
||||
// 5, original_size not in extra, size = 0, error != nil;
|
||||
// 6, original_size not in int format, size = 0, error != nil;
|
||||
// 7, normal binlog with original_size, return original_size, error = nil;
|
||||
func EstimateMemorySize(kv kv.DataKV, key string) (int64, error) {
|
||||
total := int64(0)
|
||||
|
||||
header := &eventHeader{}
|
||||
headerSize := binary.Size(header)
|
||||
|
||||
startPos := binary.Size(MagicNumber)
|
||||
endPos := startPos + headerSize
|
||||
|
||||
// get header
|
||||
headerContent, err := kv.LoadPartial(key, int64(startPos), int64(endPos))
|
||||
if err != nil {
|
||||
return total, err
|
||||
}
|
||||
|
||||
buffer := bytes.NewBuffer(headerContent)
|
||||
|
||||
header, err = readEventHeader(buffer)
|
||||
if err != nil {
|
||||
return total, err
|
||||
}
|
||||
|
||||
if header.EventLength <= 0 {
|
||||
return total, fmt.Errorf("key %v not in binlog format", key)
|
||||
}
|
||||
|
||||
var desc *descriptorEvent
|
||||
endPos = startPos + int(header.EventLength)
|
||||
descContent, err := kv.LoadPartial(key, int64(startPos), int64(endPos))
|
||||
if err != nil {
|
||||
return total, err
|
||||
}
|
||||
|
||||
buffer = bytes.NewBuffer(descContent)
|
||||
|
||||
desc, err = ReadDescriptorEvent(buffer)
|
||||
if err != nil {
|
||||
return total, err
|
||||
}
|
||||
|
||||
sizeStr, ok := desc.Extras[originalSizeKey]
|
||||
if !ok {
|
||||
return total, fmt.Errorf("key %v not in extra information", originalSizeKey)
|
||||
}
|
||||
|
||||
size, err := strconv.Atoi(fmt.Sprintf("%v", sizeStr))
|
||||
if err != nil {
|
||||
return total, fmt.Errorf("%v not in valid format, value: %v", originalSizeKey, sizeStr)
|
||||
}
|
||||
|
||||
total = int64(size)
|
||||
|
||||
return total, nil
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
func checkTsField(data *InsertData) bool {
|
||||
|
|
|
@ -19,9 +19,6 @@ package storage
|
|||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"strconv"
|
||||
"testing"
|
||||
|
@ -35,363 +32,9 @@ import (
|
|||
"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"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/kv"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
memkv "github.com/milvus-io/milvus/internal/kv/mem"
|
||||
)
|
||||
|
||||
type mockLessHeaderDataKV struct {
|
||||
kv.BaseKV
|
||||
}
|
||||
|
||||
func (kv *mockLessHeaderDataKV) LoadPartial(key string, start, end int64) ([]byte, error) {
|
||||
header := &baseEventHeader{}
|
||||
|
||||
headerSize := binary.Size(header)
|
||||
mockSize := headerSize - 1
|
||||
|
||||
ret := make([]byte, mockSize)
|
||||
_, _ = rand.Read(ret)
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
func (kv *mockLessHeaderDataKV) GetSize(key string) (int64, error) {
|
||||
return 0, errors.New("less header")
|
||||
}
|
||||
|
||||
func newMockLessHeaderDataKV() *mockLessHeaderDataKV {
|
||||
return &mockLessHeaderDataKV{}
|
||||
}
|
||||
|
||||
type mockWrongHeaderDataKV struct {
|
||||
kv.BaseKV
|
||||
}
|
||||
|
||||
func (kv *mockWrongHeaderDataKV) LoadPartial(key string, start, end int64) ([]byte, error) {
|
||||
header := &baseEventHeader{}
|
||||
|
||||
header.EventLength = -1
|
||||
header.NextPosition = -1
|
||||
|
||||
buffer := bytes.Buffer{}
|
||||
_ = binary.Write(&buffer, common.Endian, header)
|
||||
|
||||
return buffer.Bytes(), nil
|
||||
}
|
||||
|
||||
func (kv *mockWrongHeaderDataKV) GetSize(key string) (int64, error) {
|
||||
return 0, errors.New("wrong header")
|
||||
}
|
||||
|
||||
func newMockWrongHeaderDataKV() kv.DataKV {
|
||||
return &mockWrongHeaderDataKV{}
|
||||
}
|
||||
|
||||
func TestGetBinlogSize(t *testing.T) {
|
||||
memoryKV := memkv.NewMemoryKV()
|
||||
defer memoryKV.Close()
|
||||
|
||||
key := "TestGetBinlogSize"
|
||||
|
||||
var size int64
|
||||
var err error
|
||||
|
||||
// key not in memoryKV
|
||||
size, err = GetBinlogSize(memoryKV, key)
|
||||
assert.Error(t, err)
|
||||
assert.Zero(t, size)
|
||||
|
||||
// normal binlog key, for example, index binlog
|
||||
indexBuildID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
version := int64(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
collectionID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
partitionID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
segmentID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
fieldID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
indexName := funcutil.GenRandomStr()
|
||||
indexID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
indexParams := make(map[string]string)
|
||||
indexParams["index_type"] = "IVF_FLAT"
|
||||
datas := []*Blob{
|
||||
{
|
||||
Key: "ivf1",
|
||||
Value: []byte{1, 2, 3},
|
||||
},
|
||||
{
|
||||
Key: "ivf2",
|
||||
Value: []byte{4, 5, 6},
|
||||
},
|
||||
{
|
||||
Key: "large",
|
||||
Value: []byte(funcutil.RandomString(maxLengthPerRowOfIndexFile + 1)),
|
||||
},
|
||||
}
|
||||
|
||||
codec := NewIndexFileBinlogCodec()
|
||||
|
||||
serializedBlobs, err := codec.Serialize(indexBuildID, version, collectionID, partitionID, segmentID, fieldID, indexParams, indexName, indexID, datas)
|
||||
assert.Nil(t, err)
|
||||
|
||||
for _, blob := range serializedBlobs {
|
||||
err = memoryKV.Save(blob.Key, string(blob.Value))
|
||||
assert.Nil(t, err)
|
||||
|
||||
size, err = GetBinlogSize(memoryKV, blob.Key)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, size, int64(len(blob.Value)))
|
||||
}
|
||||
}
|
||||
|
||||
// cover case that failed to read event header
|
||||
func TestGetBinlogSize_less_header(t *testing.T) {
|
||||
mockKV := newMockLessHeaderDataKV()
|
||||
|
||||
key := "TestGetBinlogSize_less_header"
|
||||
|
||||
_, err := GetBinlogSize(mockKV, key)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
// cover case that file not in binlog format
|
||||
func TestGetBinlogSize_not_in_binlog_format(t *testing.T) {
|
||||
mockKV := newMockWrongHeaderDataKV()
|
||||
|
||||
key := "TestGetBinlogSize_not_in_binlog_format"
|
||||
|
||||
_, err := GetBinlogSize(mockKV, key)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func TestEstimateMemorySize(t *testing.T) {
|
||||
memoryKV := memkv.NewMemoryKV()
|
||||
defer memoryKV.Close()
|
||||
|
||||
key := "TestEstimateMemorySize"
|
||||
|
||||
var size int64
|
||||
var err error
|
||||
|
||||
// key not in memoryKV
|
||||
_, err = EstimateMemorySize(memoryKV, key)
|
||||
assert.Error(t, err)
|
||||
|
||||
// normal binlog key, for example, index binlog
|
||||
indexBuildID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
version := int64(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
collectionID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
partitionID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
segmentID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
fieldID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
indexName := funcutil.GenRandomStr()
|
||||
indexID := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt())
|
||||
indexParams := make(map[string]string)
|
||||
indexParams["index_type"] = "IVF_FLAT"
|
||||
datas := []*Blob{
|
||||
{
|
||||
Key: "ivf1",
|
||||
Value: []byte{1, 2, 3},
|
||||
},
|
||||
{
|
||||
Key: "ivf2",
|
||||
Value: []byte{4, 5, 6},
|
||||
},
|
||||
{
|
||||
Key: "large",
|
||||
Value: []byte(funcutil.RandomString(maxLengthPerRowOfIndexFile + 1)),
|
||||
},
|
||||
}
|
||||
|
||||
codec := NewIndexFileBinlogCodec()
|
||||
|
||||
serializedBlobs, err := codec.Serialize(indexBuildID, version, collectionID, partitionID, segmentID, fieldID, indexParams, indexName, indexID, datas)
|
||||
assert.Nil(t, err)
|
||||
|
||||
for _, blob := range serializedBlobs {
|
||||
err = memoryKV.Save(blob.Key, string(blob.Value))
|
||||
assert.Nil(t, err)
|
||||
|
||||
buf := bytes.NewBuffer(blob.Value)
|
||||
|
||||
_, _ = readMagicNumber(buf)
|
||||
desc, _ := ReadDescriptorEvent(buf)
|
||||
|
||||
size, err = EstimateMemorySize(memoryKV, blob.Key)
|
||||
assert.Nil(t, err)
|
||||
assert.Equal(t, fmt.Sprintf("%v", desc.Extras[originalSizeKey]), fmt.Sprintf("%v", size))
|
||||
}
|
||||
}
|
||||
|
||||
// cover case that failed to read event header
|
||||
func TestEstimateMemorySize_less_header(t *testing.T) {
|
||||
mockKV := newMockLessHeaderDataKV()
|
||||
|
||||
key := "TestEstimateMemorySize_less_header"
|
||||
|
||||
_, err := EstimateMemorySize(mockKV, key)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
// cover case that file not in binlog format
|
||||
func TestEstimateMemorySize_not_in_binlog_format(t *testing.T) {
|
||||
mockKV := newMockWrongHeaderDataKV()
|
||||
|
||||
key := "TestEstimateMemorySize_not_in_binlog_format"
|
||||
|
||||
_, err := EstimateMemorySize(mockKV, key)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
type mockFailedToGetDescDataKV struct {
|
||||
kv.BaseKV
|
||||
}
|
||||
|
||||
func (kv *mockFailedToGetDescDataKV) LoadPartial(key string, start, end int64) ([]byte, error) {
|
||||
header := &eventHeader{}
|
||||
header.EventLength = 20
|
||||
headerSize := binary.Size(header)
|
||||
|
||||
if end-start > int64(headerSize) {
|
||||
return nil, errors.New("mock failed to get desc data")
|
||||
}
|
||||
|
||||
buf := bytes.Buffer{}
|
||||
_ = binary.Write(&buf, common.Endian, header)
|
||||
return buf.Bytes(), nil
|
||||
}
|
||||
|
||||
func (kv *mockFailedToGetDescDataKV) GetSize(key string) (int64, error) {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
func newMockFailedToGetDescDataKV() *mockFailedToGetDescDataKV {
|
||||
return &mockFailedToGetDescDataKV{}
|
||||
}
|
||||
|
||||
// cover case that failed to get descriptor event content
|
||||
func TestEstimateMemorySize_failed_to_load_desc(t *testing.T) {
|
||||
mockKV := newMockFailedToGetDescDataKV()
|
||||
|
||||
key := "TestEstimateMemorySize_failed_to_load_desc"
|
||||
|
||||
_, err := EstimateMemorySize(mockKV, key)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
type mockLessDescDataKV struct {
|
||||
kv.BaseKV
|
||||
}
|
||||
|
||||
func (kv *mockLessDescDataKV) LoadPartial(key string, start, end int64) ([]byte, error) {
|
||||
header := &baseEventHeader{}
|
||||
header.EventLength = 20
|
||||
|
||||
buffer := bytes.Buffer{}
|
||||
_ = binary.Write(&buffer, common.Endian, header)
|
||||
|
||||
// no event data
|
||||
return buffer.Bytes(), nil
|
||||
|
||||
/*
|
||||
desc := &descriptorEvent{}
|
||||
desc.ExtraLength = 2
|
||||
desc.ExtraBytes = []byte{1, 2}
|
||||
buffer := bytes.Buffer{}
|
||||
_ = binary.Write(&buffer, common.Endian, desc)
|
||||
// extra not in json format
|
||||
return buffer.Bytes(), nil
|
||||
*/
|
||||
}
|
||||
|
||||
func (kv *mockLessDescDataKV) GetSize(key string) (int64, error) {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
func newMockLessDescDataKV() *mockLessDescDataKV {
|
||||
return &mockLessDescDataKV{}
|
||||
}
|
||||
|
||||
func TestEstimateMemorySize_less_desc_data(t *testing.T) {
|
||||
mockKV := newMockLessDescDataKV()
|
||||
|
||||
key := "TestEstimateMemorySize_less_desc_data"
|
||||
|
||||
_, err := EstimateMemorySize(mockKV, key)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
type mockOriginalSizeDataKV struct {
|
||||
kv.BaseKV
|
||||
impl func(key string, start, end int64) ([]byte, error)
|
||||
}
|
||||
|
||||
func (kv *mockOriginalSizeDataKV) LoadPartial(key string, start, end int64) ([]byte, error) {
|
||||
if kv.impl != nil {
|
||||
return kv.impl(key, start, end)
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (kv *mockOriginalSizeDataKV) GetSize(key string) (int64, error) {
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
func newMockOriginalSizeDataKV() *mockOriginalSizeDataKV {
|
||||
return &mockOriginalSizeDataKV{}
|
||||
}
|
||||
|
||||
func TestEstimateMemorySize_no_original_size(t *testing.T) {
|
||||
mockKV := newMockOriginalSizeDataKV()
|
||||
mockKV.impl = func(key string, start, end int64) ([]byte, error) {
|
||||
desc := &descriptorEvent{}
|
||||
desc.descriptorEventHeader.EventLength = 20
|
||||
desc.descriptorEventData = *newDescriptorEventData()
|
||||
extra := make(map[string]interface{})
|
||||
extra["key"] = "value"
|
||||
extraBytes, _ := json.Marshal(extra)
|
||||
desc.ExtraBytes = extraBytes
|
||||
desc.ExtraLength = int32(len(extraBytes))
|
||||
buf := bytes.Buffer{}
|
||||
_ = desc.descriptorEventHeader.Write(&buf)
|
||||
_ = desc.descriptorEventData.Write(&buf)
|
||||
return buf.Bytes(), nil
|
||||
}
|
||||
|
||||
key := "TestEstimateMemorySize_no_original_size"
|
||||
|
||||
_, err := EstimateMemorySize(mockKV, key)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
func TestEstimateMemorySize_cannot_convert_original_size_to_int(t *testing.T) {
|
||||
mockKV := newMockOriginalSizeDataKV()
|
||||
mockKV.impl = func(key string, start, end int64) ([]byte, error) {
|
||||
desc := &descriptorEvent{}
|
||||
desc.descriptorEventHeader.EventLength = 20
|
||||
desc.descriptorEventData = *newDescriptorEventData()
|
||||
extra := make(map[string]interface{})
|
||||
extra[originalSizeKey] = "value"
|
||||
extraBytes, _ := json.Marshal(extra)
|
||||
desc.ExtraBytes = extraBytes
|
||||
desc.ExtraLength = int32(len(extraBytes))
|
||||
buf := bytes.Buffer{}
|
||||
_ = desc.descriptorEventHeader.Write(&buf)
|
||||
_ = desc.descriptorEventData.Write(&buf)
|
||||
return buf.Bytes(), nil
|
||||
}
|
||||
|
||||
key := "TestEstimateMemorySize_cannot_convert_original_size_to_int"
|
||||
|
||||
_, err := EstimateMemorySize(mockKV, key)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
|
||||
//////////////////////////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
func TestCheckTsField(t *testing.T) {
|
||||
data := &InsertData{
|
||||
Data: make(map[FieldID]FieldData),
|
||||
|
|
|
@ -27,7 +27,11 @@ while [ -h "$SOURCE" ]; do # resolve $SOURCE until the file is no longer a symli
|
|||
done
|
||||
ROOT_DIR="$( cd -P "$( dirname "$SOURCE" )/.." && pwd )"
|
||||
|
||||
if [[ $(uname -s) == "Darwin" && "$(uname -m)" == "arm64" ]]; then
|
||||
if [[ $(uname -s) == "Darwin" ]]; then
|
||||
export MallocNanoZone=0
|
||||
fi
|
||||
|
||||
if [[ "$(uname -m)" == "arm64" ]]; then
|
||||
APPLE_SILICON_FLAG="-tags dynamic"
|
||||
fi
|
||||
|
||||
|
|
Loading…
Reference in New Issue