mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #49050 from FFFFFFFHHHHHHH/dot_product
Add Function dotProduct for array
This commit is contained in:
commit
4e3188126f
@ -90,7 +90,7 @@ public:
|
||||
return std::make_shared<DataTypeNumber<ResultType>>();
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
template <typename ResultType, typename T, typename U>
|
||||
static ResultType apply(
|
||||
const T * scores,
|
||||
const U * labels,
|
||||
|
81
src/Functions/array/arrayDotProduct.cpp
Normal file
81
src/Functions/array/arrayDotProduct.cpp
Normal file
@ -0,0 +1,81 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <DataTypes/getLeastSupertype.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>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
struct NameArrayDotProduct
|
||||
{
|
||||
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>;
|
||||
|
||||
DataTypePtr result_type;
|
||||
bool valid = castTypeToEither(Types{}, left.get(), [&](const auto & left_)
|
||||
{
|
||||
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>)
|
||||
result_type = std::make_shared<DataTypeFloat32>();
|
||||
else
|
||||
result_type = std::make_shared<DataTypeFromFieldType<ResultType>>();
|
||||
return true;
|
||||
});
|
||||
});
|
||||
|
||||
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));
|
||||
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)
|
||||
{
|
||||
ResultType result = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
result += static_cast<ResultType>(left[i]) * static_cast<ResultType>(right[i]);
|
||||
return result;
|
||||
}
|
||||
};
|
||||
|
||||
using FunctionArrayDotProduct = FunctionArrayScalarProduct<ArrayDotProductImpl, NameArrayDotProduct>;
|
||||
|
||||
REGISTER_FUNCTION(ArrayDotProduct)
|
||||
{
|
||||
factory.registerFunction<FunctionArrayDotProduct>();
|
||||
}
|
||||
|
||||
// These functions are used by TupleOrArrayFunction in Function/vectorFunctions.cpp
|
||||
FunctionPtr createFunctionArrayDotProduct(ContextPtr context_) { return FunctionArrayDotProduct::create(context_); }
|
||||
}
|
@ -6,6 +6,7 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <Core/TypeId.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -18,6 +19,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -29,29 +31,28 @@ public:
|
||||
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayScalarProduct>(); }
|
||||
|
||||
private:
|
||||
using ResultColumnType = ColumnVector<typename Method::ResultType>;
|
||||
|
||||
template <typename T>
|
||||
template <typename ResultType, typename T>
|
||||
ColumnPtr executeNumber(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
ColumnPtr res;
|
||||
if ( (res = executeNumberNumber<T, UInt8>(arguments))
|
||||
|| (res = executeNumberNumber<T, UInt16>(arguments))
|
||||
|| (res = executeNumberNumber<T, UInt32>(arguments))
|
||||
|| (res = executeNumberNumber<T, UInt64>(arguments))
|
||||
|| (res = executeNumberNumber<T, Int8>(arguments))
|
||||
|| (res = executeNumberNumber<T, Int16>(arguments))
|
||||
|| (res = executeNumberNumber<T, Int32>(arguments))
|
||||
|| (res = executeNumberNumber<T, Int64>(arguments))
|
||||
|| (res = executeNumberNumber<T, Float32>(arguments))
|
||||
|| (res = executeNumberNumber<T, Float64>(arguments)))
|
||||
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 T, typename U>
|
||||
template <typename ResultType, typename T, typename U>
|
||||
ColumnPtr executeNumberNumber(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
ColumnPtr col1 = arguments[0].column->convertToFullColumnIfConst();
|
||||
@ -72,7 +73,7 @@ private:
|
||||
if (!col_nested1 || !col_nested2)
|
||||
return nullptr;
|
||||
|
||||
auto col_res = ResultColumnType::create();
|
||||
auto col_res = ColumnVector<ResultType>::create();
|
||||
|
||||
vector(
|
||||
col_nested1->getData(),
|
||||
@ -83,12 +84,12 @@ private:
|
||||
return col_res;
|
||||
}
|
||||
|
||||
template <typename T, typename U>
|
||||
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<typename Method::ResultType> & result)
|
||||
PaddedPODArray<ResultType> & result)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
result.resize(size);
|
||||
@ -97,7 +98,7 @@ private:
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t array_size = offsets[i] - current_offset;
|
||||
result[i] = Method::apply(&data1[current_offset], &data2[current_offset], array_size);
|
||||
result[i] = Method::template apply<ResultType, T, U>(&data1[current_offset], &data2[current_offset], array_size);
|
||||
current_offset = offsets[i];
|
||||
}
|
||||
}
|
||||
@ -130,24 +131,51 @@ public:
|
||||
return Method::getReturnType(nested_types[0], nested_types[1]);
|
||||
}
|
||||
|
||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /* input_rows_count */) const override
|
||||
template <typename ResultType>
|
||||
ColumnPtr executeWithResultType(const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
ColumnPtr res;
|
||||
if (!((res = executeNumber<UInt8>(arguments))
|
||||
|| (res = executeNumber<UInt16>(arguments))
|
||||
|| (res = executeNumber<UInt32>(arguments))
|
||||
|| (res = executeNumber<UInt64>(arguments))
|
||||
|| (res = executeNumber<Int8>(arguments))
|
||||
|| (res = executeNumber<Int16>(arguments))
|
||||
|| (res = executeNumber<Int32>(arguments))
|
||||
|| (res = executeNumber<Int64>(arguments))
|
||||
|| (res = executeNumber<Float32>(arguments))
|
||||
|| (res = executeNumber<Float64>(arguments))))
|
||||
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());
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1429,6 +1429,8 @@ private:
|
||||
FunctionPtr array_function;
|
||||
};
|
||||
|
||||
extern FunctionPtr createFunctionArrayDotProduct(ContextPtr context_);
|
||||
|
||||
extern FunctionPtr createFunctionArrayL1Norm(ContextPtr context_);
|
||||
extern FunctionPtr createFunctionArrayL2Norm(ContextPtr context_);
|
||||
extern FunctionPtr createFunctionArrayL2SquaredNorm(ContextPtr context_);
|
||||
@ -1442,6 +1444,14 @@ extern FunctionPtr createFunctionArrayLpDistance(ContextPtr context_);
|
||||
extern FunctionPtr createFunctionArrayLinfDistance(ContextPtr context_);
|
||||
extern FunctionPtr createFunctionArrayCosineDistance(ContextPtr context_);
|
||||
|
||||
struct DotProduct
|
||||
{
|
||||
static constexpr auto name = "dotProduct";
|
||||
|
||||
static constexpr auto CreateTupleFunction = FunctionDotProduct::create;
|
||||
static constexpr auto CreateArrayFunction = createFunctionArrayDotProduct;
|
||||
};
|
||||
|
||||
struct L1NormTraits
|
||||
{
|
||||
static constexpr auto name = "L1Norm";
|
||||
@ -1530,6 +1540,8 @@ struct CosineDistanceTraits
|
||||
static constexpr auto CreateArrayFunction = createFunctionArrayCosineDistance;
|
||||
};
|
||||
|
||||
using TupleOrArrayFunctionDotProduct = TupleOrArrayFunction<DotProduct>;
|
||||
|
||||
using TupleOrArrayFunctionL1Norm = TupleOrArrayFunction<L1NormTraits>;
|
||||
using TupleOrArrayFunctionL2Norm = TupleOrArrayFunction<L2NormTraits>;
|
||||
using TupleOrArrayFunctionL2SquaredNorm = TupleOrArrayFunction<L2SquaredNormTraits>;
|
||||
@ -1615,8 +1627,8 @@ If the types of the first interval (or the interval in the tuple) and the second
|
||||
factory.registerFunction<FunctionTupleMultiplyByNumber>();
|
||||
factory.registerFunction<FunctionTupleDivideByNumber>();
|
||||
|
||||
factory.registerFunction<FunctionDotProduct>();
|
||||
factory.registerAlias("scalarProduct", FunctionDotProduct::name, FunctionFactory::CaseInsensitive);
|
||||
factory.registerFunction<TupleOrArrayFunctionDotProduct>();
|
||||
factory.registerAlias("scalarProduct", TupleOrArrayFunctionDotProduct::name, FunctionFactory::CaseInsensitive);
|
||||
|
||||
factory.registerFunction<TupleOrArrayFunctionL1Norm>();
|
||||
factory.registerFunction<TupleOrArrayFunctionL2Norm>();
|
||||
|
@ -97,6 +97,7 @@ arrayCumSum
|
||||
arrayCumSumNonNegative
|
||||
arrayDifference
|
||||
arrayDistinct
|
||||
arrayDotProduct
|
||||
arrayElement
|
||||
arrayEnumerate
|
||||
arrayEnumerateDense
|
||||
|
14
tests/queries/0_stateless/02708_dot_product.reference
Normal file
14
tests/queries/0_stateless/02708_dot_product.reference
Normal file
@ -0,0 +1,14 @@
|
||||
3881.304
|
||||
3881.304
|
||||
3881.304
|
||||
376.5
|
||||
230
|
||||
0
|
||||
0
|
||||
Float64
|
||||
Float32
|
||||
Float64
|
||||
Float64
|
||||
UInt16
|
||||
UInt64
|
||||
Int64
|
55
tests/queries/0_stateless/02708_dot_product.sql
Normal file
55
tests/queries/0_stateless/02708_dot_product.sql
Normal file
@ -0,0 +1,55 @@
|
||||
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;
|
Loading…
Reference in New Issue
Block a user