feat: Update segcore for VECTOR_INT8 (#39415)

Issue: #38666

Signed-off-by: Cai Yudong <yudong.cai@zilliz.com>
pull/39489/head
Cai Yudong 2025-01-21 11:03:03 +08:00 committed by GitHub
parent 905c3b81dd
commit 341d6c1eb7
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
29 changed files with 332 additions and 50 deletions

View File

@ -356,13 +356,14 @@ create_chunk(const FieldMeta& field_meta,
}
case milvus::DataType::VECTOR_FLOAT: {
w = std::make_shared<
ChunkWriter<arrow::FixedSizeBinaryArray, float>>(dim, nullable);
ChunkWriter<arrow::FixedSizeBinaryArray, knowhere::fp32>>(
dim, nullable);
break;
}
case milvus::DataType::VECTOR_BINARY: {
w = std::make_shared<
ChunkWriter<arrow::FixedSizeBinaryArray, uint8_t>>(dim / 8,
nullable);
ChunkWriter<arrow::FixedSizeBinaryArray, knowhere::bin1>>(
dim / 8, nullable);
break;
}
case milvus::DataType::VECTOR_FLOAT16: {
@ -377,6 +378,12 @@ create_chunk(const FieldMeta& field_meta,
dim, nullable);
break;
}
case milvus::DataType::VECTOR_INT8: {
w = std::make_shared<
ChunkWriter<arrow::FixedSizeBinaryArray, knowhere::int8>>(
dim, nullable);
break;
}
case milvus::DataType::VARCHAR:
case milvus::DataType::STRING: {
w = std::make_shared<StringChunkWriter>(nullable);
@ -450,13 +457,13 @@ create_chunk(const FieldMeta& field_meta,
}
case milvus::DataType::VECTOR_FLOAT: {
w = std::make_shared<
ChunkWriter<arrow::FixedSizeBinaryArray, float>>(
ChunkWriter<arrow::FixedSizeBinaryArray, knowhere::fp32>>(
dim, file, file_offset, nullable);
break;
}
case milvus::DataType::VECTOR_BINARY: {
w = std::make_shared<
ChunkWriter<arrow::FixedSizeBinaryArray, uint8_t>>(
ChunkWriter<arrow::FixedSizeBinaryArray, knowhere::bin1>>(
dim / 8, file, file_offset, nullable);
break;
}
@ -472,6 +479,12 @@ create_chunk(const FieldMeta& field_meta,
dim, file, file_offset, nullable);
break;
}
case milvus::DataType::VECTOR_INT8: {
w = std::make_shared<
ChunkWriter<arrow::FixedSizeBinaryArray, knowhere::int8>>(
dim, file, file_offset, nullable);
break;
}
case milvus::DataType::VARCHAR:
case milvus::DataType::STRING: {
w = std::make_shared<StringChunkWriter>(

View File

@ -238,6 +238,7 @@ FieldDataImpl<Type, is_type_entire_row>::FillFieldData(
case DataType::VECTOR_FLOAT:
case DataType::VECTOR_FLOAT16:
case DataType::VECTOR_BFLOAT16:
case DataType::VECTOR_INT8:
case DataType::VECTOR_BINARY: {
auto array_info =
GetDataInfoFromArray<arrow::FixedSizeBinaryArray,

View File

@ -141,6 +141,17 @@ class FieldData<SparseFloatVector> : public FieldDataSparseVectorImpl {
}
};
template <>
class FieldData<Int8Vector> : public FieldDataImpl<int8, false> {
public:
explicit FieldData(int64_t dim,
DataType data_type,
int64_t buffered_num_rows = 0)
: FieldDataImpl<int8, false>::FieldDataImpl(
dim, data_type, false, buffered_num_rows) {
}
};
using FieldDataPtr = std::shared_ptr<FieldDataBase>;
using FieldDataChannel = Channel<FieldDataPtr>;
using FieldDataChannelPtr = std::shared_ptr<FieldDataChannel>;

View File

@ -125,12 +125,12 @@ GetDataTypeSize(DataType data_type, int dim = 1) {
AssertInfo(dim % 8 == 0, "dim={}", dim);
return dim / 8;
}
case DataType::VECTOR_FLOAT16: {
case DataType::VECTOR_FLOAT16:
return sizeof(float16) * dim;
}
case DataType::VECTOR_BFLOAT16: {
case DataType::VECTOR_BFLOAT16:
return sizeof(bfloat16) * dim;
}
case DataType::VECTOR_INT8:
return sizeof(int8) * dim;
// Not supporting variable length types(such as VECTOR_SPARSE_FLOAT and
// VARCHAR) here intentionally. We can't easily estimate the size of
// them. Caller of this method must handle this case themselves and must
@ -192,6 +192,8 @@ GetDataTypeName(DataType data_type) {
return "vector_bfloat16";
case DataType::VECTOR_SPARSE_FLOAT:
return "vector_sparse_float";
case DataType::VECTOR_INT8:
return "vector_int8";
default:
PanicInfo(DataTypeInvalid, "Unsupported DataType({})", data_type);
}
@ -325,7 +327,7 @@ IsSparseFloatVectorDataType(DataType data_type) {
}
inline bool
IsInt8VectorDataType(DataType data_type) {
IsIntVectorDataType(DataType data_type) {
return data_type == DataType::VECTOR_INT8;
}
@ -338,7 +340,7 @@ IsFloatVectorDataType(DataType data_type) {
inline bool
IsVectorDataType(DataType data_type) {
return IsBinaryVectorDataType(data_type) ||
IsFloatVectorDataType(data_type) || IsInt8VectorDataType(data_type);
IsFloatVectorDataType(data_type) || IsIntVectorDataType(data_type);
}
inline bool
@ -642,6 +644,9 @@ struct fmt::formatter<milvus::DataType> : formatter<string_view> {
case milvus::DataType::VECTOR_SPARSE_FLOAT:
name = "VECTOR_SPARSE_FLOAT";
break;
case milvus::DataType::VECTOR_INT8:
name = "VECTOR_INT8";
break;
}
return formatter<string_view>::format(name, ctx);
}

View File

@ -156,14 +156,6 @@ IsMetricType(const std::string_view str,
return !strcasecmp(str.data(), metric_type.c_str());
}
inline bool
IsFloatMetricType(const knowhere::MetricType& metric_type) {
return IsMetricType(metric_type, knowhere::metric::L2) ||
IsMetricType(metric_type, knowhere::metric::IP) ||
IsMetricType(metric_type, knowhere::metric::COSINE) ||
IsMetricType(metric_type, knowhere::metric::BM25);
}
inline bool
PositivelyRelated(const knowhere::MetricType& metric_type) {
return IsMetricType(metric_type, knowhere::metric::IP) ||

View File

@ -30,25 +30,30 @@ namespace milvus {
#define GET_ELEM_TYPE_FOR_VECTOR_TRAIT \
using elem_type = std::conditional_t< \
std::is_same_v<TraitType, milvus::BinaryVector>, \
BinaryVector::embedded_type, \
std::is_same_v<TraitType, milvus::FloatVector>, \
milvus::FloatVector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::Float16Vector>, \
Float16Vector::embedded_type, \
milvus::Float16Vector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::BFloat16Vector>, \
BFloat16Vector::embedded_type, \
FloatVector::embedded_type>>>;
milvus::BFloat16Vector::embedded_type, \
std::conditional_t< \
std::is_same_v<TraitType, milvus::Int8Vector>, \
milvus::Int8Vector::embedded_type, \
milvus::BinaryVector::embedded_type>>>>;
#define GET_SCHEMA_DATA_TYPE_FOR_VECTOR_TRAIT \
auto schema_data_type = \
std::is_same_v<TraitType, milvus::FloatVector> \
? FloatVector::schema_data_type \
? milvus::FloatVector::schema_data_type \
: std::is_same_v<TraitType, milvus::Float16Vector> \
? Float16Vector::schema_data_type \
? milvus::Float16Vector::schema_data_type \
: std::is_same_v<TraitType, milvus::BFloat16Vector> \
? BFloat16Vector::schema_data_type \
: BinaryVector::schema_data_type;
? milvus::BFloat16Vector::schema_data_type \
: std::is_same_v<TraitType, milvus::Int8Vector> \
? milvus::Int8Vector::schema_data_type \
: milvus::BinaryVector::schema_data_type;
class VectorTrait {};
@ -118,6 +123,19 @@ class SparseFloatVector : public VectorTrait {
proto::common::PlaceholderType::SparseFloatVector;
};
class Int8Vector : public VectorTrait {
public:
using embedded_type = int8;
static constexpr int32_t dim_factor = 1;
static constexpr auto data_type = DataType::VECTOR_INT8;
static constexpr auto c_data_type = CDataType::Int8Vector;
static constexpr auto schema_data_type =
proto::schema::DataType::Int8Vector;
static constexpr auto vector_type = proto::plan::VectorType::Int8Vector;
static constexpr auto placeholder_type =
proto::common::PlaceholderType::Int8Vector;
};
template <typename T>
constexpr bool IsVector = std::is_base_of_v<VectorTrait, T>;

View File

@ -55,6 +55,7 @@ enum CDataType {
Float16Vector = 102,
BFloat16Vector = 103,
SparseFloatVector = 104,
Int8Vector = 105,
};
typedef enum CDataType CDataType;

View File

@ -179,6 +179,16 @@ IndexFactory::VecIndexLoadResource(
knowhere::IndexStaticFaced<knowhere::fp32>::HasRawData(
index_type, index_version, config);
break;
case milvus::DataType::VECTOR_INT8:
resource = knowhere::IndexStaticFaced<
knowhere::int8>::EstimateLoadResource(index_type,
index_version,
index_size_gb,
config);
has_raw_data =
knowhere::IndexStaticFaced<knowhere::int8>::HasRawData(
index_type, index_version, config);
break;
default:
LOG_ERROR("invalid data type to estimate index load resource: {}",
field_type);
@ -426,6 +436,9 @@ IndexFactory::CreateVectorIndex(
return std::make_unique<VectorDiskAnnIndex<float>>(
index_type, metric_type, version, file_manager_context);
}
case DataType::VECTOR_INT8: {
// TODO caiyd, not support yet
}
default:
PanicInfo(
DataTypeInvalid,

View File

@ -222,4 +222,9 @@ ExecPlanNodeVisitor::visit(SparseFloatVectorANNS& node) {
VectorVisitorImpl<SparseFloatVector>(node);
}
void
ExecPlanNodeVisitor::visit(Int8VectorANNS& node) {
VectorVisitorImpl<Int8Vector>(node);
}
} // namespace milvus::query

View File

@ -37,6 +37,9 @@ class ExecPlanNodeVisitor : public PlanNodeVisitor {
void
visit(SparseFloatVectorANNS& node) override;
void
visit(Int8VectorANNS& node) override;
void
visit(RetrievePlanNode& node) override;

View File

@ -40,6 +40,11 @@ SparseFloatVectorANNS::accept(PlanNodeVisitor& visitor) {
visitor.visit(*this);
}
void
Int8VectorANNS::accept(PlanNodeVisitor& visitor) {
visitor.visit(*this);
}
void
RetrievePlanNode::accept(PlanNodeVisitor& visitor) {
visitor.visit(*this);

View File

@ -71,6 +71,12 @@ struct SparseFloatVectorANNS : VectorPlanNode {
accept(PlanNodeVisitor&) override;
};
struct Int8VectorANNS : VectorPlanNode {
public:
void
accept(PlanNodeVisitor&) override;
};
struct RetrievePlanNode : PlanNode {
public:
void

View File

@ -34,6 +34,9 @@ class PlanNodeVisitor {
virtual void
visit(SparseFloatVectorANNS&) = 0;
virtual void
visit(Int8VectorANNS&) = 0;
virtual void
visit(RetrievePlanNode&) = 0;
};

View File

@ -123,6 +123,9 @@ ProtoParser::PlanNodeFromProto(const planpb::PlanNode& plan_node_proto) {
} else if (anns_proto.vector_type() ==
milvus::proto::plan::VectorType::SparseFloatVector) {
return std::make_unique<SparseFloatVectorANNS>();
} else if (anns_proto.vector_type() ==
milvus::proto::plan::VectorType::Int8Vector) {
return std::make_unique<Int8VectorANNS>();
} else {
return std::make_unique<FloatVectorANNS>();
}

View File

@ -35,10 +35,19 @@ CheckBruteForceSearchParam(const FieldMeta& field,
AssertInfo(IsVectorDataType(data_type),
"[BruteForceSearch] Data type isn't vector type");
bool is_float_vec_data_type = IsFloatVectorDataType(data_type);
bool is_float_metric_type = IsFloatMetricType(metric_type);
AssertInfo(is_float_vec_data_type == is_float_metric_type,
"[BruteForceSearch] Data type and metric type miss-match");
if (IsBinaryVectorDataType(data_type)) {
AssertInfo(IsBinaryVectorMetricType(metric_type),
"[BruteForceSearch] Binary vector, data type and metric type miss-match");
} else if (IsFloatVectorDataType(data_type)) {
AssertInfo(IsFloatVectorMetricType(metric_type),
"[BruteForceSearch] Float vector, data type and metric type miss-match");
} else if (IsIntVectorDataType(data_type)) {
AssertInfo(IsIntVectorMetricType(metric_type),
"[BruteForceSearch] Int vector, data type and metric type miss-match");
} else {
AssertInfo(IsVectorDataType(data_type),
"[BruteForceSearch] Unsupported vector data type");
}
}
knowhere::Json
@ -94,6 +103,12 @@ PrepareBFDataSet(const dataset::SearchDataset& query_ds,
knowhere::ConvertFromDataTypeIfNeeded<float16>(base_dataset);
query_dataset =
knowhere::ConvertFromDataTypeIfNeeded<float16>(query_dataset);
} else if (data_type == DataType::VECTOR_INT8) {
// TODO caiyd: if knowhere support real int8 bf, remove this
base_dataset =
knowhere::ConvertFromDataTypeIfNeeded<int8>(base_dataset);
query_dataset =
knowhere::ConvertFromDataTypeIfNeeded<int8>(query_dataset);
}
base_dataset->SetTensorBeginId(raw_ds.begin_id);
return std::make_pair(query_dataset, base_dataset);
@ -147,6 +162,10 @@ BruteForceSearch(const dataset::SearchDataset& query_ds,
res = knowhere::BruteForce::RangeSearch<
knowhere::sparse::SparseRow<float>>(
base_dataset, query_dataset, search_cfg, bitset);
} else if (data_type == DataType::VECTOR_INT8) {
// TODO caiyd: if knowhere support real int8 bf, change it
res = knowhere::BruteForce::RangeSearch<float>(
base_dataset, query_dataset, search_cfg, bitset);
} else {
PanicInfo(
ErrorCode::Unsupported,
@ -211,6 +230,15 @@ BruteForceSearch(const dataset::SearchDataset& query_ds,
sub_result.mutable_distances().data(),
search_cfg,
bitset);
} else if (data_type == DataType::VECTOR_INT8) {
// TODO caiyd: if knowhere support real int8 bf, change it
stat = knowhere::BruteForce::SearchWithBuf<float>(
base_dataset,
query_dataset,
sub_result.mutable_seg_offsets().data(),
sub_result.mutable_distances().data(),
search_cfg,
bitset);
} else {
PanicInfo(ErrorCode::Unsupported,
"Unsupported dataType for chunk brute force search:{}",
@ -236,22 +264,22 @@ DispatchBruteForceIteratorByDataType(const knowhere::DataSetPtr& base_dataset,
case DataType::VECTOR_FLOAT:
return knowhere::BruteForce::AnnIterator<float>(
base_dataset, query_dataset, config, bitset);
break;
case DataType::VECTOR_FLOAT16:
//todo: if knowhere support real fp16/bf16 bf, change it
return knowhere::BruteForce::AnnIterator<float>(
base_dataset, query_dataset, config, bitset);
break;
case DataType::VECTOR_BFLOAT16:
//todo: if knowhere support real fp16/bf16 bf, change it
return knowhere::BruteForce::AnnIterator<float>(
base_dataset, query_dataset, config, bitset);
break;
case DataType::VECTOR_SPARSE_FLOAT:
return knowhere::BruteForce::AnnIterator<
knowhere::sparse::SparseRow<float>>(
base_dataset, query_dataset, config, bitset);
break;
case DataType::VECTOR_INT8:
// TODO caiyd: if knowhere support real int8 bf, change it
return knowhere::BruteForce::AnnIterator<float>(
base_dataset, query_dataset, config, bitset);
default:
PanicInfo(ErrorCode::Unsupported,
"Unsupported dataType for chunk brute force iterator:{}",

View File

@ -1729,7 +1729,15 @@ ChunkedSegmentSealedImpl::get_raw_data(FieldId field_id,
ret->mutable_vectors()->set_dim(dst->dim());
break;
}
case DataType::VECTOR_INT8: {
bulk_subscript_impl(
field_meta.get_sizeof(),
column.get(),
seg_offsets,
count,
ret->mutable_vectors()->mutable_int8_vector()->data());
break;
}
default: {
PanicInfo(DataTypeInvalid,
fmt::format("unsupported data type {}",

View File

@ -44,6 +44,9 @@ VectorBase::set_data_raw(ssize_t element_offset,
data->vectors().sparse_float_vector().contents())
.get(),
element_count);
} else if (field_meta.get_data_type() == DataType::VECTOR_INT8) {
return set_data_raw(
element_offset, VEC_FIELD_DATA(data, int8), element_count);
} else {
PanicInfo(DataTypeInvalid, "unsupported vector type");
}

View File

@ -167,9 +167,13 @@ class ConcurrentVectorImpl : public VectorBase {
std::conditional_t<
std::is_same_v<Type, float16>,
Float16Vector,
std::conditional_t<std::is_same_v<Type, bfloat16>,
BFloat16Vector,
BinaryVector>>>>;
std::conditional_t<
std::is_same_v<Type, bfloat16>,
BFloat16Vector,
std::conditional_t<
std::is_same_v<Type, int8>,
Int8Vector,
BinaryVector>>>>>;
public:
explicit ConcurrentVectorImpl(
@ -541,4 +545,16 @@ class ConcurrentVector<BFloat16Vector>
}
};
template <>
class ConcurrentVector<Int8Vector>
: public ConcurrentVectorImpl<int8, false> {
public:
ConcurrentVector(int64_t dim,
int64_t size_per_chunk,
storage::MmapChunkDescriptorPtr mmap_descriptor = nullptr)
: ConcurrentVectorImpl<int8, false>::ConcurrentVectorImpl(
dim, size_per_chunk, std::move(mmap_descriptor)) {
}
};
} // namespace milvus::segcore

View File

@ -324,6 +324,7 @@ CreateIndex(const FieldMeta& field_meta,
if (field_meta.get_data_type() == DataType::VECTOR_FLOAT ||
field_meta.get_data_type() == DataType::VECTOR_FLOAT16 ||
field_meta.get_data_type() == DataType::VECTOR_BFLOAT16 ||
field_meta.get_data_type() == DataType::VECTOR_INT8 ||
field_meta.get_data_type() == DataType::VECTOR_SPARSE_FLOAT) {
return std::make_unique<VectorFieldIndexing>(field_meta,
field_index_meta,

View File

@ -412,6 +412,11 @@ struct InsertRecord {
this->append_data<SparseFloatVector>(field_id,
size_per_chunk);
continue;
} else if (field_meta.get_data_type() ==
DataType::VECTOR_INT8) {
this->append_data<Int8Vector>(
field_id, field_meta.get_dim(), size_per_chunk);
continue;
} else {
PanicInfo(DataTypeInvalid,
fmt::format("unsupported vector type",

View File

@ -486,6 +486,14 @@ SegmentGrowingImpl::bulk_subscript(FieldId field_id,
result->mutable_vectors()->mutable_sparse_float_vector());
result->mutable_vectors()->set_dim(
result->vectors().sparse_float_vector().dim());
} else if (field_meta.get_data_type() == DataType::VECTOR_INT8) {
bulk_subscript_impl<Int8Vector>(
field_id,
field_meta.get_sizeof(),
vec_ptr,
seg_offsets,
count,
result->mutable_vectors()->mutable_int8_vector()->data());
} else {
PanicInfo(DataTypeInvalid, "logical error");
}

View File

@ -1549,7 +1549,15 @@ SegmentSealedImpl::get_raw_data(FieldId field_id,
ret->mutable_vectors()->set_dim(dst->dim());
break;
}
case DataType::VECTOR_INT8: {
bulk_subscript_impl(
field_meta.get_sizeof(),
column->Data(0),
seg_offsets,
count,
ret->mutable_vectors()->mutable_int8_vector()->data());
break;
}
default: {
PanicInfo(DataTypeInvalid,
fmt::format("unsupported data type {}",

View File

@ -356,6 +356,12 @@ CreateVectorDataArray(int64_t count, const FieldMeta& field_meta) {
// does nothing here
break;
}
case DataType::VECTOR_INT8: {
auto length = count * dim;
auto obj = vector_array->mutable_int8_vector();
obj->resize(length * sizeof(int8));
break;
}
default: {
PanicInfo(DataTypeInvalid,
fmt::format("unsupported datatype {}", data_type));
@ -519,6 +525,13 @@ CreateVectorDataArrayFrom(const void* data_raw,
vector_array->set_dim(vector_array->sparse_float_vector().dim());
break;
}
case DataType::VECTOR_INT8: {
auto length = count * dim;
auto data = reinterpret_cast<const char*>(data_raw);
auto obj = vector_array->mutable_int8_vector();
obj->assign(data, length * sizeof(int8));
break;
}
default: {
PanicInfo(DataTypeInvalid,
fmt::format("unsupported datatype {}", data_type));
@ -596,6 +609,11 @@ MergeDataArray(std::vector<MergeBase>& merge_bases,
}
vector_array->set_dim(dst->dim());
*dst->mutable_contents() = src.contents();
} else if (field_meta.get_data_type() ==
DataType::VECTOR_INT8) {
auto data = VEC_FIELD_DATA(src_field_data, int8);
auto obj = vector_array->mutable_int8_vector();
obj->assign(data, dim * sizeof(int8));
} else {
PanicInfo(DataTypeInvalid,
fmt::format("unsupported datatype {}", data_type));

View File

@ -196,6 +196,7 @@ AddPayloadToArrowBuilder(std::shared_ptr<arrow::ArrayBuilder> builder,
case DataType::VECTOR_FLOAT16:
case DataType::VECTOR_BFLOAT16:
case DataType::VECTOR_BINARY:
case DataType::VECTOR_INT8:
case DataType::VECTOR_FLOAT: {
add_vector_payload(builder, const_cast<uint8_t*>(raw_data), length);
break;
@ -312,6 +313,11 @@ CreateArrowBuilder(DataType data_type, int dim) {
return std::make_shared<arrow::FixedSizeBinaryBuilder>(
arrow::fixed_size_binary(dim * sizeof(bfloat16)));
}
case DataType::VECTOR_INT8: {
AssertInfo(dim > 0, "invalid dim value");
return std::make_shared<arrow::FixedSizeBinaryBuilder>(
arrow::fixed_size_binary(dim * sizeof(int8)));
}
default: {
PanicInfo(
DataTypeInvalid, "unsupported vector data type {}", data_type);
@ -405,6 +411,13 @@ CreateArrowSchema(DataType data_type, int dim, bool nullable) {
return arrow::schema(
{arrow::field("val", arrow::binary(), nullable)});
}
case DataType::VECTOR_INT8: {
AssertInfo(dim > 0, "invalid dim value");
return arrow::schema(
{arrow::field("val",
arrow::fixed_size_binary(dim * sizeof(int8)),
nullable)});
}
default: {
PanicInfo(
DataTypeInvalid, "unsupported vector data type {}", data_type);
@ -433,6 +446,9 @@ GetDimensionFromFileMetaData(const parquet::ColumnDescriptor* schema,
fmt::format("GetDimensionFromFileMetaData should not be "
"called for sparse vector"));
}
case DataType::VECTOR_INT8: {
return schema->type_length() / sizeof(int8);
}
default:
PanicInfo(DataTypeInvalid, "unsupported data type {}", data_type);
}
@ -478,6 +494,15 @@ GetDimensionFromArrowArray(std::shared_ptr<arrow::Array> data,
std::dynamic_pointer_cast<arrow::FixedSizeBinaryArray>(data);
return array->byte_width() / sizeof(bfloat16);
}
case DataType::VECTOR_INT8: {
AssertInfo(
data->type()->id() == arrow::Type::type::FIXED_SIZE_BINARY,
"inconsistent data type: {}",
data->type_id());
auto array =
std::dynamic_pointer_cast<arrow::FixedSizeBinaryArray>(data);
return array->byte_width() / sizeof(int8);
}
default:
PanicInfo(DataTypeInvalid, "unsupported data type {}", data_type);
}
@ -810,6 +835,9 @@ CreateFieldData(const DataType& type,
case DataType::VECTOR_SPARSE_FLOAT:
return std::make_shared<FieldData<SparseFloatVector>>(
type, total_num_rows);
case DataType::VECTOR_INT8:
return std::make_shared<FieldData<Int8Vector>>(
dim, type, total_num_rows);
default:
PanicInfo(DataTypeInvalid,
"CreateFieldData not support data type " +

View File

@ -43,7 +43,6 @@
#include "test_utils/GenExprProto.h"
#include "expr/ITypeExpr.h"
#include "plan/PlanNode.h"
#include "exec/expression/Expr.h"
#include "segcore/load_index_c.h"
#include "test_utils/c_api_test_utils.h"
#include "segcore/vector_index_c.h"
@ -174,7 +173,7 @@ template <class TraitType>
std::string
generate_collection_schema(std::string metric_type, int dim) {
namespace schema = milvus::proto::schema;
GET_SCHEMA_DATA_TYPE_FOR_VECTOR_TRAIT;
GET_SCHEMA_DATA_TYPE_FOR_VECTOR_TRAIT
schema::CollectionSchema collection_schema;
collection_schema.set_name("collection_test");
@ -326,6 +325,7 @@ TEST(CApiTest, CPlan) {
Test_CPlan<milvus::FloatVector>(knowhere::metric::L2);
Test_CPlan<milvus::Float16Vector>(knowhere::metric::L2);
Test_CPlan<milvus::BFloat16Vector>(knowhere::metric::L2);
Test_CPlan<milvus::Int8Vector>(knowhere::metric::L2);
}
TEST(CApiTest, InsertTest) {
@ -1783,11 +1783,14 @@ TEST(CApiTest, ReduceSearchWithExpr) {
testReduceSearchWithExpr<milvus::FloatVector>(10000, 1, 1);
testReduceSearchWithExpr<milvus::FloatVector>(10000, 10, 10);
// float16
testReduceSearchWithExpr<milvus::Float16Vector>(2, 10, 10, false);
testReduceSearchWithExpr<milvus::Float16Vector>(100, 10, 10, false);
testReduceSearchWithExpr<milvus::Float16Vector>(2, 10, 10);
testReduceSearchWithExpr<milvus::Float16Vector>(100, 10, 10);
// bfloat16
testReduceSearchWithExpr<milvus::BFloat16Vector>(2, 10, 10, false);
testReduceSearchWithExpr<milvus::BFloat16Vector>(100, 10, 10, false);
testReduceSearchWithExpr<milvus::BFloat16Vector>(2, 10, 10);
testReduceSearchWithExpr<milvus::BFloat16Vector>(100, 10, 10);
// int8
testReduceSearchWithExpr<milvus::Int8Vector>(2, 10, 10);
testReduceSearchWithExpr<milvus::Int8Vector>(100, 10, 10);
}
TEST(CApiTest, ReduceSearchWithExprFilterAll) {
@ -1796,8 +1799,13 @@ TEST(CApiTest, ReduceSearchWithExprFilterAll) {
testReduceSearchWithExpr<milvus::FloatVector>(2, 10, 10, true);
// float16
testReduceSearchWithExpr<milvus::Float16Vector>(2, 1, 1, true);
testReduceSearchWithExpr<milvus::Float16Vector>(2, 10, 10, true);
// bfloat16
testReduceSearchWithExpr<milvus::BFloat16Vector>(2, 1, 1, true);
testReduceSearchWithExpr<milvus::BFloat16Vector>(2, 10, 10, true);
// int8
testReduceSearchWithExpr<milvus::Int8Vector>(2, 1, 1, true);
testReduceSearchWithExpr<milvus::Int8Vector>(2, 10, 10, true);
}
TEST(CApiTest, LoadIndexInfo) {
@ -2053,6 +2061,7 @@ TEST(CApiTest, Indexing_Without_Predicate) {
Test_Indexing_Without_Predicate<milvus::FloatVector>();
Test_Indexing_Without_Predicate<milvus::Float16Vector>();
Test_Indexing_Without_Predicate<milvus::BFloat16Vector>();
Test_Indexing_Without_Predicate<milvus::Int8Vector>();
}
TEST(CApiTest, Indexing_Expr_Without_Predicate) {
@ -4373,6 +4382,7 @@ TEST(CApiTest, Range_Search_With_Radius_And_Range_Filter) {
Test_Range_Search_With_Radius_And_Range_Filter<milvus::FloatVector>();
Test_Range_Search_With_Radius_And_Range_Filter<milvus::Float16Vector>();
Test_Range_Search_With_Radius_And_Range_Filter<milvus::BFloat16Vector>();
Test_Range_Search_With_Radius_And_Range_Filter<milvus::Int8Vector>();
}
std::vector<SegOffset>

View File

@ -83,6 +83,9 @@ TestVecIndex() {
} else if (std::is_same_v<TraitType, milvus::BFloat16Vector>) {
auto xb_data = dataset.template get_col<uint8_t>(milvus::FieldId(100));
status = BuildBFloat16VecIndex(index, NB * DIM, xb_data.data());
} else if (std::is_same_v<TraitType, milvus::Int8Vector>) {
auto xb_data = dataset.template get_col<int8_t>(milvus::FieldId(100));
status = BuildInt8VecIndex(index, NB * DIM, xb_data.data());
}
ASSERT_EQ(milvus::Success, status.error_code);
@ -111,6 +114,7 @@ TEST(VecIndex, All) {
TestVecIndex<milvus::FloatVector>();
TestVecIndex<milvus::Float16Vector>();
TestVecIndex<milvus::BFloat16Vector>();
TestVecIndex<milvus::Int8Vector>();
}
TEST(CBoolIndexTest, All) {

View File

@ -126,6 +126,13 @@ struct GeneratedData {
auto src_data = reinterpret_cast<const T*>(
target_field_data.vectors().bfloat16_vector().data());
std::copy_n(src_data, len, ret.data());
} else if (field_meta.get_data_type() ==
DataType::VECTOR_INT8) {
int len = raw_->num_rows() * field_meta.get_dim();
ret.resize(len);
auto src_data = reinterpret_cast<const T*>(
target_field_data.vectors().int8_vector().data());
std::copy_n(src_data, len, ret.data());
} else {
PanicInfo(Unsupported, "unsupported");
}
@ -410,7 +417,6 @@ inline GeneratedData DataGen(SchemaPtr schema,
array.release());
break;
}
case DataType::VECTOR_BFLOAT16: {
auto dim = field_meta.get_dim();
vector<bfloat16> final(dim * N);
@ -420,6 +426,15 @@ inline GeneratedData DataGen(SchemaPtr schema,
insert_cols(final, N, field_meta, random_valid);
break;
}
case DataType::VECTOR_INT8: {
auto dim = field_meta.get_dim();
vector<int8> final(dim * N);
for (auto& x : final) {
x = int8_t(rand() % 256 - 128);
}
insert_cols(final, N, field_meta, random_valid);
break;
}
case DataType::BOOL: {
FixedVector<bool> data(N);
for (int i = 0; i < N; ++i) {
@ -834,6 +849,46 @@ CreateSparseFloatPlaceholderGroup(int64_t num_queries, int64_t seed = 42) {
return raw_group;
}
inline auto
CreateInt8PlaceholderGroup(int64_t num_queries,
int64_t dim,
int64_t seed = 42) {
namespace ser = milvus::proto::common;
ser::PlaceholderGroup raw_group;
auto value = raw_group.add_placeholders();
value->set_tag("$0");
value->set_type(ser::PlaceholderType::Int8Vector);
std::default_random_engine e(seed);
for (int i = 0; i < num_queries; ++i) {
std::vector<int8> vec;
for (int d = 0; d < dim; ++d) {
vec.push_back(e());
}
value->add_values(vec.data(), vec.size() * sizeof(int8));
}
return raw_group;
}
inline auto
CreateInt8PlaceholderGroupFromBlob(int64_t num_queries,
int64_t dim,
const int8* ptr) {
namespace ser = milvus::proto::common;
ser::PlaceholderGroup raw_group;
auto value = raw_group.add_placeholders();
value->set_tag("$0");
value->set_type(ser::PlaceholderType::Int8Vector);
for (int i = 0; i < num_queries; ++i) {
std::vector<int8> vec;
for (int d = 0; d < dim; ++d) {
vec.push_back(*ptr);
++ptr;
}
value->add_values(vec.data(), vec.size() * sizeof(int8));
}
return raw_group;
}
inline auto
SearchResultToVector(const SearchResult& sr) {
int64_t num_queries = sr.total_nq_;
@ -934,6 +989,12 @@ CreateFieldDataFromDataArray(ssize_t raw_count,
createFieldData(rows.get(), DataType::VECTOR_SPARSE_FLOAT, 0);
break;
}
case DataType::VECTOR_INT8: {
auto raw_data = data->vectors().int8_vector().data();
dim = field_meta.get_dim();
createFieldData(raw_data, DataType::VECTOR_INT8, dim);
break;
}
default: {
PanicInfo(Unsupported, "unsupported");
}

View File

@ -38,6 +38,7 @@ enum VectorType {
Float16Vector = 2;
BFloat16Vector = 3;
SparseFloatVector = 4;
Int8Vector = 5;
};
message GenericValue {

View File

@ -175,6 +175,7 @@ const (
VectorType_Float16Vector VectorType = 2
VectorType_BFloat16Vector VectorType = 3
VectorType_SparseFloatVector VectorType = 4
VectorType_Int8Vector VectorType = 5
)
// Enum value maps for VectorType.
@ -185,6 +186,7 @@ var (
2: "Float16Vector",
3: "BFloat16Vector",
4: "SparseFloatVector",
5: "Int8Vector",
}
VectorType_value = map[string]int32{
"BinaryVector": 0,
@ -192,6 +194,7 @@ var (
"Float16Vector": 2,
"BFloat16Vector": 3,
"SparseFloatVector": 4,
"Int8Vector": 5,
}
)
@ -2897,14 +2900,15 @@ var file_plan_proto_rawDesc = []byte{
0x12, 0x07, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x10, 0x01, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x75, 0x62,
0x10, 0x02, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x75, 0x6c, 0x10, 0x03, 0x12, 0x07, 0x0a, 0x03, 0x44,
0x69, 0x76, 0x10, 0x04, 0x12, 0x07, 0x0a, 0x03, 0x4d, 0x6f, 0x64, 0x10, 0x05, 0x12, 0x0f, 0x0a,
0x0b, 0x41, 0x72, 0x72, 0x61, 0x79, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x10, 0x06, 0x2a, 0x6d,
0x0b, 0x41, 0x72, 0x72, 0x61, 0x79, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x10, 0x06, 0x2a, 0x7d,
0x0a, 0x0a, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, 0x0a, 0x0c,
0x42, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x00, 0x12, 0x0f,
0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x01, 0x12,
0x11, 0x0a, 0x0d, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72,
0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x42, 0x46, 0x6c, 0x6f, 0x61, 0x74, 0x31, 0x36, 0x56, 0x65,
0x63, 0x74, 0x6f, 0x72, 0x10, 0x03, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x70, 0x61, 0x72, 0x73, 0x65,
0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x04, 0x42, 0x2e, 0x5a,
0x46, 0x6c, 0x6f, 0x61, 0x74, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x04, 0x12, 0x0e, 0x0a,
0x0a, 0x49, 0x6e, 0x74, 0x38, 0x56, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x10, 0x05, 0x42, 0x2e, 0x5a,
0x2c, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67,
0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x6c, 0x61, 0x6e, 0x70, 0x62, 0x62, 0x06, 0x70,