mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
use return type as matrix value type
This commit is contained in:
parent
2865c8141d
commit
a17da05bda
@ -1,30 +1,31 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
|
||||
#include <Eigen/Core>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
template <const int N>
|
||||
struct LpDistance
|
||||
{
|
||||
static inline String name = "L" + std::to_string(N);
|
||||
template <typename L, typename R, typename T>
|
||||
static void compute(const Eigen::MatrixBase<L> & left, const Eigen::MatrixBase<R> & right, PaddedPODArray<T> & array)
|
||||
template <typename T>
|
||||
static void compute(const Eigen::MatrixX<T> & left, const Eigen::MatrixX<T> & right, PaddedPODArray<T> & array)
|
||||
{
|
||||
auto & norms = (left - right).colwise().template lpNorm<N>();
|
||||
// array.insert() failed to work with Eigen iterators
|
||||
for (auto n : norms)
|
||||
array.emplace_back(n);
|
||||
array.push_back(n);
|
||||
}
|
||||
};
|
||||
|
||||
@ -36,19 +37,16 @@ struct LinfDistance : LpDistance<Eigen::Infinity>
|
||||
struct CosineDistance
|
||||
{
|
||||
static inline String name = "Cosine";
|
||||
template <typename L, typename R, typename T>
|
||||
static void compute(const Eigen::MatrixBase<L> & left, const Eigen::MatrixBase<R> & right, PaddedPODArray<T> & array)
|
||||
template <typename T>
|
||||
static void compute(const Eigen::MatrixX<T> & left, const Eigen::MatrixX<T> & right, PaddedPODArray<T> & array)
|
||||
{
|
||||
// auto & nx = left.colwise().normalized().eval();
|
||||
// auto & ny = right.colwise().normalized().eval();
|
||||
// auto & dist = 1.0 - x.cwiseProduct(y).colwise().sum().array();
|
||||
auto & prod = left.cwiseProduct(right).colwise().sum();
|
||||
auto & nx = left.colwise().norm();
|
||||
auto & ny = right.colwise().norm();
|
||||
auto & nm = nx.cwiseProduct(ny).cwiseInverse();
|
||||
auto & dist = 1.0 - prod.cwiseProduct(nm).array();
|
||||
for (auto d : dist)
|
||||
array.emplace_back(d);
|
||||
array.push_back(d);
|
||||
}
|
||||
};
|
||||
|
||||
@ -63,30 +61,18 @@ public:
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & /*arguments*/) const override
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
return std::make_shared<DataTypeFloat32>();
|
||||
}
|
||||
|
||||
ColumnPtr
|
||||
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto & left = arguments[0];
|
||||
const auto & right = arguments[1];
|
||||
|
||||
const auto & col_x = left.column->convertToFullColumnIfConst();
|
||||
const auto & col_y = right.column->convertToFullColumnIfConst();
|
||||
const auto * arr_x = checkAndGetColumn<ColumnArray>(col_x.get());
|
||||
const auto * arr_y = checkAndGetColumn<ColumnArray>(col_y.get());
|
||||
if (!arr_x || !arr_y)
|
||||
DataTypes types;
|
||||
for (const auto & argument : arguments)
|
||||
{
|
||||
throw Exception("Argument of function " + String(name) + " must be array. ", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
const auto * array_type = checkAndGetDataType<DataTypeArray>(argument.type.get());
|
||||
if (!array_type)
|
||||
throw Exception("Argument of function " + getName() + " must be array. ", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
types.push_back(array_type->getNestedType());
|
||||
}
|
||||
|
||||
const auto & type_x = checkAndGetDataType<DataTypeArray>(left.type.get())->getNestedType();
|
||||
const auto & type_y = checkAndGetDataType<DataTypeArray>(right.type.get())->getNestedType();
|
||||
|
||||
const auto & common_type = getLeastSupertype(DataTypes{type_x, type_y});
|
||||
const auto & common_type = getLeastSupertype(types);
|
||||
switch (common_type->getTypeId())
|
||||
{
|
||||
case TypeIndex::UInt8:
|
||||
@ -96,31 +82,55 @@ public:
|
||||
case TypeIndex::Int16:
|
||||
case TypeIndex::Int32:
|
||||
case TypeIndex::Float32:
|
||||
return executeWithType<float>(*arr_x, *arr_y, type_x, type_y, result_type);
|
||||
return std::make_shared<DataTypeFloat32>();
|
||||
case TypeIndex::UInt64:
|
||||
case TypeIndex::Int64:
|
||||
case TypeIndex::Float64:
|
||||
return executeWithType<double>(*arr_x, *arr_y, type_x, type_y, result_type);
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
default:
|
||||
throw Exception(
|
||||
"Array nested type " + common_type->getName()
|
||||
"Arguments of function " + getName() + " has nested type " + common_type->getName()
|
||||
+ ". Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr
|
||||
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
||||
{
|
||||
const auto & type_x = typeid_cast<const DataTypeArray *>(arguments[0].type.get())->getNestedType();
|
||||
const auto & type_y = typeid_cast<const DataTypeArray *>(arguments[1].type.get())->getNestedType();
|
||||
|
||||
const auto & col_x = arguments[0].column->convertToFullColumnIfConst();
|
||||
const auto & col_y = arguments[1].column->convertToFullColumnIfConst();
|
||||
|
||||
const auto * arr_x = assert_cast<const ColumnArray *>(col_x.get());
|
||||
const auto * arr_y = assert_cast<const ColumnArray *>(col_y.get());
|
||||
|
||||
auto result = result_type->createColumn();
|
||||
switch (result_type->getTypeId())
|
||||
{
|
||||
case TypeIndex::Float32:
|
||||
executeWithType<Float32>(*arr_x, *arr_y, type_x, type_y, result);
|
||||
break;
|
||||
case TypeIndex::Float64:
|
||||
executeWithType<Float64>(*arr_x, *arr_y, type_x, type_y, result);
|
||||
break;
|
||||
default:
|
||||
throw Exception("Unexpected result type.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename MatrixType>
|
||||
static ColumnPtr executeWithType(
|
||||
void executeWithType(
|
||||
const ColumnArray & array_x,
|
||||
const ColumnArray & array_y,
|
||||
const DataTypePtr & type_x,
|
||||
const DataTypePtr & type_y,
|
||||
const DataTypePtr & result_type)
|
||||
MutableColumnPtr & column) const
|
||||
{
|
||||
auto result = result_type->createColumn();
|
||||
auto & array = typeid_cast<ColumnFloat32 &>(*result).getData();
|
||||
|
||||
Eigen::MatrixX<MatrixType> mx, my;
|
||||
columnToMatrix(array_x, type_x, mx);
|
||||
columnToMatrix(array_y, type_y, my);
|
||||
@ -128,15 +138,14 @@ private:
|
||||
if (mx.rows() && my.rows() && mx.rows() != my.rows())
|
||||
{
|
||||
throw Exception(
|
||||
"Arguments of function " + String(name) + " have different array sizes.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
"Arguments of function " + getName() + " have different array sizes.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
Kernel::compute(mx, my, array);
|
||||
return result;
|
||||
auto & data = assert_cast<ColumnVector<MatrixType> &>(*column).getData();
|
||||
Kernel::compute(mx, my, data);
|
||||
}
|
||||
|
||||
template <typename MatrixType>
|
||||
static void columnToMatrix(const ColumnArray & array, const DataTypePtr & nested_type, Eigen::MatrixX<MatrixType> & mat)
|
||||
void columnToMatrix(const ColumnArray & array, const DataTypePtr & nested_type, Eigen::MatrixX<MatrixType> & mat) const
|
||||
{
|
||||
switch (nested_type->getTypeId())
|
||||
{
|
||||
@ -172,7 +181,7 @@ private:
|
||||
break;
|
||||
default:
|
||||
throw Exception(
|
||||
"Array nested type " + nested_type->getName()
|
||||
"Arguments of function " + getName() + " has nested type " + nested_type->getName()
|
||||
+ ". Support: UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -180,25 +189,28 @@ private:
|
||||
|
||||
// optimize for float/ double
|
||||
template <typename MatrixType, typename DataType, typename std::enable_if<std::is_same<MatrixType, DataType>::value>::type>
|
||||
static void fillMatrix(Eigen::MatrixX<MatrixType> & mat, const ColumnArray & array)
|
||||
void fillMatrix(Eigen::MatrixX<MatrixType> & mat, const ColumnArray & array) const
|
||||
{
|
||||
const auto & vec = typeid_cast<const ColumnVector<DataType> &>(array.getData());
|
||||
const auto & data = vec.getData();
|
||||
const auto & data = typeid_cast<const ColumnVector<DataType> &>(array.getData()).getData();
|
||||
const auto & offsets = array.getOffsets();
|
||||
mat = Eigen::Map<const Eigen::MatrixX<MatrixType>>(data.data(), offsets.front(), offsets.size());
|
||||
}
|
||||
|
||||
template <typename MatrixType, typename DataType>
|
||||
static void fillMatrix(Eigen::MatrixX<MatrixType> & mat, const ColumnArray & array)
|
||||
void fillMatrix(Eigen::MatrixX<MatrixType> & mat, const ColumnArray & array) const
|
||||
{
|
||||
const auto & vec = typeid_cast<const ColumnVector<DataType> &>(array.getData());
|
||||
const auto & data = vec.getData();
|
||||
const auto & data = typeid_cast<const ColumnVector<DataType> &>(array.getData()).getData();
|
||||
const auto & offsets = array.getOffsets();
|
||||
mat.resize(offsets.front(), offsets.size());
|
||||
size_t rows = offsets.front(), cols = offsets.size();
|
||||
mat.resize(rows, cols);
|
||||
|
||||
ColumnArray::Offset prev = 0, col = 0;
|
||||
for (auto off : offsets)
|
||||
for (ColumnArray::Offset off : offsets)
|
||||
{
|
||||
if (off - prev != rows)
|
||||
throw Exception(
|
||||
"Arguments of function " + getName() + " have different array sizes.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
|
||||
|
||||
for (ColumnArray::Offset row = 0; row < off - prev; ++row)
|
||||
{
|
||||
mat(row, col) = static_cast<MatrixType>(data[prev + row]);
|
||||
|
@ -19,12 +19,12 @@ nan
|
||||
6
|
||||
8
|
||||
9
|
||||
0.020204103
|
||||
0.118082896
|
||||
0.020204102886728692
|
||||
0.11808289631180302
|
||||
0
|
||||
1 1 218.74643
|
||||
1 2 1348.2118
|
||||
2 1 219.28064
|
||||
2 2 1347.4009
|
||||
3 1 214.35251
|
||||
3 2 1342.8857
|
||||
1 1 218.74642854227358
|
||||
1 2 1348.2117786164013
|
||||
2 1 219.28064210048274
|
||||
2 2 1347.4008312302617
|
||||
3 1 214.35251339790725
|
||||
3 2 1342.8856987845243
|
||||
|
@ -22,6 +22,9 @@ SELECT arrayCosineDistance(v, materialize([1., 1., 1.])) FROM vec1;
|
||||
INSERT INTO vec2 VALUES (1, [100, 200, 0]), (2, [888, 777, 666]);
|
||||
SELECT v1.id, v2.id, arrayL2Distance(v1.v, v2.v) as dist FROM vec1 v1, vec2 v2;
|
||||
|
||||
INSERT INTO vec2 VALUES (3, [123]);
|
||||
SELECT v1.id, v2.id, arrayL2Distance(v1.v, v2.v) as dist FROM vec1 v1, vec2 v2; -- { serverError 190 }
|
||||
|
||||
SELECT arrayL1Distance([0, 0], [1]); -- { serverError 190 }
|
||||
SELECT arrayL2Distance((1, 2), (3,4)); -- { serverError 43 }
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user