Merge pull request #49050 from FFFFFFFHHHHHHH/dot_product

Add Function dotProduct for array
This commit is contained in:
Alexey Milovidov 2023-05-20 03:07:13 +03:00 committed by GitHub
commit 4e3188126f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 222 additions and 31 deletions

View File

@ -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,

View 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_); }
}

View File

@ -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());
}
}
};
}

View File

@ -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>();

View File

@ -97,6 +97,7 @@ arrayCumSum
arrayCumSumNonNegative
arrayDifference
arrayDistinct
arrayDotProduct
arrayElement
arrayEnumerate
arrayEnumerateDense

View File

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

View 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;