Fix search error when running single node

Signed-off-by: xige-16 <xi.ge@zilliz.com>
This commit is contained in:
xige-16 2021-03-12 19:23:06 +08:00 committed by yefu.chen
parent e33d0a797c
commit 8a5c039137
10 changed files with 255 additions and 109 deletions

View File

@ -1,4 +1,4 @@
timeout(time: 60, unit: 'MINUTES') {
timeout(time: 120, unit: 'MINUTES') {
container('deploy-env') {
dir ('milvus-helm-chart') {
sh " helm version && \

View File

@ -172,11 +172,21 @@ Insert(CSegmentInterface c_segment,
}
}
int64_t
PreInsert(CSegmentInterface c_segment, int64_t size) {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
return segment->PreInsert(size);
CStatus
PreInsert(CSegmentInterface c_segment, int64_t size, int64_t* offset) {
try {
auto segment = (milvus::segcore::SegmentGrowing*)c_segment;
*offset = segment->PreInsert(size);
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedException;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus

View File

@ -67,8 +67,8 @@ Insert(CSegmentInterface c_segment,
int sizeof_per_row,
int64_t count);
int64_t
PreInsert(CSegmentInterface c_segment, int64_t size);
CStatus
PreInsert(CSegmentInterface c_segment, int64_t size, int64_t* offset);
CStatus
Delete(CSegmentInterface c_segment,

View File

@ -82,7 +82,8 @@ TEST(CApiTest, InsertTest) {
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
auto offset = PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
@ -134,7 +135,8 @@ TEST(CApiTest, SearchTest) {
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
auto offset = PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto ins_res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
ASSERT_EQ(ins_res.error_code, Success);
@ -334,7 +336,9 @@ TEST(CApiTest, GetMemoryUsageInBytesTest) {
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
auto offset = PreInsert(segment, N);
// auto offset = PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
@ -574,7 +578,9 @@ TEST(CApiTest, GetRowCountTest) {
int N = 10000;
auto [raw_data, timestamps, uids] = generate_data(N);
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
auto offset = PreInsert(segment, N);
// auto offset = PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
assert(res.error_code == Success);
@ -652,8 +658,9 @@ TEST(CApiTest, Reduce) {
auto line_sizeof = (sizeof(int) + sizeof(float) * 16);
auto offset = PreInsert(segment, N);
// auto offset = PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto ins_res = Insert(segment, offset, N, uids.data(), timestamps.data(), raw_data.data(), (int)line_sizeof, N);
assert(ins_res.error_code == Success);
@ -853,7 +860,8 @@ TEST(CApiTest, UpdateSegmentIndex_Without_Predicate) {
auto vec_col = dataset.get_col<float>(0);
auto query_ptr = vec_col.data() + 420000 * DIM;
PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto ins_res = Insert(segment, 0, N, dataset.row_ids_.data(), dataset.timestamps_.data(), dataset.raw_.raw_data,
dataset.raw_.sizeof_per_row, dataset.raw_.count);
assert(ins_res.error_code == Success);
@ -978,7 +986,8 @@ TEST(CApiTest, UpdateSegmentIndex_With_float_Predicate_Range) {
auto vec_col = dataset.get_col<float>(0);
auto query_ptr = vec_col.data() + 420000 * DIM;
PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto ins_res = Insert(segment, 0, N, dataset.row_ids_.data(), dataset.timestamps_.data(), dataset.raw_.raw_data,
dataset.raw_.sizeof_per_row, dataset.raw_.count);
assert(ins_res.error_code == Success);
@ -1116,7 +1125,8 @@ TEST(CApiTest, UpdateSegmentIndex_With_float_Predicate_Term) {
auto vec_col = dataset.get_col<float>(0);
auto query_ptr = vec_col.data() + 420000 * DIM;
PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto ins_res = Insert(segment, 0, N, dataset.row_ids_.data(), dataset.timestamps_.data(), dataset.raw_.raw_data,
dataset.raw_.sizeof_per_row, dataset.raw_.count);
assert(ins_res.error_code == Success);
@ -1253,7 +1263,8 @@ TEST(CApiTest, UpdateSegmentIndex_With_binary_Predicate_Range) {
auto vec_col = dataset.get_col<uint8_t>(0);
auto query_ptr = vec_col.data() + 420000 * DIM / 8;
PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto ins_res = Insert(segment, 0, N, dataset.row_ids_.data(), dataset.timestamps_.data(), dataset.raw_.raw_data,
dataset.raw_.sizeof_per_row, dataset.raw_.count);
assert(ins_res.error_code == Success);
@ -1392,7 +1403,8 @@ TEST(CApiTest, UpdateSegmentIndex_With_binary_Predicate_Term) {
auto vec_col = dataset.get_col<uint8_t>(0);
auto query_ptr = vec_col.data() + 420000 * DIM / 8;
PreInsert(segment, N);
int64_t offset;
PreInsert(segment, N, &offset);
auto ins_res = Insert(segment, 0, N, dataset.row_ids_.data(), dataset.timestamps_.data(), dataset.raw_.raw_data,
dataset.raw_.sizeof_per_row, dataset.raw_.count);
assert(ins_res.error_code == Success);

View File

@ -64,8 +64,10 @@ type ReplicaInterface interface {
hasSegment(segmentID UniqueID) bool
getSegmentNum() int
setSegmentEnableIndex(segmentID UniqueID, enable bool) error
setSegmentEnableLoadBinLog(segmentID UniqueID, enable bool) error
getSegmentsToLoadBySegmentType(segType segmentType) ([]UniqueID, []UniqueID, []UniqueID)
getSegmentStatistics() []*internalpb.SegmentStats
getEnabledSegmentsBySegmentType(segType segmentType) ([]UniqueID, []UniqueID, []UniqueID)
getSegmentsBySegmentType(segType segmentType) ([]UniqueID, []UniqueID, []UniqueID)
replaceGrowingSegmentBySealedSegment(segment *Segment) error
@ -557,6 +559,20 @@ func (colReplica *collectionReplica) setSegmentEnableIndex(segmentID UniqueID, e
return nil
}
func (colReplica *collectionReplica) setSegmentEnableLoadBinLog(segmentID UniqueID, enable bool) error {
colReplica.mu.Lock()
defer colReplica.mu.Unlock()
targetSegment, err := colReplica.getSegmentByIDPrivate(segmentID)
if targetSegment.segmentType != segmentTypeGrowing {
return errors.New("unexpected segment type")
}
if err == nil && targetSegment != nil {
targetSegment.setLoadBinLogEnable(enable)
}
return nil
}
//-----------------------------------------------------------------------------------------------------
func (colReplica *collectionReplica) getTSafe() tSafer {
return colReplica.tSafe
@ -592,3 +608,29 @@ func newCollectionReplica() ReplicaInterface {
return replica
}
func (colReplica *collectionReplica) getSegmentsToLoadBySegmentType(segType segmentType) ([]UniqueID, []UniqueID, []UniqueID) {
colReplica.mu.RLock()
defer colReplica.mu.RUnlock()
targetCollectionIDs := make([]UniqueID, 0)
targetPartitionIDs := make([]UniqueID, 0)
targetSegmentIDs := make([]UniqueID, 0)
for _, segment := range colReplica.segments {
if !segment.enableLoadBinLog {
continue
}
if segment.getType() == segType {
if segType == segmentTypeSealed && !segment.getEnableIndex() {
continue
}
targetCollectionIDs = append(targetCollectionIDs, segment.collectionID)
targetPartitionIDs = append(targetPartitionIDs, segment.partitionID)
targetSegmentIDs = append(targetSegmentIDs, segment.segmentID)
}
}
return targetCollectionIDs, targetPartitionIDs, targetSegmentIDs
}

View File

@ -50,10 +50,6 @@ func (iNode *insertNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.
// 1. hash insertMessages to insertData
for _, task := range iMsg.insertMessages {
insertData.insertIDs[task.SegmentID] = append(insertData.insertIDs[task.SegmentID], task.RowIDs...)
insertData.insertTimestamps[task.SegmentID] = append(insertData.insertTimestamps[task.SegmentID], task.Timestamps...)
insertData.insertRecords[task.SegmentID] = append(insertData.insertRecords[task.SegmentID], task.RowData...)
// check if segment exists, if not, create this segment
if !iNode.replica.hasSegment(task.SegmentID) {
err := iNode.replica.addSegment(task.SegmentID, task.PartitionID, task.CollectionID, segmentTypeGrowing)
@ -62,6 +58,19 @@ func (iNode *insertNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.
continue
}
}
segment, err := iNode.replica.getSegmentByID(task.SegmentID)
if err != nil {
log.Error(err.Error())
continue
}
if segment.enableLoadBinLog {
continue
}
insertData.insertIDs[task.SegmentID] = append(insertData.insertIDs[task.SegmentID], task.RowIDs...)
insertData.insertTimestamps[task.SegmentID] = append(insertData.insertTimestamps[task.SegmentID], task.Timestamps...)
insertData.insertRecords[task.SegmentID] = append(insertData.insertRecords[task.SegmentID], task.RowData...)
}
// 2. do preInsert
@ -74,7 +83,10 @@ func (iNode *insertNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.
var numOfRecords = len(insertData.insertRecords[segmentID])
if targetSegment != nil {
var offset = targetSegment.segmentPreInsert(numOfRecords)
offset, err := targetSegment.segmentPreInsert(numOfRecords)
if err != nil {
log.Error(err.Error())
}
insertData.insertOffset[segmentID] = offset
}
}
@ -96,6 +108,10 @@ func (iNode *insertNode) Operate(ctx context.Context, in []Msg) ([]Msg, context.
func (iNode *insertNode) insert(insertData *InsertData, segmentID int64, wg *sync.WaitGroup) {
var targetSegment, err = iNode.replica.getSegmentByID(segmentID)
if targetSegment.segmentType != segmentTypeGrowing || targetSegment.enableLoadBinLog {
wg.Done()
return
}
if err != nil {
log.Error("cannot find segment:", zap.Int64("segmentID", segmentID))
// TODO: add error handling

View File

@ -47,7 +47,7 @@ func (s *loadService) close() {
}
func (s *loadService) loadSegmentActively(wg *sync.WaitGroup) {
collectionIDs, partitionIDs, segmentIDs := s.segLoader.replica.getSegmentsBySegmentType(segmentTypeGrowing)
collectionIDs, partitionIDs, segmentIDs := s.segLoader.replica.getSegmentsToLoadBySegmentType(segmentTypeGrowing)
if len(collectionIDs) <= 0 {
wg.Done()
return
@ -91,6 +91,11 @@ func (s *loadService) loadSegment(collectionID UniqueID, partitionID UniqueID, s
log.Warn(err.Error())
continue
}
err = s.segLoader.replica.setSegmentEnableLoadBinLog(segmentID, true)
if err != nil {
log.Warn(err.Error())
continue
}
err = s.loadSegmentInternal(collectionID, partitionID, segmentID, fieldIDs)
if err != nil {
log.Warn(err.Error())

View File

@ -45,8 +45,9 @@ type Segment struct {
lastMemSize int64
lastRowCount int64
once sync.Once // guards enableIndex
enableIndex bool
once sync.Once // guards enableIndex
enableIndex bool
enableLoadBinLog bool
rmMutex sync.Mutex // guards recentlyModified
recentlyModified bool
@ -148,12 +149,13 @@ func newSegment(collection *Collection, segmentID int64, partitionID UniqueID, c
log.Debug("create segment", zap.Int64("segmentID", segmentID))
var newSegment = &Segment{
segmentPtr: segmentPtr,
segmentType: segType,
segmentID: segmentID,
partitionID: partitionID,
collectionID: collectionID,
indexParam: initIndexParam,
segmentPtr: segmentPtr,
segmentType: segType,
segmentID: segmentID,
partitionID: partitionID,
collectionID: collectionID,
indexParam: initIndexParam,
enableLoadBinLog: false,
}
return newSegment
@ -331,14 +333,26 @@ func (s *Segment) matchIndexParam(fieldID int64, indexParams indexParam) bool {
}
//-------------------------------------------------------------------------------------- interfaces for growing segment
func (s *Segment) segmentPreInsert(numOfRecords int) int64 {
func (s *Segment) segmentPreInsert(numOfRecords int) (int64, error) {
/*
long int
PreInsert(CSegmentInterface c_segment, long int size);
*/
var offset = C.PreInsert(s.segmentPtr, C.long(int64(numOfRecords)))
if s.segmentType != segmentTypeGrowing || s.enableLoadBinLog {
return 0, nil
}
var offset int64
cOffset := C.long(offset)
status := C.PreInsert(s.segmentPtr, C.long(int64(numOfRecords)), &cOffset)
return int64(offset)
errorCode := status.error_code
if errorCode != 0 {
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
return 0, errors.New("PreInsert failed, C runtime error detected, error code = " + strconv.Itoa(int(errorCode)) + ", error msg = " + errorMsg)
}
return offset, nil
}
func (s *Segment) segmentPreDelete(numOfRecords int) int64 {
@ -526,3 +540,7 @@ func (s *Segment) segmentLoadFieldData(fieldID int64, rowCount int, data interfa
return nil
}
func (s *Segment) setLoadBinLogEnable(enable bool) {
s.enableLoadBinLog = enable
}

View File

@ -78,10 +78,11 @@ func TestSegment_getRowCount(t *testing.T) {
records = append(records, blob)
}
var offset = segment.segmentPreInsert(N)
offset, err := segment.segmentPreInsert(N)
assert.Nil(t, err)
assert.GreaterOrEqual(t, offset, int64(0))
err := segment.segmentInsert(offset, &ids, &timestamps, &records)
err = segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
rowCount := segment.getRowCount()
@ -125,10 +126,11 @@ func TestSegment_getDeletedCount(t *testing.T) {
records = append(records, blob)
}
var offsetInsert = segment.segmentPreInsert(N)
offsetInsert, err := segment.segmentPreInsert(N)
assert.Nil(t, err)
assert.GreaterOrEqual(t, offsetInsert, int64(0))
var err = segment.segmentInsert(offsetInsert, &ids, &timestamps, &records)
err = segment.segmentInsert(offsetInsert, &ids, &timestamps, &records)
assert.NoError(t, err)
var offsetDelete = segment.segmentPreDelete(10)
@ -178,10 +180,11 @@ func TestSegment_getMemSize(t *testing.T) {
records = append(records, blob)
}
var offset = segment.segmentPreInsert(N)
offset, err := segment.segmentPreInsert(N)
assert.Nil(t, err)
assert.GreaterOrEqual(t, offset, int64(0))
err := segment.segmentInsert(offset, &ids, &timestamps, &records)
err = segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
var memSize = segment.getMemSize()
@ -225,10 +228,11 @@ func TestSegment_segmentInsert(t *testing.T) {
records = append(records, blob)
}
var offset = segment.segmentPreInsert(N)
offset, err := segment.segmentPreInsert(N)
assert.Nil(t, err)
assert.GreaterOrEqual(t, offset, int64(0))
err := segment.segmentInsert(offset, &ids, &timestamps, &records)
err = segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
deleteSegment(segment)
deleteCollection(collection)
@ -268,10 +272,11 @@ func TestSegment_segmentDelete(t *testing.T) {
records = append(records, blob)
}
var offsetInsert = segment.segmentPreInsert(N)
offsetInsert, err := segment.segmentPreInsert(N)
assert.Nil(t, err)
assert.GreaterOrEqual(t, offsetInsert, int64(0))
var err = segment.segmentInsert(offsetInsert, &ids, &timestamps, &records)
err = segment.segmentInsert(offsetInsert, &ids, &timestamps, &records)
assert.NoError(t, err)
var offsetDelete = segment.segmentPreDelete(10)
@ -317,10 +322,11 @@ func TestSegment_segmentSearch(t *testing.T) {
records = append(records, blob)
}
var offset = segment.segmentPreInsert(N)
offset, err := segment.segmentPreInsert(N)
assert.Nil(t, err)
assert.GreaterOrEqual(t, offset, int64(0))
err := segment.segmentInsert(offset, &ids, &timestamps, &records)
err = segment.segmentInsert(offset, &ids, &timestamps, &records)
assert.NoError(t, err)
dslString := "{\"bool\": { \n\"vector\": {\n \"vec\": {\n \"metric_type\": \"L2\", \n \"params\": {\n \"nprobe\": 10 \n},\n \"query\": \"$0\",\"topk\": 10 \n } \n } \n } \n }"
@ -428,7 +434,8 @@ func TestSegment_segmentPreInsert(t *testing.T) {
records = append(records, blob)
}
var offset = segment.segmentPreInsert(N)
offset, err := segment.segmentPreInsert(N)
assert.Nil(t, err)
assert.GreaterOrEqual(t, offset, int64(0))
deleteSegment(segment)
@ -469,10 +476,11 @@ func TestSegment_segmentPreDelete(t *testing.T) {
records = append(records, blob)
}
var offsetInsert = segment.segmentPreInsert(N)
offsetInsert, err := segment.segmentPreInsert(N)
assert.Nil(t, err)
assert.GreaterOrEqual(t, offsetInsert, int64(0))
var err = segment.segmentInsert(offsetInsert, &ids, &timestamps, &records)
err = segment.segmentInsert(offsetInsert, &ids, &timestamps, &records)
assert.NoError(t, err)
var offsetDelete = segment.segmentPreDelete(10)

View File

@ -145,7 +145,7 @@ class TestSearchBase:
def get_nq(self, request):
yield request.param
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_search_flat(self, connect, collection, get_top_k, get_nq):
'''
target: test basic search function, all the search params is correct, change top-k value
@ -166,7 +166,7 @@ class TestSearchBase:
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_search_flat_top_k(self, connect, collection, get_nq):
'''
target: test basic search function, all the search params is correct, change top-k value
@ -187,6 +187,7 @@ class TestSearchBase:
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.skip("r0.3-test")
def test_search_field(self, connect, collection, get_top_k, get_nq):
'''
target: test basic search function, all the search params is correct, change top-k value
@ -256,7 +257,8 @@ class TestSearchBase:
assert res2[0][0].entity.get("int64") == res[0][1].entity.get("int64")
# TODO:
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
#@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_after_index(self, connect, collection, get_simple_index, get_top_k, get_nq):
'''
@ -285,6 +287,8 @@ class TestSearchBase:
assert res[0]._distances[0] < epsilon
assert check_id_result(res[0], ids[0])
@pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
def test_search_after_index_different_metric_type(self, connect, collection, get_simple_index):
'''
target: test search with different metric_type
@ -304,6 +308,7 @@ class TestSearchBase:
assert len(res[0]) == default_top_k
assert res[0]._distances[0] > res[0]._distances[default_top_k - 1]
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_index_empty_partition(self, connect, collection, get_simple_index, get_top_k, get_nq):
'''
@ -332,11 +337,12 @@ class TestSearchBase:
assert len(res[0]) >= top_k
assert res[0]._distances[0] < epsilon
assert check_id_result(res[0], ids[0])
connect.release(collection)
connect.release_collection(collection)
connect.load_partitions(collection, [default_tag])
res = connect.search(collection, query, partition_tags=[default_tag])
assert len(res[0]) == 0
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_index_partition(self, connect, collection, get_simple_index, get_top_k, get_nq):
'''
@ -367,8 +373,9 @@ class TestSearchBase:
assert res[0]._distances[0] < epsilon
assert check_id_result(res[0], ids[0])
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_index_partition_not_existed(self, connect, collection, get_top_k, get_nq):
def test_search_index_partition_not_existed(self, connect, collection, get_top_k, get_nq, get_simple_index):
'''
target: test basic search function, all the search params is correct, test all index params, and build
method: search with the given vectors and tag (tag name not existed in collection), check the result
@ -388,7 +395,8 @@ class TestSearchBase:
assert len(res) == nq
assert len(res[0]) == 0
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_index_partitions(self, connect, collection, get_simple_index, get_top_k):
'''
@ -424,7 +432,8 @@ class TestSearchBase:
assert res[1]._distances[0] > epsilon
connect.release_collection(collection)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_index_partitions_B(self, connect, collection, get_simple_index, get_top_k):
'''
@ -463,7 +472,8 @@ class TestSearchBase:
#
# test for ip metric
#
@pytest.mark.tag("0331")
# @pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_ip_flat(self, connect, collection, get_simple_index, get_top_k, get_nq):
'''
@ -481,7 +491,8 @@ class TestSearchBase:
assert res[0]._distances[0] >= 1 - gen_inaccuracy(res[0]._distances[0])
assert check_id_result(res[0], ids[0])
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_ip_after_index(self, connect, collection, get_simple_index, get_top_k, get_nq):
'''
@ -507,6 +518,8 @@ class TestSearchBase:
assert check_id_result(res[0], ids[0])
assert res[0]._distances[0] >= 1 - gen_inaccuracy(res[0]._distances[0])
# @pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_ip_index_empty_partition(self, connect, collection, get_simple_index, get_top_k, get_nq):
'''
@ -540,7 +553,8 @@ class TestSearchBase:
res = connect.search(collection, query, partition_tags=[default_tag])
assert len(res[0]) == 0
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_ip_index_partitions(self, connect, collection, get_simple_index, get_top_k):
'''
@ -596,7 +610,7 @@ class TestSearchBase:
with pytest.raises(Exception) as e:
res = connect.search(collection_name, default_query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_search_distance_l2(self, connect, collection):
'''
target: search collection, and check the result: distance
@ -617,7 +631,8 @@ class TestSearchBase:
assert abs(np.sqrt(res[0]._distances[0]) - min(distance_0, distance_1)) <= gen_inaccuracy(res[0]._distances[0])
# TODO
@pytest.mark.tag("0331")
# @pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
def test_search_distance_l2_after_index(self, connect, id_collection, get_simple_index):
'''
target: search collection, and check the result: distance
@ -648,7 +663,8 @@ class TestSearchBase:
# TODO:
# assert abs(np.sqrt(res[0]._distances[0]) - min_distance) <= tmp_epsilon
@pytest.mark.tag("0331")
# @pytest.mark.tags("0331")
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_search_distance_ip(self, connect, collection):
'''
@ -671,7 +687,8 @@ class TestSearchBase:
res = connect.search(collection, query)
assert abs(res[0]._distances[0] - max(distance_0, distance_1)) <= epsilon
@pytest.mark.tag("0331")
# @pytest.mark.tags("0331")
# @pytest.mark.skip("r0.3-test")
def test_search_distance_ip_after_index(self, connect, id_collection, get_simple_index):
'''
target: search collection, and check the result: distance
@ -705,7 +722,7 @@ class TestSearchBase:
# TODO:
# assert abs(res[0]._distances[0] - max_distance) <= tmp_epsilon
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_search_distance_jaccard_flat_index(self, connect, binary_collection):
'''
target: search binary_collection, and check the result: distance
@ -722,7 +739,7 @@ class TestSearchBase:
res = connect.search(binary_collection, query)
assert abs(res[0]._distances[0] - min(distance_0, distance_1)) <= epsilon
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_binary_flat_with_L2(self, connect, binary_collection):
'''
@ -737,7 +754,7 @@ class TestSearchBase:
with pytest.raises(Exception) as e:
connect.search(binary_collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_distance_hamming_flat_index(self, connect, binary_collection):
'''
@ -756,7 +773,7 @@ class TestSearchBase:
assert abs(res[0][0].distance - min(distance_0, distance_1).astype(float)) <= epsilon
# TODO
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_distance_substructure_flat_index(self, connect, binary_collection):
'''
@ -775,7 +792,7 @@ class TestSearchBase:
res = connect.search(binary_collection, query)
assert len(res[0]) == 0
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_distance_substructure_flat_index_B(self, connect, binary_collection):
'''
@ -795,7 +812,7 @@ class TestSearchBase:
assert res[1][0].distance <= epsilon
assert res[1][0].id == ids[1]
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_distance_superstructure_flat_index(self, connect, binary_collection):
'''
@ -814,7 +831,7 @@ class TestSearchBase:
res = connect.search(binary_collection, query)
assert len(res[0]) == 0
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_distance_superstructure_flat_index_B(self, connect, binary_collection):
'''
@ -836,7 +853,7 @@ class TestSearchBase:
assert res[1][0].id in ids
assert res[1][0].distance <= epsilon
@pytest.mark.tag("0331")
# @pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_distance_tanimoto_flat_index(self, connect, binary_collection):
'''
@ -854,7 +871,7 @@ class TestSearchBase:
res = connect.search(binary_collection, query)
assert abs(res[0][0].distance - min(distance_0, distance_1)) <= epsilon
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
@pytest.mark.timeout(30)
def test_search_concurrent_multithreads(self, connect, args):
@ -890,7 +907,7 @@ class TestSearchBase:
for t in threads:
t.join()
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
@pytest.mark.timeout(30)
def test_search_concurrent_multithreads_single_connection(self, connect, args):
@ -925,7 +942,7 @@ class TestSearchBase:
for t in threads:
t.join()
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_multi_collections(self, connect, args):
'''
@ -950,6 +967,7 @@ class TestSearchBase:
assert res[i]._distances[0] < epsilon
assert res[i]._distances[1] > epsilon
@pytest.mark.skip("r0.3-test")
def test_query_entities_with_field_less_than_top_k(self, connect, id_collection):
"""
target: test search with field, and let return entities less than topk
@ -979,7 +997,7 @@ class TestSearchDSL(object):
******************************************************************
"""
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_no_must(self, connect, collection):
'''
method: build query without must expr
@ -990,7 +1008,7 @@ class TestSearchDSL(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_no_vector_term_only(self, connect, collection):
'''
method: build query without vector only term
@ -1004,7 +1022,7 @@ class TestSearchDSL(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_no_vector_range_only(self, connect, collection):
'''
method: build query without vector only range
@ -1018,7 +1036,7 @@ class TestSearchDSL(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_vector_only(self, connect, collection):
entities, ids = init_data(connect, collection)
connect.load_collection(collection)
@ -1040,7 +1058,7 @@ class TestSearchDSL(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_empty(self, connect, collection):
'''
method: search with empty query
@ -1056,6 +1074,7 @@ class TestSearchDSL(object):
******************************************************************
"""
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_term_value_not_in(self, connect, collection):
'''
@ -1073,6 +1092,7 @@ class TestSearchDSL(object):
# TODO:
# TODO:
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_term_value_all_in(self, connect, collection):
'''
@ -1089,6 +1109,7 @@ class TestSearchDSL(object):
# TODO:
# TODO:
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_term_values_not_in(self, connect, collection):
'''
@ -1105,6 +1126,7 @@ class TestSearchDSL(object):
assert len(res[0]) == 0
# TODO:
@pytest.mark.skip("r0.3-test")
def test_query_term_values_all_in(self, connect, collection):
'''
method: build query with vector and term expr, with all term can be filtered
@ -1124,6 +1146,7 @@ class TestSearchDSL(object):
assert result.id in ids[:limit]
# TODO:
@pytest.mark.skip("r0.3-test")
def test_query_term_values_parts_in(self, connect, collection):
'''
method: build query with vector and term expr, with parts of term can be filtered
@ -1141,6 +1164,7 @@ class TestSearchDSL(object):
# TODO:
# TODO:
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_term_values_repeat(self, connect, collection):
'''
@ -1158,7 +1182,7 @@ class TestSearchDSL(object):
assert len(res[0]) == 1
# TODO:
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_term_value_empty(self, connect, collection):
'''
method: build query with term value empty
@ -1171,7 +1195,7 @@ class TestSearchDSL(object):
assert len(res) == nq
assert len(res[0]) == 0
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_complex_dsl(self, connect, collection):
'''
method: query with complicated dsl
@ -1195,7 +1219,7 @@ class TestSearchDSL(object):
"""
# TODO
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_query_term_key_error(self, connect, collection):
'''
@ -1215,7 +1239,7 @@ class TestSearchDSL(object):
def get_invalid_term(self, request):
return request.param
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_query_term_wrong_format(self, connect, collection, get_invalid_term):
'''
@ -1230,6 +1254,7 @@ class TestSearchDSL(object):
res = connect.search(collection, query)
# TODO
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_term_field_named_term(self, connect, collection):
'''
@ -1257,7 +1282,7 @@ class TestSearchDSL(object):
assert len(res[0]) == default_top_k
connect.drop_collection(collection_term)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_query_term_one_field_not_existed(self, connect, collection):
'''
@ -1279,7 +1304,7 @@ class TestSearchDSL(object):
"""
# TODO
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_range_key_error(self, connect, collection):
'''
method: build query with range key error
@ -1299,7 +1324,7 @@ class TestSearchDSL(object):
return request.param
# TODO
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_query_range_wrong_format(self, connect, collection, get_invalid_range):
'''
@ -1313,7 +1338,7 @@ class TestSearchDSL(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_query_range_string_ranges(self, connect, collection):
'''
@ -1328,6 +1353,7 @@ class TestSearchDSL(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_range_invalid_ranges(self, connect, collection):
'''
@ -1350,6 +1376,7 @@ class TestSearchDSL(object):
def get_valid_ranges(self, request):
return request.param
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_range_valid_ranges(self, connect, collection, get_valid_ranges):
'''
@ -1366,7 +1393,7 @@ class TestSearchDSL(object):
assert len(res) == nq
assert len(res[0]) == default_top_k
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_range_one_field_not_existed(self, connect, collection):
'''
method: build query with two fields ranges, one of fields not existed
@ -1387,6 +1414,7 @@ class TestSearchDSL(object):
"""
# TODO
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_multi_term_has_common(self, connect, collection):
'''
@ -1404,6 +1432,7 @@ class TestSearchDSL(object):
assert len(res[0]) == default_top_k
# TODO
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_multi_term_no_common(self, connect, collection):
'''
@ -1421,6 +1450,7 @@ class TestSearchDSL(object):
assert len(res[0]) == 0
# TODO
@pytest.mark.skip("r0.3-test")
def test_query_multi_term_different_fields(self, connect, collection):
'''
method: build query with multi range with same field, and ranges no common
@ -1438,7 +1468,7 @@ class TestSearchDSL(object):
assert len(res[0]) == 0
# TODO
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_query_single_term_multi_fields(self, connect, collection):
'''
@ -1455,6 +1485,7 @@ class TestSearchDSL(object):
res = connect.search(collection, query)
# TODO
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_multi_range_has_common(self, connect, collection):
'''
@ -1472,6 +1503,7 @@ class TestSearchDSL(object):
assert len(res[0]) == default_top_k
# TODO
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_multi_range_no_common(self, connect, collection):
'''
@ -1489,6 +1521,7 @@ class TestSearchDSL(object):
assert len(res[0]) == 0
# TODO
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_multi_range_different_fields(self, connect, collection):
'''
@ -1506,7 +1539,7 @@ class TestSearchDSL(object):
assert len(res[0]) == 0
# TODO
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_query_single_range_multi_fields(self, connect, collection):
'''
@ -1529,6 +1562,7 @@ class TestSearchDSL(object):
"""
# TODO
@pytest.mark.skip("r0.3-test")
@pytest.mark.level(2)
def test_query_single_term_range_has_common(self, connect, collection):
'''
@ -1546,6 +1580,7 @@ class TestSearchDSL(object):
assert len(res[0]) == default_top_k
# TODO
@pytest.mark.skip("r0.3-test")
def test_query_single_term_range_no_common(self, connect, collection):
'''
method: build query with single term single range
@ -1568,7 +1603,7 @@ class TestSearchDSL(object):
"""
# TODO
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_multi_vectors_same_field(self, connect, collection):
'''
method: build query with two vectors same field
@ -1592,7 +1627,7 @@ class TestSearchDSLBools(object):
******************************************************************
"""
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_query_no_bool(self, connect, collection):
'''
@ -1605,7 +1640,7 @@ class TestSearchDSLBools(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_should_only_term(self, connect, collection):
'''
method: build query without must, with should.term instead
@ -1616,7 +1651,7 @@ class TestSearchDSLBools(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_should_only_vector(self, connect, collection):
'''
method: build query without must, with should.vector instead
@ -1627,7 +1662,7 @@ class TestSearchDSLBools(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_must_not_only_term(self, connect, collection):
'''
method: build query without must, with must_not.term instead
@ -1638,7 +1673,7 @@ class TestSearchDSLBools(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_must_not_vector(self, connect, collection):
'''
method: build query without must, with must_not.vector instead
@ -1649,7 +1684,7 @@ class TestSearchDSLBools(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
def test_query_must_should(self, connect, collection):
'''
method: build query must, and with should.term
@ -1668,7 +1703,7 @@ class TestSearchDSLBools(object):
******************************************************************
"""
@pytest.mark.skip("r0.3-test")
class TestSearchInvalid(object):
"""
Test search collection with invalid collection names
@ -1705,14 +1740,14 @@ class TestSearchInvalid(object):
# pytest.skip("sq8h not support in CPU mode")
return request.param
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_with_invalid_collection(self, connect, get_collection_name):
collection_name = get_collection_name
with pytest.raises(Exception) as e:
res = connect.search(collection_name, default_query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_with_invalid_partition(self, connect, collection, get_invalid_partition):
# tag = " "
@ -1720,14 +1755,14 @@ class TestSearchInvalid(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, default_query, partition_tags=tag)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_with_invalid_field_name(self, connect, collection, get_invalid_field):
fields = [get_invalid_field]
with pytest.raises(Exception) as e:
res = connect.search(collection, default_query, fields=fields)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(1)
def test_search_with_not_existed_field(self, connect, collection):
fields = [gen_unique_str("field_name")]
@ -1745,7 +1780,7 @@ class TestSearchInvalid(object):
def get_top_k(self, request):
yield request.param
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(1)
def test_search_with_invalid_top_k(self, connect, collection, get_top_k):
'''
@ -1769,7 +1804,7 @@ class TestSearchInvalid(object):
def get_search_params(self, request):
yield request.param
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_with_invalid_params(self, connect, collection, get_simple_index, get_search_params):
'''
@ -1790,7 +1825,7 @@ class TestSearchInvalid(object):
with pytest.raises(Exception) as e:
res = connect.search(collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_with_invalid_params_binary(self, connect, binary_collection):
'''
@ -1809,7 +1844,7 @@ class TestSearchInvalid(object):
with pytest.raises(Exception) as e:
res = connect.search(binary_collection, query)
@pytest.mark.tag("0331")
@pytest.mark.tags("0331")
@pytest.mark.level(2)
def test_search_with_empty_params(self, connect, collection, args, get_simple_index):
'''