mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
dbms: added least and greatest functions [#METR-17233].
This commit is contained in:
parent
1f4f0f591b
commit
2af61829fe
@ -262,6 +262,30 @@ struct BitShiftRightImpl
|
||||
}
|
||||
};
|
||||
|
||||
template<typename A, typename B>
|
||||
struct LeastImpl
|
||||
{
|
||||
typedef typename NumberTraits::ResultOfIf<A, B>::Type ResultType;
|
||||
|
||||
template <typename Result = ResultType>
|
||||
static inline Result apply(A a, B b)
|
||||
{
|
||||
return static_cast<Result>(a) < static_cast<Result>(b) ? static_cast<Result>(a) : static_cast<Result>(b);
|
||||
}
|
||||
};
|
||||
|
||||
template<typename A, typename B>
|
||||
struct GreatestImpl
|
||||
{
|
||||
typedef typename NumberTraits::ResultOfIf<A, B>::Type ResultType;
|
||||
|
||||
template <typename Result = ResultType>
|
||||
static inline Result apply(A a, B b)
|
||||
{
|
||||
return static_cast<Result>(a) > static_cast<Result>(b) ? static_cast<Result>(a) : static_cast<Result>(b);
|
||||
}
|
||||
};
|
||||
|
||||
template<typename A>
|
||||
struct NegateImpl
|
||||
{
|
||||
@ -319,6 +343,12 @@ template <typename T> using Else = T;
|
||||
/// Used to indicate undefined operation
|
||||
struct InvalidType;
|
||||
|
||||
template <>
|
||||
struct DataTypeFromFieldType<NumberTraits::Error>
|
||||
{
|
||||
using Type = InvalidType;
|
||||
};
|
||||
|
||||
template <typename DataType> struct IsIntegral { static constexpr auto value = false; };
|
||||
template <> struct IsIntegral<DataTypeUInt8> { static constexpr auto value = true; };
|
||||
template <> struct IsIntegral<DataTypeUInt16> { static constexpr auto value = true; };
|
||||
@ -342,11 +372,13 @@ template <typename DataType> struct IsDateOrDateTime { static constexpr auto val
|
||||
template <> struct IsDateOrDateTime<DataTypeDate> { static constexpr auto value = true; };
|
||||
template <> struct IsDateOrDateTime<DataTypeDateTime> { static constexpr auto value = true; };
|
||||
|
||||
/** Returns appropriate result type for binary operator on dates:
|
||||
/** Returns appropriate result type for binary operator on dates (or datetimes):
|
||||
* Date + Integral -> Date
|
||||
* Integral + Date -> Date
|
||||
* Date - Date -> Int32
|
||||
* Date - Integral -> Date
|
||||
* least(Date, Date) -> Date
|
||||
* greatest(Date, Date) -> Date
|
||||
* All other operations are not defined and return InvalidType, operations on
|
||||
* distinct date types are also undefined (e.g. DataTypeDate - DataTypeDateTime) */
|
||||
template <template <typename, typename> class Operation, typename LeftDataType, typename RightDataType>
|
||||
@ -387,7 +419,13 @@ struct DateBinaryOperationTraits
|
||||
Else<InvalidType>
|
||||
>
|
||||
>,
|
||||
Else<InvalidType>
|
||||
Else<
|
||||
If<std::is_same<T0, T1>::value
|
||||
&& (std::is_same<Op, LeastImpl<T0, T1>>::value || std::is_same<Op, GreatestImpl<T0, T1>>::value),
|
||||
Then<LeftDataType>,
|
||||
Else<InvalidType>
|
||||
>
|
||||
>
|
||||
>
|
||||
>
|
||||
>;
|
||||
@ -479,35 +517,27 @@ private:
|
||||
}
|
||||
|
||||
/// Overload for date operations
|
||||
template <typename LeftDataType, typename RightDataType, typename ColumnType,
|
||||
typename std::enable_if<IsDateOrDateTime<LeftDataType>::value || IsDateOrDateTime<RightDataType>::value>::type * = nullptr>
|
||||
template <typename LeftDataType, typename RightDataType, typename ColumnType>
|
||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, const size_t result, const ColumnType * col_left)
|
||||
{
|
||||
if (!typeid_cast<const RightDataType *>(block.getByPosition(arguments[1]).type.get()))
|
||||
return false;
|
||||
|
||||
using ResultDataType = typename DateBinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
|
||||
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
|
||||
|
||||
return executeRightTypeDispatch<LeftDataType, RightDataType, ResultDataType>(
|
||||
block, arguments, result, col_left);
|
||||
}
|
||||
|
||||
/// Overload for numeric operations
|
||||
template <typename LeftDataType, typename RightDataType, typename ColumnType,
|
||||
typename T0 = typename LeftDataType::FieldType, typename T1 = typename RightDataType::FieldType,
|
||||
typename std::enable_if<IsNumeric<LeftDataType>::value && IsNumeric<RightDataType>::value>::type * = nullptr>
|
||||
bool executeRightType(Block & block, const ColumnNumbers & arguments, const size_t result, const ColumnType * col_left)
|
||||
{
|
||||
return executeRightTypeImpl<T0, T1>(block, arguments, result, col_left);
|
||||
}
|
||||
|
||||
/// Overload for InvalidType
|
||||
template <typename LeftDataType, typename RightDataType, typename ResultDataType, typename ColumnType,
|
||||
typename std::enable_if<std::is_same<ResultDataType, InvalidType>::value>::type * = nullptr>
|
||||
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
|
||||
const ColumnType * col_left)
|
||||
{
|
||||
return false;
|
||||
throw Exception("Types " + TypeName<typename LeftDataType::FieldType>::get()
|
||||
+ " and " + TypeName<typename LeftDataType::FieldType>::get()
|
||||
+ " are incompatible for function " + getName() + " or not upscaleable to common type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
/// Overload for well-defined operations
|
||||
@ -527,7 +557,7 @@ private:
|
||||
template <typename T0, typename T1, typename ResultType = typename Op<T0, T1>::ResultType>
|
||||
bool executeRightTypeImpl(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnVector<T0> * col_left)
|
||||
{
|
||||
if (auto col_right = typeid_cast<ColumnVector<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||
if (auto col_right = typeid_cast<const ColumnVector<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||
{
|
||||
auto col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -538,7 +568,7 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (auto col_right = typeid_cast<ColumnConst<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||
else if (auto col_right = typeid_cast<const ColumnConst<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||
{
|
||||
auto col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -550,14 +580,14 @@ private:
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
throw Exception("Logical error: unexpected type of column", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
/// ColumnConst overload
|
||||
template <typename T0, typename T1, typename ResultType = typename Op<T0, T1>::ResultType>
|
||||
bool executeRightTypeImpl(Block & block, const ColumnNumbers & arguments, size_t result, const ColumnConst<T0> * col_left)
|
||||
{
|
||||
if (auto col_right = typeid_cast<ColumnVector<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||
if (auto col_right = typeid_cast<const ColumnVector<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||
{
|
||||
auto col_res = new ColumnVector<ResultType>;
|
||||
block.getByPosition(result).column = col_res;
|
||||
@ -568,7 +598,7 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (auto col_right = typeid_cast<ColumnConst<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||
else if (auto col_right = typeid_cast<const ColumnConst<T1> *>(block.getByPosition(arguments[1]).column.get()))
|
||||
{
|
||||
ResultType res = 0;
|
||||
BinaryOperationImpl<T0, T1, Op<T0, T1>, ResultType>::constant_constant(col_left->getData(), col_right->getData(), res);
|
||||
@ -582,26 +612,12 @@ private:
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename LeftDataType,
|
||||
typename std::enable_if<IsDateOrDateTime<LeftDataType>::value>::type * = nullptr>
|
||||
template <typename LeftDataType>
|
||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
if (!typeid_cast<const LeftDataType *>(block.getByPosition(arguments[0]).type.get()))
|
||||
return false;
|
||||
|
||||
return executeLeftTypeDispatch<LeftDataType>(block, arguments, result);
|
||||
}
|
||||
|
||||
template <typename LeftDataType,
|
||||
typename std::enable_if<IsNumeric<LeftDataType>::value>::type * = nullptr>
|
||||
bool executeLeftType(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
return executeLeftTypeDispatch<LeftDataType>(block, arguments, result);
|
||||
}
|
||||
|
||||
template <typename LeftDataType>
|
||||
bool executeLeftTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
using T0 = typename LeftDataType::FieldType;
|
||||
|
||||
if ( executeLeftTypeImpl<LeftDataType, ColumnVector<T0>>(block, arguments, result)
|
||||
@ -614,7 +630,7 @@ private:
|
||||
template <typename LeftDataType, typename ColumnType>
|
||||
bool executeLeftTypeImpl(Block & block, const ColumnNumbers & arguments, const size_t result)
|
||||
{
|
||||
if (auto col_left = typeid_cast<ColumnType *>(block.getByPosition(arguments[0]).column.get()))
|
||||
if (auto col_left = typeid_cast<const ColumnType *>(block.getByPosition(arguments[0]).column.get()))
|
||||
{
|
||||
if ( executeRightType<LeftDataType, DataTypeDate>(block, arguments, result, col_left)
|
||||
|| executeRightType<LeftDataType, DataTypeDateTime>(block, arguments, result, col_left)
|
||||
@ -718,7 +734,7 @@ private:
|
||||
template <typename T0>
|
||||
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (ColumnVector<T0> * col = typeid_cast<ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
if (const ColumnVector<T0> * col = typeid_cast<const ColumnVector<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
typedef typename Op<T0>::ResultType ResultType;
|
||||
|
||||
@ -731,7 +747,7 @@ private:
|
||||
|
||||
return true;
|
||||
}
|
||||
else if (ColumnConst<T0> * col = typeid_cast<ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
else if (const ColumnConst<T0> * col = typeid_cast<const ColumnConst<T0> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
typedef typename Op<T0>::ResultType ResultType;
|
||||
|
||||
@ -815,6 +831,8 @@ struct NameBitXor { static constexpr auto name = "bitXor"; };
|
||||
struct NameBitNot { static constexpr auto name = "bitNot"; };
|
||||
struct NameBitShiftLeft { static constexpr auto name = "bitShiftLeft"; };
|
||||
struct NameBitShiftRight { static constexpr auto name = "bitShiftRight"; };
|
||||
struct NameLeast { static constexpr auto name = "least"; };
|
||||
struct NameGreatest { static constexpr auto name = "greatest"; };
|
||||
|
||||
typedef FunctionBinaryArithmetic<PlusImpl, NamePlus> FunctionPlus;
|
||||
typedef FunctionBinaryArithmetic<MinusImpl, NameMinus> FunctionMinus;
|
||||
@ -831,7 +849,8 @@ typedef FunctionBinaryArithmetic<BitXorImpl, NameBitXor> FunctionBitXor;
|
||||
typedef FunctionUnaryArithmetic<BitNotImpl, NameBitNot> FunctionBitNot;
|
||||
typedef FunctionBinaryArithmetic<BitShiftLeftImpl, NameBitShiftLeft> FunctionBitShiftLeft;
|
||||
typedef FunctionBinaryArithmetic<BitShiftRightImpl, NameBitShiftRight> FunctionBitShiftRight;
|
||||
|
||||
typedef FunctionBinaryArithmetic<LeastImpl, NameLeast> FunctionLeast;
|
||||
typedef FunctionBinaryArithmetic<GreatestImpl, NameGreatest> FunctionGreatest;
|
||||
|
||||
|
||||
/// Оптимизации для целочисленного деления на константу.
|
||||
|
@ -725,7 +725,11 @@ public:
|
||||
column_first_array = column_array;
|
||||
}
|
||||
|
||||
temp_block.insert(ColumnWithNameAndType(column_array->getDataPtr(), argument_type, argument_name));
|
||||
temp_block.insert(ColumnWithNameAndType(
|
||||
column_array->getDataPtr(),
|
||||
argument_type,
|
||||
argument_name));
|
||||
|
||||
argument_names.insert(argument_name);
|
||||
}
|
||||
|
||||
@ -745,6 +749,7 @@ public:
|
||||
|
||||
replicated_column.name = name;
|
||||
replicated_column.column = typeid_cast<ColumnArray &>(*replicated_column.column).getDataPtr();
|
||||
replicated_column.type = typeid_cast<const DataTypeArray &>(*replicated_column.type).getNestedType(),
|
||||
temp_block.insert(replicated_column);
|
||||
|
||||
++prerequisite_index;
|
||||
|
@ -21,6 +21,8 @@ void registerFunctionsArithmetic(FunctionFactory & factory)
|
||||
factory.registerFunction<FunctionBitNot>();
|
||||
factory.registerFunction<FunctionBitShiftLeft>();
|
||||
factory.registerFunction<FunctionBitShiftRight>();
|
||||
factory.registerFunction<FunctionLeast>();
|
||||
factory.registerFunction<FunctionGreatest>();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,12 @@
|
||||
1 2 1 2 1 2 2 UInt8
|
||||
1.1 2 1.1 2 1.1 2 2 Float64
|
||||
-1 2 -1 2 -1 2 2 Int16
|
||||
1 2 1 2 1 2 2 Float64
|
||||
1 2000 1 2000 1 2000 2000 UInt16
|
||||
1 200000 1 200000 1 200000 200000 UInt32
|
||||
1 20000000000 1 20000000000 1 20000000000 20000000000 UInt64
|
||||
123 123 123 123 123 123 123 UInt8
|
||||
2010-01-02 2011-02-03 2010-01-02 2011-02-03 2010-01-02 2011-02-03 2011-02-03 Date
|
||||
2010-01-02 03:04:05 2011-02-03 04:05:06 2010-01-02 03:04:05 2011-02-03 04:05:06 2010-01-02 03:04:05 2011-02-03 04:05:06 2011-02-03 04:05:06 DateTime
|
||||
10
|
||||
9
|
12
dbms/tests/queries/0_stateless/00192_least_greatest.sql
Normal file
12
dbms/tests/queries/0_stateless/00192_least_greatest.sql
Normal file
@ -0,0 +1,12 @@
|
||||
SELECT 1 AS x, 2 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT 1.1 AS x, 2 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT -1 AS x, 2 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT 1.0 AS x, 2.0 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT 1 AS x, 2000 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT 1 AS x, 200000 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT 1 AS x, 20000000000 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT 123 AS x, 123 AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT toDate('2010-01-02') AS x, toDate('2011-02-03') AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT toDateTime('2010-01-02 03:04:05') AS x, toDateTime('2011-02-03 04:05:06') AS y, least(x, y), greatest(x, y), least(x, materialize(y)), greatest(materialize(x), y), greatest(materialize(x), materialize(y)), toTypeName(least(x, y));
|
||||
SELECT greatest(now(), now() + 10) - now();
|
||||
SELECT greatest(today(), yesterday() + 10) - today();
|
Loading…
Reference in New Issue
Block a user