Merge pull request #60202 from rschu1ze/vector-dot-product

Vectorize `dotProduct`
This commit is contained in:
Robert Schulze 2024-02-22 11:19:31 +01:00 committed by GitHub
commit 2d2acd6bf9
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 444 additions and 294 deletions

View File

@ -542,7 +542,7 @@ Alias: `scalarProduct`.
- Scalar product.
Type: [Int/UInt](../../sql-reference/data-types/int-uint.md), [Float](../../sql-reference/data-types/float.md) or [Decimal](../../sql-reference/data-types/decimal.md).
Type: [Int/UInt](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md).
**Example**

View File

@ -90,17 +90,19 @@ struct L2Distance
size_t & i_y,
State<ResultType> & state)
{
static constexpr bool is_float32 = std::is_same_v<ResultType, Float32>;
__m512 sums;
if constexpr (std::is_same_v<ResultType, Float32>)
if constexpr (is_float32)
sums = _mm512_setzero_ps();
else
sums = _mm512_setzero_pd();
const size_t n = (std::is_same_v<ResultType, Float32>) ? 16 : 8;
constexpr size_t n = is_float32 ? 16 : 8;
for (; i_x + n < i_max; i_x += n, i_y += n)
{
if constexpr (std::is_same_v<ResultType, Float32>)
if constexpr (is_float32)
{
__m512 x = _mm512_loadu_ps(data_x + i_x);
__m512 y = _mm512_loadu_ps(data_y + i_y);
@ -116,7 +118,7 @@ struct L2Distance
}
}
if constexpr (std::is_same_v<ResultType, Float32>)
if constexpr (is_float32)
state.sum = _mm512_reduce_add_ps(sums);
else
state.sum = _mm512_reduce_add_pd(sums);
@ -247,11 +249,13 @@ struct CosineDistance
size_t & i_y,
State<ResultType> & state)
{
static constexpr bool is_float32 = std::is_same_v<ResultType, Float32>;
__m512 dot_products;
__m512 x_squareds;
__m512 y_squareds;
if constexpr (std::is_same_v<ResultType, Float32>)
if constexpr (is_float32)
{
dot_products = _mm512_setzero_ps();
x_squareds = _mm512_setzero_ps();
@ -264,11 +268,11 @@ struct CosineDistance
y_squareds = _mm512_setzero_pd();
}
const size_t n = (std::is_same_v<ResultType, Float32>) ? 16 : 8;
constexpr size_t n = is_float32 ? 16 : 8;
for (; i_x + n < i_max; i_x += n, i_y += n)
{
if constexpr (std::is_same_v<ResultType, Float32>)
if constexpr (is_float32)
{
__m512 x = _mm512_loadu_ps(data_x + i_x);
__m512 y = _mm512_loadu_ps(data_y + i_y);
@ -286,7 +290,7 @@ struct CosineDistance
}
}
if constexpr (std::is_same_v<ResultType, Float32>)
if constexpr (is_float32)
{
state.dot_prod = _mm512_reduce_add_ps(dot_products);
state.x_squared = _mm512_reduce_add_ps(x_squareds);
@ -312,7 +316,11 @@ template <class Kernel>
class FunctionArrayDistance : public IFunction
{
public:
String getName() const override { static auto name = String("array") + Kernel::name + "Distance"; return name; }
String getName() const override
{
static auto name = String("array") + Kernel::name + "Distance";
return name;
}
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayDistance<Kernel>>(); }
size_t getNumberOfArguments() const override { return 2; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }

View File

@ -1,44 +1,51 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Core/Types_fwd.h>
#include <DataTypes/Serializations/ISerialization.h>
#include <Functions/castTypeToEither.h>
#include <Functions/array/arrayScalarProduct.h>
#include <base/types.h>
#include <Functions/FunctionBinaryArithmetic.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Functions/castTypeToEither.h>
#include <Interpreters/Context_fwd.h>
#include <base/types.h>
#if USE_MULTITARGET_CODE
#include <immintrin.h>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int LOGICAL_ERROR;
}
struct NameArrayDotProduct
struct DotProduct
{
static constexpr auto name = "arrayDotProduct";
};
class ArrayDotProductImpl
{
public:
static DataTypePtr getReturnType(const DataTypePtr & left, const DataTypePtr & right)
{
using Types = TypeList<DataTypeFloat32, DataTypeFloat64,
DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64,
DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64>;
Types types;
DataTypePtr result_type;
bool valid = castTypeToEither(Types{}, left.get(), [&](const auto & left_)
bool valid = castTypeToEither(types, left.get(), [&](const auto & left_)
{
return castTypeToEither(Types{}, right.get(), [&](const auto & right_)
return castTypeToEither(types, right.get(), [&](const auto & right_)
{
using LeftDataType = typename std::decay_t<decltype(left_)>::FieldType;
using RightDataType = typename std::decay_t<decltype(right_)>::FieldType;
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<LeftDataType, RightDataType>::Type;
if (std::is_same_v<LeftDataType, Float32> && std::is_same_v<RightDataType, Float32>)
using LeftType = typename std::decay_t<decltype(left_)>::FieldType;
using RightType = typename std::decay_t<decltype(right_)>::FieldType;
using ResultType = typename NumberTraits::ResultOfAdditionMultiplication<LeftType, RightType>::Type;
if constexpr (std::is_same_v<LeftType, Float32> && std::is_same_v<RightType, Float32>)
result_type = std::make_shared<DataTypeFloat32>();
else
result_type = std::make_shared<DataTypeFromFieldType<ResultType>>();
@ -49,26 +56,268 @@ public:
if (!valid)
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Arguments of function {} "
"only support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.",
std::string(NameArrayDotProduct::name));
"Arguments of function {} only support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.", name);
return result_type;
}
template <typename ResultType, typename T, typename U>
static inline NO_SANITIZE_UNDEFINED ResultType apply(
const T * left,
const U * right,
size_t size)
template <typename Type>
struct State
{
ResultType result = 0;
for (size_t i = 0; i < size; ++i)
result += static_cast<ResultType>(left[i]) * static_cast<ResultType>(right[i]);
return result;
Type sum = 0;
};
template <typename Type>
static void accumulate(State<Type> & state, Type x, Type y)
{
state.sum += x * y;
}
template <typename Type>
static void combine(State<Type> & state, const State<Type> & other_state)
{
state.sum += other_state.sum;
}
#if USE_MULTITARGET_CODE
template <typename Type>
AVX512_FUNCTION_SPECIFIC_ATTRIBUTE static void accumulateCombine(
const Type * __restrict data_x,
const Type * __restrict data_y,
size_t i_max,
size_t & i,
State<Type> & state)
{
static constexpr bool is_float32 = std::is_same_v<Type, Float32>;
__m512 sums;
if constexpr (is_float32)
sums = _mm512_setzero_ps();
else
sums = _mm512_setzero_pd();
constexpr size_t n = is_float32 ? 16 : 8;
for (; i + n < i_max; i += n)
{
if constexpr (is_float32)
{
__m512 x = _mm512_loadu_ps(data_x + i);
__m512 y = _mm512_loadu_ps(data_y + i);
sums = _mm512_fmadd_ps(x, y, sums);
}
else
{
__m512 x = _mm512_loadu_pd(data_x + i);
__m512 y = _mm512_loadu_pd(data_y + i);
sums = _mm512_fmadd_pd(x, y, sums);
}
}
if constexpr (is_float32)
state.sum = _mm512_reduce_add_ps(sums);
else
state.sum = _mm512_reduce_add_pd(sums);
}
#endif
template <typename Type>
static Type finalize(const State<Type> & state)
{
return state.sum;
}
};
/// The implementation is modeled after the implementation of distance functions arrayL1Distance, arrayL2Distance, etc.
/// The main difference is that arrayDotProduct() interferes the result type differently.
template <typename Kernel>
class FunctionArrayScalarProduct : public IFunction
{
public:
static constexpr auto name = Kernel::name;
String getName() const override { return name; }
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayScalarProduct>(); }
size_t getNumberOfArguments() const override { return 2; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
std::array<DataTypePtr, 2> nested_types;
for (size_t i = 0; i < 2; ++i)
{
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[i].get());
if (!array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Arguments for function {} must be of type Array", getName());
const auto & nested_type = array_type->getNestedType();
if (!isNativeNumber(nested_type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Function {} cannot process values of type {}", getName(), nested_type->getName());
nested_types[i] = nested_type;
}
return Kernel::getReturnType(nested_types[0], nested_types[1]);
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override
{
switch (result_type->getTypeId())
{
#define SUPPORTED_TYPE(type) \
case TypeIndex::type: \
return executeWithResultType<type>(arguments); \
break;
SUPPORTED_TYPE(UInt8)
SUPPORTED_TYPE(UInt16)
SUPPORTED_TYPE(UInt32)
SUPPORTED_TYPE(UInt64)
SUPPORTED_TYPE(Int8)
SUPPORTED_TYPE(Int16)
SUPPORTED_TYPE(Int32)
SUPPORTED_TYPE(Int64)
SUPPORTED_TYPE(Float32)
SUPPORTED_TYPE(Float64)
#undef SUPPORTED_TYPE
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type {}", result_type->getName());
}
}
private:
template <typename ResultType>
ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const
{
ColumnPtr res;
if (!((res = executeWithResultTypeAndLeft<ResultType, UInt8>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, UInt16>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, UInt32>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, UInt64>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, Int8>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, Int16>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, Int32>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, Int64>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, Float32>(arguments))
|| (res = executeWithResultTypeAndLeft<ResultType, Float64>(arguments))))
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName());
return res;
}
template <typename ResultType, typename LeftType>
ColumnPtr executeWithResultTypeAndLeft(const ColumnsWithTypeAndName & arguments) const
{
ColumnPtr res;
if ( (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, UInt8>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, UInt16>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, UInt32>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, UInt64>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, Int8>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, Int16>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, Int32>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, Int64>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, Float32>(arguments))
|| (res = executeWithResultTypeAndLeftAndRight<ResultType, LeftType, Float64>(arguments)))
return res;
return nullptr;
}
template <typename ResultType, typename LeftType, typename RightType>
ColumnPtr executeWithResultTypeAndLeftAndRight(const ColumnsWithTypeAndName & arguments) const
{
ColumnPtr col_left = arguments[0].column->convertToFullColumnIfConst();
ColumnPtr col_right = arguments[1].column->convertToFullColumnIfConst();
if (!col_left || !col_right)
return nullptr;
const ColumnArray * col_arr_left = checkAndGetColumn<ColumnArray>(col_left.get());
const ColumnArray * cokl_arr_right = checkAndGetColumn<ColumnArray>(col_right.get());
if (!col_arr_left || !cokl_arr_right)
return nullptr;
const ColumnVector<LeftType> * col_arr_nested_left = checkAndGetColumn<ColumnVector<LeftType>>(col_arr_left->getData());
const ColumnVector<RightType> * col_arr_nested_right = checkAndGetColumn<ColumnVector<RightType>>(cokl_arr_right->getData());
if (!col_arr_nested_left || !col_arr_nested_right)
return nullptr;
if (!col_arr_left->hasEqualOffsets(*cokl_arr_right))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName());
auto col_res = ColumnVector<ResultType>::create();
vector(
col_arr_nested_left->getData(),
col_arr_nested_right->getData(),
col_arr_left->getOffsets(),
col_res->getData());
return col_res;
}
template <typename ResultType, typename LeftType, typename RightType>
static void vector(
const PaddedPODArray<LeftType> & left,
const PaddedPODArray<RightType> & right,
const ColumnArray::Offsets & offsets,
PaddedPODArray<ResultType> & result)
{
size_t size = offsets.size();
result.resize(size);
ColumnArray::Offset current_offset = 0;
for (size_t row = 0; row < size; ++row)
{
size_t array_size = offsets[row] - current_offset;
typename Kernel::template State<ResultType> state;
size_t i = 0;
/// SIMD optimization: process multiple elements in both input arrays at once.
/// To avoid combinatorial explosion of SIMD kernels, focus on
/// - the two most common input/output types (Float32 x Float32) --> Float32 and (Float64 x Float64) --> Float64 instead of 10 x
/// 10 input types x 8 output types,
/// - the most powerful SIMD instruction set (AVX-512F).
#if USE_MULTITARGET_CODE
if constexpr ((std::is_same_v<ResultType, Float32> || std::is_same_v<ResultType, Float64>)
&& std::is_same_v<ResultType, LeftType> && std::is_same_v<LeftType, RightType>)
{
if (isArchSupported(TargetArch::AVX512F))
Kernel::template accumulateCombine<ResultType>(&left[current_offset], &right[current_offset], array_size, i, state);
}
#else
/// Process chunks in vectorized manner
static constexpr size_t VEC_SIZE = 4;
typename Kernel::template State<ResultType> states[VEC_SIZE];
for (; i + VEC_SIZE < array_size; i += VEC_SIZE)
{
for (size_t j = 0; j < VEC_SIZE; ++j)
Kernel::template accumulate<ResultType>(states[j], static_cast<ResultType>(left[i + j]), static_cast<ResultType>(right[i + j]));
}
for (const auto & other_state : states)
Kernel::template combine<ResultType>(state, other_state);
#endif
/// Process the tail
for (; i < array_size; ++i)
Kernel::template accumulate<ResultType>(state, static_cast<ResultType>(left[i]), static_cast<ResultType>(right[i]));
/// ResultType res = Kernel::template finalize<ResultType>(state);
result[row] = Kernel::template finalize<ResultType>(state);
current_offset = offsets[row];
}
}
};
using FunctionArrayDotProduct = FunctionArrayScalarProduct<ArrayDotProductImpl, NameArrayDotProduct>;
using FunctionArrayDotProduct = FunctionArrayScalarProduct<DotProduct>;
REGISTER_FUNCTION(ArrayDotProduct)
{
@ -77,4 +326,5 @@ REGISTER_FUNCTION(ArrayDotProduct)
// These functions are used by TupleOrArrayFunction in Function/vectorFunctions.cpp
FunctionPtr createFunctionArrayDotProduct(ContextPtr context_) { return FunctionArrayDotProduct::create(context_); }
}

View File

@ -1,182 +0,0 @@
#pragma once
#include <Columns/ColumnArray.h>
#include <Columns/ColumnVector.h>
#include <DataTypes/DataTypeArray.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/Context_fwd.h>
#include <Core/TypeId.h>
namespace DB
{
class Context;
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
template <typename Method, typename Name>
class FunctionArrayScalarProduct : public IFunction
{
public:
static constexpr auto name = Name::name;
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayScalarProduct>(); }
private:
template <typename ResultType, typename T>
ColumnPtr executeNumber(const ColumnsWithTypeAndName & arguments) const
{
ColumnPtr res;
if ( (res = executeNumberNumber<ResultType, T, UInt8>(arguments))
|| (res = executeNumberNumber<ResultType, T, UInt16>(arguments))
|| (res = executeNumberNumber<ResultType, T, UInt32>(arguments))
|| (res = executeNumberNumber<ResultType, T, UInt64>(arguments))
|| (res = executeNumberNumber<ResultType, T, Int8>(arguments))
|| (res = executeNumberNumber<ResultType, T, Int16>(arguments))
|| (res = executeNumberNumber<ResultType, T, Int32>(arguments))
|| (res = executeNumberNumber<ResultType, T, Int64>(arguments))
|| (res = executeNumberNumber<ResultType, T, Float32>(arguments))
|| (res = executeNumberNumber<ResultType, T, Float64>(arguments)))
return res;
return nullptr;
}
template <typename ResultType, typename T, typename U>
ColumnPtr executeNumberNumber(const ColumnsWithTypeAndName & arguments) const
{
ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst();
ColumnPtr col2 = arguments[1].column->convertToFullColumnIfConst();
if (!col1 || !col2)
return nullptr;
const ColumnArray * col_array1 = checkAndGetColumn<ColumnArray>(col1.get());
const ColumnArray * col_array2 = checkAndGetColumn<ColumnArray>(col2.get());
if (!col_array1 || !col_array2)
return nullptr;
if (!col_array1->hasEqualOffsets(*col_array2))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Array arguments for function {} must have equal sizes", getName());
const ColumnVector<T> * col_nested1 = checkAndGetColumn<ColumnVector<T>>(col_array1->getData());
const ColumnVector<U> * col_nested2 = checkAndGetColumn<ColumnVector<U>>(col_array2->getData());
if (!col_nested1 || !col_nested2)
return nullptr;
auto col_res = ColumnVector<ResultType>::create();
vector(
col_nested1->getData(),
col_nested2->getData(),
col_array1->getOffsets(),
col_res->getData());
return col_res;
}
template <typename ResultType, typename T, typename U>
static NO_INLINE void vector(
const PaddedPODArray<T> & data1,
const PaddedPODArray<U> & data2,
const ColumnArray::Offsets & offsets,
PaddedPODArray<ResultType> & result)
{
size_t size = offsets.size();
result.resize(size);
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
size_t array_size = offsets[i] - current_offset;
result[i] = Method::template apply<ResultType, T, U>(&data1[current_offset], &data2[current_offset], array_size);
current_offset = offsets[i];
}
}
public:
String getName() const override { return name; }
size_t getNumberOfArguments() const override { return 2; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
// Basic type check
std::vector<DataTypePtr> nested_types(2, nullptr);
for (size_t i = 0; i < getNumberOfArguments(); ++i)
{
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[i].get());
if (!array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "All arguments for function {} must be an array.", getName());
const auto & nested_type = array_type->getNestedType();
if (!isNativeNumber(nested_type) && !isEnum(nested_type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{} cannot process values of type {}",
getName(), nested_type->getName());
nested_types[i] = nested_type;
}
// Detail type check in Method, then return ReturnType
return Method::getReturnType(nested_types[0], nested_types[1]);
}
template <typename ResultType>
ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const
{
ColumnPtr res;
if (!((res = executeNumber<ResultType, UInt8>(arguments))
|| (res = executeNumber<ResultType, UInt16>(arguments))
|| (res = executeNumber<ResultType, UInt32>(arguments))
|| (res = executeNumber<ResultType, UInt64>(arguments))
|| (res = executeNumber<ResultType, Int8>(arguments))
|| (res = executeNumber<ResultType, Int16>(arguments))
|| (res = executeNumber<ResultType, Int32>(arguments))
|| (res = executeNumber<ResultType, Int64>(arguments))
|| (res = executeNumber<ResultType, Float32>(arguments))
|| (res = executeNumber<ResultType, Float64>(arguments))))
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Illegal column {} of first argument of function {}", arguments[0].column->getName(), getName());
return res;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /* input_rows_count */) const override
{
switch (result_type->getTypeId())
{
#define SUPPORTED_TYPE(type) \
case TypeIndex::type: \
return executeWithResultType<type>(arguments); \
break;
SUPPORTED_TYPE(UInt8)
SUPPORTED_TYPE(UInt16)
SUPPORTED_TYPE(UInt32)
SUPPORTED_TYPE(UInt64)
SUPPORTED_TYPE(Int8)
SUPPORTED_TYPE(Int16)
SUPPORTED_TYPE(Int32)
SUPPORTED_TYPE(Int64)
SUPPORTED_TYPE(Float32)
SUPPORTED_TYPE(Float64)
#undef SUPPORTED_TYPE
default:
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type {}", result_type->getName());
}
}
};
}

View File

@ -0,0 +1,62 @@
<test>
<substitutions>
<substitution>
<name>element_type</name>
<values>
<!-- integer cases are not relevant in practice for vector search, disable to reduce test runtime -->
<!-- <value>UInt8</value> -->
<!-- <value>Int16</value> -->
<!-- <value>Int32</value> -->
<!-- <value>Int64</value> -->
<value>Float32</value>
<value>Float64</value>
</values>
</substitution>
</substitutions>
<create_query>
CREATE TABLE vecs_{element_type} (
v Array({element_type})
) ENGINE=Memory;
</create_query>
<!-- Gererate arrays with random data -->
<!-- Dimension = 150 is realistic for vector search use cases -->
<fill_query>
INSERT INTO vecs_{element_type}
SELECT v FROM (
SELECT
number AS n,
[
rand(n*10), rand(n*10+1), rand(n*10+2), rand(n*10+3), rand(n*10+4), rand(n*10+5), rand(n*10+6), rand(n*10+7), rand(n*10+8), rand(n*10+9),
rand(n*10+10), rand(n*10+11), rand(n*10+12), rand(n*10+13), rand(n*10+14), rand(n*10+15), rand(n*10+16), rand(n*10+17), rand(n*10+18), rand(n*10+19),
rand(n*10+20), rand(n*10+21), rand(n*10+22), rand(n*10+23), rand(n*10+24), rand(n*10+25), rand(n*10+26), rand(n*10+27), rand(n*10+28), rand(n*10+29),
rand(n*10+30), rand(n*10+31), rand(n*10+32), rand(n*10+33), rand(n*10+34), rand(n*10+35), rand(n*10+36), rand(n*10+37), rand(n*10+38), rand(n*10+39),
rand(n*10+40), rand(n*10+41), rand(n*10+42), rand(n*10+43), rand(n*10+44), rand(n*10+45), rand(n*10+46), rand(n*10+47), rand(n*10+48), rand(n*10+49),
rand(n*10+50), rand(n*10+51), rand(n*10+52), rand(n*10+53), rand(n*10+54), rand(n*10+55), rand(n*10+56), rand(n*10+57), rand(n*10+58), rand(n*10+59),
rand(n*10+60), rand(n*10+61), rand(n*10+62), rand(n*10+63), rand(n*10+64), rand(n*10+65), rand(n*10+66), rand(n*10+67), rand(n*10+68), rand(n*10+69),
rand(n*10+70), rand(n*10+71), rand(n*10+72), rand(n*10+73), rand(n*10+74), rand(n*10+75), rand(n*10+76), rand(n*10+77), rand(n*10+78), rand(n*10+79),
rand(n*10+80), rand(n*10+81), rand(n*10+82), rand(n*10+83), rand(n*10+84), rand(n*10+85), rand(n*10+86), rand(n*10+87), rand(n*10+88), rand(n*10+89),
rand(n*10+90), rand(n*10+91), rand(n*10+92), rand(n*10+93), rand(n*10+94), rand(n*10+95), rand(n*10+96), rand(n*10+97), rand(n*10+98), rand(n*10+99),
rand(n*10+100), rand(n*10+101), rand(n*10+102), rand(n*10+103), rand(n*10+104), rand(n*10+105), rand(n*10+106), rand(n*10+107), rand(n*10+108), rand(n*10+109),
rand(n*10+110), rand(n*10+111), rand(n*10+112), rand(n*10+113), rand(n*10+114), rand(n*10+115), rand(n*10+116), rand(n*10+117), rand(n*10+118), rand(n*10+119),
rand(n*10+120), rand(n*10+121), rand(n*10+122), rand(n*10+123), rand(n*10+124), rand(n*10+125), rand(n*10+126), rand(n*10+127), rand(n*10+128), rand(n*10+129),
rand(n*10+130), rand(n*10+131), rand(n*10+132), rand(n*10+133), rand(n*10+134), rand(n*10+135), rand(n*10+136), rand(n*10+137), rand(n*10+138), rand(n*10+139),
rand(n*10+140), rand(n*10+141), rand(n*10+142), rand(n*10+143), rand(n*10+144), rand(n*10+145), rand(n*10+146), rand(n*10+147), rand(n*10+148), rand(n*10+149)
] AS v
FROM system.numbers
LIMIT 5000000
);
</fill_query>
<settings>
<max_threads>1</max_threads>
</settings>
<query>SELECT sum(dp) FROM (SELECT dotProduct(v, v) AS dp FROM vecs_{element_type})</query>
<drop_query>DROP TABLE vecs_{element_type}</drop_query>
</test>

View File

@ -4,11 +4,11 @@
<substitution>
<name>element_type</name>
<values>
<!-- 8 and 16 bit cases are not relevant in practice, disable to reduce test runtime -->
<!-- integer cases are not relevant in practice for vector search, disable to reduce test runtime -->
<!-- <value>UInt8</value> -->
<!-- <value>Int16</value> -->
<value>Int32</value>
<value>Int64</value>
<!-- <value>Int32</value> -->
<!-- <value>Int64</value> -->
<value>Float32</value>
<value>Float64</value>
</values>

View File

@ -0,0 +1,34 @@
-- Negative tests
-- Tests
-- Array
[1,2,3] [4,5,6] 32 UInt16
[1,2,3] [4,5,6] 32 UInt32
[1,2,3] [4,5,6] 32 UInt64
[1,2,3] [4,5,6] 32 UInt64
[-1,-2,-3] [4,5,6] -32 Int16
[-1,-2,-3] [4,5,6] -32 Int32
[-1,-2,-3] [4,5,6] -32 Int64
[-1,-2,-3] [4,5,6] -32 Int64
[1,2,3] [4,5,6] 32 Float32
[1,2,3] [4,5,6] 32 Float64
-- Tuple
(1,2,3) (4,5,6) 32 UInt64
(1,2,3) (4,5,6) 32 UInt64
(1,2,3) (4,5,6) 32 UInt64
(1,2,3) (4,5,6) 32 UInt64
(-1,-2,-3) (4,5,6) -32 Int64
(-1,-2,-3) (4,5,6) -32 Int64
(-1,-2,-3) (4,5,6) -32 Int64
(-1,-2,-3) (4,5,6) -32 Int64
(1,2,3) (4,5,6) 32 Float64
(1,2,3) (4,5,6) 32 Float64
-- Non-const argument
[1,2,3] [4,5,6] 32 UInt16
-- Array with mixed element arguments types (result type is the supertype)
[1,2,3] [4,5,6] 32 Float32
-- Tuple with mixed element arguments types
(1,2,3) (4,5,6) 32 Float64
-- Aliases
32
32
32

View File

@ -0,0 +1,47 @@
SELECT '-- Negative tests';
SELECT arrayDotProduct([1, 2]); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH }
SELECT arrayDotProduct([1, 2], 'abc'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayDotProduct('abc', [1, 2]); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayDotProduct([1, 2], ['abc', 'def']); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayDotProduct([1, 2], [3, 4, 5]); -- { serverError BAD_ARGUMENTS }
SELECT dotProduct([1, 2], (3, 4, 5)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT '-- Tests';
SELECT ' -- Array';
SELECT [1, 2, 3]::Array(UInt8) AS x, [4, 5, 6]::Array(UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [1, 2, 3]::Array(UInt16) AS x, [4, 5, 6]::Array(UInt16) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [1, 2, 3]::Array(UInt32) AS x, [4, 5, 6]::Array(UInt32) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [1, 2, 3]::Array(UInt64) AS x, [4, 5, 6]::Array(UInt64) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [-1, -2, -3]::Array(Int8) AS x, [4, 5, 6]::Array(Int8) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [-1, -2, -3]::Array(Int16) AS x, [4, 5, 6]::Array(Int16) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [-1, -2, -3]::Array(Int32) AS x, [4, 5, 6]::Array(Int32) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [-1, -2, -3]::Array(Int64) AS x, [4, 5, 6]::Array(Int64) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [1, 2, 3]::Array(Float32) AS x, [4, 5, 6]::Array(Float32) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT [1, 2, 3]::Array(Float64) AS x, [4, 5, 6]::Array(Float64) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT ' -- Tuple';
SELECT (1::UInt8, 2::UInt8, 3::UInt8) AS x, (4::UInt8, 5::UInt8, 6::UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (1::UInt16, 2::UInt16, 3::UInt16) AS x, (4::UInt16, 5::UInt16, 6::UInt16) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (1::UInt32, 2::UInt32, 3::UInt32) AS x, (4::UInt32, 5::UInt32, 6::UInt32) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (1::UInt64, 2::UInt64, 3::UInt64) AS x, (4::UInt64, 5::UInt64, 6::UInt64) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (-1::Int8, -2::Int8, -3::Int8) AS x, (4::Int8, 5::Int8, 6::Int8) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (-1::Int16, -2::Int16, -3::Int16) AS x, (4::Int16, 5::Int16, 6::Int16) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (-1::Int32, -2::Int32, -3::Int32) AS x, (4::Int32, 5::Int32, 6::Int32) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (-1::Int64, -2::Int64, -3::Int64) AS x, (4::Int64, 5::Int64, 6::Int64) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (1::Float32, 2::Float32, 3::Float32) AS x, (4::Float32, 5::Float32, 6::Float32) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT (1::Float64, 2::Float64, 3::Float64) AS x, (4::Float64, 5::Float64, 6::Float64) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT '-- Non-const argument';
SELECT materialize([1::UInt8, 2::UInt8, 3::UInt8]) AS x, [4::UInt8, 5::UInt8, 6::UInt8] AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT ' -- Array with mixed element arguments types (result type is the supertype)';
SELECT [1::UInt16, 2::UInt8, 3::Float32] AS x, [4::Int16, 5::Float32, 6::UInt8] AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT ' -- Tuple with mixed element arguments types';
SELECT (1::UInt16, 2::UInt8, 3::Float32) AS x, (4::Int16, 5::Float32, 6::UInt8) AS y, dotProduct(x, y) AS res, toTypeName(res);
SELECT '-- Aliases';
SELECT scalarProduct([1, 2, 3], [4, 5, 6]);
SELECT scalarProduct((1, 2, 3), (4, 5, 6));
SELECT arrayDotProduct([1, 2, 3], [4, 5, 6]); -- actually no alias but the internal function for arrays

View File

@ -1,14 +0,0 @@
3881.304
3881.304
3881.304
376.5
230
0
0
Float64
Float32
Float64
Float64
UInt16
UInt64
Int64

View File

@ -1,55 +0,0 @@
SELECT dotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]);
SELECT scalarProduct([12, 2.22, 302], [1.32, 231.2, 11.1]);
SELECT arrayDotProduct([12, 2.22, 302], [1.32, 231.2, 11.1]);
SELECT dotProduct([1.3, 2, 3, 4, 5], [222, 12, 5.3, 2, 8]);
SELECT dotProduct([1, 1, 1, 1, 1], [222, 12, 0, -12, 8]);
SELECT round(dotProduct([12345678901234567], [1]) - dotProduct(tuple(12345678901234567), tuple(1)), 2);
SELECT round(dotProduct([-1, 2, 3.002], [2, 3.4, 4]) - dotProduct((-1, 2, 3.002), (2, 3.4, 4)), 2);
DROP TABLE IF EXISTS product_fp64_fp64;
CREATE TABLE product_fp64_fp64 (x Array(Float64), y Array(Float64)) engine = MergeTree() order by x;
INSERT INTO TABLE product_fp64_fp64 (x, y) values ([1, 2], [3, 4]);
SELECT toTypeName(dotProduct(x, y)) from product_fp64_fp64;
DROP TABLE product_fp64_fp64;
DROP TABLE IF EXISTS product_fp32_fp32;
CREATE TABLE product_fp32_fp32 (x Array(Float32), y Array(Float32)) engine = MergeTree() order by x;
INSERT INTO TABLE product_fp32_fp32 (x, y) values ([1, 2], [3, 4]);
SELECT toTypeName(dotProduct(x, y)) from product_fp32_fp32;
DROP TABLE product_fp32_fp32;
DROP TABLE IF EXISTS product_fp32_fp64;
CREATE TABLE product_fp32_fp64 (x Array(Float32), y Array(Float64)) engine = MergeTree() order by x;
INSERT INTO TABLE product_fp32_fp64 (x, y) values ([1, 2], [3, 4]);
SELECT toTypeName(dotProduct(x, y)) from product_fp32_fp64;
DROP TABLE product_fp32_fp64;
DROP TABLE IF EXISTS product_uint8_fp64;
CREATE TABLE product_uint8_fp64 (x Array(UInt8), y Array(Float64)) engine = MergeTree() order by x;
INSERT INTO TABLE product_uint8_fp64 (x, y) values ([1, 2], [3, 4]);
SELECT toTypeName(dotProduct(x, y)) from product_uint8_fp64;
DROP TABLE product_uint8_fp64;
DROP TABLE IF EXISTS product_uint8_uint8;
CREATE TABLE product_uint8_uint8 (x Array(UInt8), y Array(UInt8)) engine = MergeTree() order by x;
INSERT INTO TABLE product_uint8_uint8 (x, y) values ([1, 2], [3, 4]);
SELECT toTypeName(dotProduct(x, y)) from product_uint8_uint8;
DROP TABLE product_uint8_uint8;
DROP TABLE IF EXISTS product_uint64_uint64;
CREATE TABLE product_uint64_uint64 (x Array(UInt64), y Array(UInt64)) engine = MergeTree() order by x;
INSERT INTO TABLE product_uint64_uint64 (x, y) values ([1, 2], [3, 4]);
SELECT toTypeName(dotProduct(x, y)) from product_uint64_uint64;
DROP TABLE product_uint64_uint64;
DROP TABLE IF EXISTS product_int32_uint64;
CREATE TABLE product_int32_uint64 (x Array(Int32), y Array(UInt64)) engine = MergeTree() order by x;
INSERT INTO TABLE product_int32_uint64 (x, y) values ([1, 2], [3, 4]);
SELECT toTypeName(dotProduct(x, y)) from product_int32_uint64;
DROP TABLE product_int32_uint64;