2017-04-01 09:19:00 +00:00
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
#include <Functions/FunctionsLogical.h>
|
2014-08-22 00:57:20 +00:00
|
|
|
|
2019-07-16 20:57:11 +00:00
|
|
|
#include <Columns/IColumn.h>
|
|
|
|
#include <Columns/ColumnVector.h>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <Columns/ColumnConst.h>
|
|
|
|
#include <Columns/ColumnNullable.h>
|
|
|
|
#include <Common/FieldVisitors.h>
|
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <Functions/FunctionHelpers.h>
|
|
|
|
|
|
|
|
#include <algorithm>
|
|
|
|
|
|
|
|
|
2014-08-22 00:57:20 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
void registerFunctionsLogical(FunctionFactory & factory)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
factory.registerFunction<FunctionAnd>();
|
|
|
|
factory.registerFunction<FunctionOr>();
|
|
|
|
factory.registerFunction<FunctionXor>();
|
|
|
|
factory.registerFunction<FunctionNot>();
|
2014-08-22 00:57:20 +00:00
|
|
|
}
|
|
|
|
|
2019-07-16 20:57:11 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int LOGICAL_ERROR;
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
|
|
|
extern const int ILLEGAL_COLUMN;
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
using namespace FunctionsLogicalDetail;
|
|
|
|
|
|
|
|
using UInt8Container = ColumnUInt8::Container;
|
|
|
|
using UInt8ColumnPtrs = std::vector<const ColumnUInt8 *>;
|
|
|
|
|
|
|
|
|
|
|
|
MutableColumnPtr convertFromTernaryData(const UInt8Container & ternary_data, const bool make_nullable)
|
|
|
|
{
|
|
|
|
const size_t rows_count = ternary_data.size();
|
|
|
|
|
|
|
|
auto new_column = ColumnUInt8::create(rows_count);
|
|
|
|
std::transform(
|
|
|
|
ternary_data.cbegin(), ternary_data.cend(), new_column->getData().begin(),
|
|
|
|
[](const auto x) { return x == Ternary::True; });
|
|
|
|
|
|
|
|
if (!make_nullable)
|
|
|
|
return new_column;
|
|
|
|
|
|
|
|
auto null_column = ColumnUInt8::create(rows_count);
|
|
|
|
std::transform(
|
|
|
|
ternary_data.cbegin(), ternary_data.cend(), null_column->getData().begin(),
|
|
|
|
[](const auto x) { return x == Ternary::Null; });
|
|
|
|
|
|
|
|
return ColumnNullable::create(std::move(new_column), std::move(null_column));
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename T>
|
|
|
|
bool tryConvertColumnToUInt8(const IColumn * column, UInt8Container & res)
|
|
|
|
{
|
|
|
|
const auto col = checkAndGetColumn<ColumnVector<T>>(column);
|
|
|
|
if (!col)
|
|
|
|
return false;
|
|
|
|
|
|
|
|
std::transform(
|
|
|
|
col->getData().cbegin(), col->getData().cend(), res.begin(),
|
|
|
|
[](const auto x) { return x != 0; });
|
|
|
|
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
void convertColumnToUInt8(const IColumn * column, UInt8Container & res)
|
|
|
|
{
|
|
|
|
if (!tryConvertColumnToUInt8<Int8>(column, res) &&
|
|
|
|
!tryConvertColumnToUInt8<Int16>(column, res) &&
|
|
|
|
!tryConvertColumnToUInt8<Int32>(column, res) &&
|
|
|
|
!tryConvertColumnToUInt8<Int64>(column, res) &&
|
|
|
|
!tryConvertColumnToUInt8<UInt16>(column, res) &&
|
|
|
|
!tryConvertColumnToUInt8<UInt32>(column, res) &&
|
|
|
|
!tryConvertColumnToUInt8<UInt64>(column, res) &&
|
|
|
|
!tryConvertColumnToUInt8<Float32>(column, res) &&
|
|
|
|
!tryConvertColumnToUInt8<Float64>(column, res))
|
|
|
|
throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
template <class Op, typename Func>
|
|
|
|
static bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res, Func && func)
|
2019-07-16 21:06:23 +00:00
|
|
|
{
|
2019-07-17 16:10:37 +00:00
|
|
|
bool has_res = false;
|
2019-07-22 22:39:42 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
for (int i = static_cast<int>(in.size()) - 1; i >= 0; --i)
|
2019-07-16 21:06:23 +00:00
|
|
|
{
|
2019-07-22 22:39:42 +00:00
|
|
|
if (!isColumnConst(*in[i]))
|
2019-07-17 16:10:37 +00:00
|
|
|
continue;
|
|
|
|
|
|
|
|
UInt8 x = func((*in[i])[0]);
|
|
|
|
if (has_res)
|
2019-07-16 21:06:23 +00:00
|
|
|
{
|
2019-07-17 16:10:37 +00:00
|
|
|
res = Op::apply(res, x);
|
2019-07-16 21:06:23 +00:00
|
|
|
}
|
|
|
|
else
|
2019-07-17 16:10:37 +00:00
|
|
|
{
|
|
|
|
res = x;
|
|
|
|
has_res = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
in.erase(in.begin() + i);
|
2019-07-16 21:06:23 +00:00
|
|
|
}
|
2019-07-22 22:39:42 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
return has_res;
|
|
|
|
}
|
2019-07-16 21:06:23 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
template <class Op>
|
|
|
|
inline bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res)
|
2019-07-16 21:06:23 +00:00
|
|
|
{
|
2019-07-17 16:10:37 +00:00
|
|
|
return extractConstColumns<Op>(
|
|
|
|
in, res,
|
|
|
|
[](const Field & value)
|
|
|
|
{
|
|
|
|
return !value.isNull() && applyVisitor(FieldVisitorConvertToNumber<bool>(), value);
|
|
|
|
}
|
|
|
|
);
|
|
|
|
}
|
2019-07-16 21:06:23 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
template <class Op>
|
|
|
|
inline bool extractConstColumnsTernary(ColumnRawPtrs & in, UInt8 & res_3v)
|
|
|
|
{
|
|
|
|
return extractConstColumns<Op>(
|
|
|
|
in, res_3v,
|
|
|
|
[](const Field & value)
|
|
|
|
{
|
|
|
|
return value.isNull()
|
|
|
|
? Ternary::makeValue(false, true)
|
|
|
|
: Ternary::makeValue(applyVisitor(FieldVisitorConvertToNumber<bool>(), value));
|
|
|
|
}
|
|
|
|
);
|
|
|
|
}
|
2019-07-16 21:06:23 +00:00
|
|
|
|
|
|
|
|
2019-07-16 20:57:11 +00:00
|
|
|
template <typename Op, size_t N>
|
|
|
|
class AssociativeApplierImpl
|
|
|
|
{
|
|
|
|
using ResultValueType = typename Op::ResultType;
|
|
|
|
|
|
|
|
public:
|
|
|
|
/// Remembers the last N columns from `in`.
|
2020-03-18 03:27:32 +00:00
|
|
|
explicit AssociativeApplierImpl(const UInt8ColumnPtrs & in)
|
2019-07-16 20:57:11 +00:00
|
|
|
: vec(in[in.size() - N]->getData()), next(in) {}
|
|
|
|
|
|
|
|
/// Returns a combination of values in the i-th row of all columns stored in the constructor.
|
2019-07-17 16:10:37 +00:00
|
|
|
inline ResultValueType apply(const size_t i) const
|
2019-07-16 20:57:11 +00:00
|
|
|
{
|
2019-07-17 16:10:37 +00:00
|
|
|
const auto & a = vec[i];
|
2019-07-16 20:57:11 +00:00
|
|
|
if constexpr (Op::isSaturable())
|
2019-07-17 16:10:37 +00:00
|
|
|
return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(i));
|
2019-07-16 20:57:11 +00:00
|
|
|
else
|
2019-07-17 16:10:37 +00:00
|
|
|
return Op::apply(a, next.apply(i));
|
2019-07-16 20:57:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
|
|
|
const UInt8Container & vec;
|
|
|
|
const AssociativeApplierImpl<Op, N - 1> next;
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename Op>
|
|
|
|
class AssociativeApplierImpl<Op, 1>
|
|
|
|
{
|
|
|
|
using ResultValueType = typename Op::ResultType;
|
|
|
|
|
|
|
|
public:
|
2020-03-18 03:27:32 +00:00
|
|
|
explicit AssociativeApplierImpl(const UInt8ColumnPtrs & in)
|
2019-07-16 20:57:11 +00:00
|
|
|
: vec(in[in.size() - 1]->getData()) {}
|
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
inline ResultValueType apply(const size_t i) const { return vec[i]; }
|
2019-07-16 20:57:11 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
const UInt8Container & vec;
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
/// A helper class used by AssociativeGenericApplierImpl
|
|
|
|
/// Allows for on-the-fly conversion of any data type into intermediate ternary representation
|
|
|
|
using ValueGetter = std::function<Ternary::ResultType (size_t)>;
|
2019-07-16 20:57:11 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
template <typename ... Types>
|
|
|
|
struct ValueGetterBuilderImpl;
|
2019-07-16 20:57:11 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
template <typename Type, typename ...Types>
|
|
|
|
struct ValueGetterBuilderImpl<Type, Types...>
|
|
|
|
{
|
|
|
|
static ValueGetter build(const IColumn * x)
|
|
|
|
{
|
|
|
|
if (const auto nullable_column = typeid_cast<const ColumnNullable *>(x))
|
2019-07-16 20:57:11 +00:00
|
|
|
{
|
2019-07-17 16:10:37 +00:00
|
|
|
if (const auto nested_column = typeid_cast<const ColumnVector<Type> *>(nullable_column->getNestedColumnPtr().get()))
|
|
|
|
{
|
|
|
|
return [&null_data = nullable_column->getNullMapData(), &column_data = nested_column->getData()](size_t i)
|
|
|
|
{ return Ternary::makeValue(column_data[i], null_data[i]); };
|
|
|
|
}
|
|
|
|
else
|
|
|
|
return ValueGetterBuilderImpl<Types...>::build(x);
|
2019-07-16 20:57:11 +00:00
|
|
|
}
|
2019-07-17 16:10:37 +00:00
|
|
|
else if (const auto column = typeid_cast<const ColumnVector<Type> *>(x))
|
|
|
|
return [&column_data = column->getData()](size_t i) { return Ternary::makeValue(column_data[i]); };
|
2019-07-16 20:57:11 +00:00
|
|
|
else
|
2019-07-17 16:10:37 +00:00
|
|
|
return ValueGetterBuilderImpl<Types...>::build(x);
|
|
|
|
}
|
|
|
|
};
|
2019-07-16 20:57:11 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
template <>
|
|
|
|
struct ValueGetterBuilderImpl<>
|
|
|
|
{
|
|
|
|
static ValueGetter build(const IColumn * x)
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
std::string("Unknown numeric column of type: ") + demangle(typeid(x).name()),
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
2019-07-16 20:57:11 +00:00
|
|
|
}
|
2019-07-17 16:10:37 +00:00
|
|
|
};
|
2019-07-16 20:57:11 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
using ValueGetterBuilder =
|
|
|
|
ValueGetterBuilderImpl<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
|
2019-07-16 20:57:11 +00:00
|
|
|
|
2019-07-17 16:10:37 +00:00
|
|
|
/// This class together with helper class ValueGetterBuilder can be used with columns of arbitrary data type
|
|
|
|
/// Allows for on-the-fly conversion of any type of data into intermediate ternary representation
|
|
|
|
/// and eliminates the need to materialize data columns in intermediate representation
|
2019-07-16 20:57:11 +00:00
|
|
|
template <typename Op, size_t N>
|
|
|
|
class AssociativeGenericApplierImpl
|
|
|
|
{
|
|
|
|
using ResultValueType = typename Op::ResultType;
|
|
|
|
|
|
|
|
public:
|
|
|
|
/// Remembers the last N columns from `in`.
|
2020-03-18 03:27:32 +00:00
|
|
|
explicit AssociativeGenericApplierImpl(const ColumnRawPtrs & in)
|
2019-07-16 20:57:11 +00:00
|
|
|
: val_getter{ValueGetterBuilder::build(in[in.size() - N])}, next{in} {}
|
|
|
|
|
|
|
|
/// Returns a combination of values in the i-th row of all columns stored in the constructor.
|
2019-07-17 16:10:37 +00:00
|
|
|
inline ResultValueType apply(const size_t i) const
|
2019-07-16 20:57:11 +00:00
|
|
|
{
|
2019-07-17 16:10:37 +00:00
|
|
|
const auto a = val_getter(i);
|
2019-07-16 20:57:11 +00:00
|
|
|
if constexpr (Op::isSaturable())
|
|
|
|
return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(i));
|
|
|
|
else
|
|
|
|
return Op::apply(a, next.apply(i));
|
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
2019-07-16 21:06:23 +00:00
|
|
|
const ValueGetter val_getter;
|
2019-07-16 20:57:11 +00:00
|
|
|
const AssociativeGenericApplierImpl<Op, N - 1> next;
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
template <typename Op>
|
|
|
|
class AssociativeGenericApplierImpl<Op, 1>
|
|
|
|
{
|
|
|
|
using ResultValueType = typename Op::ResultType;
|
|
|
|
|
|
|
|
public:
|
|
|
|
/// Remembers the last N columns from `in`.
|
2020-03-18 03:27:32 +00:00
|
|
|
explicit AssociativeGenericApplierImpl(const ColumnRawPtrs & in)
|
2019-07-16 20:57:11 +00:00
|
|
|
: val_getter{ValueGetterBuilder::build(in[in.size() - 1])} {}
|
|
|
|
|
|
|
|
inline ResultValueType apply(const size_t i) const { return val_getter(i); }
|
|
|
|
|
|
|
|
private:
|
2019-07-16 21:06:23 +00:00
|
|
|
const ValueGetter val_getter;
|
2019-07-16 20:57:11 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2019-07-18 09:09:29 +00:00
|
|
|
/// Apply target function by feeding it "batches" of N columns
|
2019-07-18 08:07:24 +00:00
|
|
|
/// Combining 10 columns per pass is the fastest for large block sizes.
|
|
|
|
/// For small block sizes - more columns is faster.
|
2019-07-16 20:57:11 +00:00
|
|
|
template <
|
2019-07-18 08:07:24 +00:00
|
|
|
typename Op, template <typename, size_t> typename OperationApplierImpl, size_t N = 10>
|
2019-07-16 20:57:11 +00:00
|
|
|
struct OperationApplier
|
|
|
|
{
|
2020-01-18 21:02:43 +00:00
|
|
|
template <typename Columns, typename ResultData>
|
2020-01-19 06:07:30 +00:00
|
|
|
static void apply(Columns & in, ResultData & result_data, bool use_result_data_as_input = false)
|
2019-07-18 08:07:24 +00:00
|
|
|
{
|
2020-01-19 06:07:30 +00:00
|
|
|
if (!use_result_data_as_input)
|
|
|
|
doBatchedApply<false>(in, result_data);
|
2020-03-09 03:38:43 +00:00
|
|
|
while (!in.empty())
|
2020-01-18 21:02:43 +00:00
|
|
|
doBatchedApply<true>(in, result_data);
|
2019-07-18 08:07:24 +00:00
|
|
|
}
|
|
|
|
|
2020-01-19 06:07:30 +00:00
|
|
|
template <bool CarryResult, typename Columns, typename ResultData>
|
2019-07-18 09:09:29 +00:00
|
|
|
static void NO_INLINE doBatchedApply(Columns & in, ResultData & result_data)
|
2019-07-16 20:57:11 +00:00
|
|
|
{
|
|
|
|
if (N > in.size())
|
|
|
|
{
|
2020-01-18 21:02:43 +00:00
|
|
|
OperationApplier<Op, OperationApplierImpl, N - 1>
|
2020-01-19 06:07:30 +00:00
|
|
|
::template doBatchedApply<CarryResult>(in, result_data);
|
2019-07-16 20:57:11 +00:00
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2020-03-23 02:12:31 +00:00
|
|
|
const OperationApplierImpl<Op, N> operation_applier_impl(in);
|
2019-07-16 20:57:11 +00:00
|
|
|
size_t i = 0;
|
2019-07-18 08:07:24 +00:00
|
|
|
for (auto & res : result_data)
|
2020-01-19 06:07:30 +00:00
|
|
|
if constexpr (CarryResult)
|
2020-03-23 02:12:31 +00:00
|
|
|
res = Op::apply(res, operation_applier_impl.apply(i++));
|
2020-01-18 21:02:43 +00:00
|
|
|
else
|
2020-03-23 02:12:31 +00:00
|
|
|
res = operation_applier_impl.apply(i++);
|
2019-07-16 20:57:11 +00:00
|
|
|
|
|
|
|
in.erase(in.end() - N, in.end());
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <
|
2019-07-18 08:07:24 +00:00
|
|
|
typename Op, template <typename, size_t> typename OperationApplierImpl>
|
2020-01-18 21:02:43 +00:00
|
|
|
struct OperationApplier<Op, OperationApplierImpl, 0>
|
2019-07-16 20:57:11 +00:00
|
|
|
{
|
2020-01-18 21:02:43 +00:00
|
|
|
template <bool, typename Columns, typename Result>
|
2019-07-18 09:09:29 +00:00
|
|
|
static void NO_INLINE doBatchedApply(Columns &, Result &)
|
2019-07-16 20:57:11 +00:00
|
|
|
{
|
|
|
|
throw Exception(
|
2019-07-18 09:09:29 +00:00
|
|
|
"OperationApplier<...>::apply(...): not enough arguments to run this method",
|
2019-07-16 20:57:11 +00:00
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
template <class Op>
|
|
|
|
static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & result_info, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
/// Combine all constant columns into a single constant value.
|
|
|
|
UInt8 const_3v_value = 0;
|
|
|
|
const bool has_consts = extractConstColumnsTernary<Op>(arguments, const_3v_value);
|
|
|
|
|
|
|
|
/// If the constant value uniquely determines the result, return it.
|
2020-01-18 21:03:45 +00:00
|
|
|
if (has_consts && (arguments.empty() || Op::isSaturatedValue(const_3v_value)))
|
2019-07-16 20:57:11 +00:00
|
|
|
{
|
|
|
|
result_info.column = ColumnConst::create(
|
|
|
|
convertFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()),
|
|
|
|
input_rows_count
|
|
|
|
);
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2020-01-19 06:22:01 +00:00
|
|
|
const auto result_column = has_consts ?
|
|
|
|
ColumnUInt8::create(input_rows_count, const_3v_value) : ColumnUInt8::create(input_rows_count);
|
2019-07-16 20:57:11 +00:00
|
|
|
|
2020-01-19 06:22:01 +00:00
|
|
|
OperationApplier<Op, AssociativeGenericApplierImpl>::apply(arguments, result_column->getData(), has_consts);
|
2019-07-16 20:57:11 +00:00
|
|
|
|
|
|
|
result_info.column = convertFromTernaryData(result_column->getData(), result_info.type->isNullable());
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-07-18 08:07:24 +00:00
|
|
|
template <typename Op, typename ... Types>
|
|
|
|
struct TypedExecutorInvoker;
|
|
|
|
|
|
|
|
template <typename Op>
|
2019-07-18 09:09:29 +00:00
|
|
|
using FastApplierImpl =
|
2019-07-18 08:07:24 +00:00
|
|
|
TypedExecutorInvoker<Op, UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
|
|
|
|
|
|
|
|
template <typename Op, typename Type, typename ... Types>
|
|
|
|
struct TypedExecutorInvoker<Op, Type, Types ...>
|
|
|
|
{
|
|
|
|
template <typename T, typename Result>
|
2019-07-18 09:09:29 +00:00
|
|
|
static void apply(const ColumnVector<T> & x, const IColumn & y, Result & result)
|
2019-07-18 08:07:24 +00:00
|
|
|
{
|
|
|
|
if (const auto column = typeid_cast<const ColumnVector<Type> *>(&y))
|
|
|
|
std::transform(
|
2019-07-18 09:09:29 +00:00
|
|
|
x.getData().cbegin(), x.getData().cend(),
|
|
|
|
column->getData().cbegin(), result.begin(),
|
2019-07-22 13:56:38 +00:00
|
|
|
[](const auto a, const auto b) { return Op::apply(!!a, !!b); });
|
2019-07-18 08:07:24 +00:00
|
|
|
else
|
2019-07-18 09:09:29 +00:00
|
|
|
TypedExecutorInvoker<Op, Types ...>::template apply<T>(x, y, result);
|
2019-07-18 08:07:24 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Result>
|
2019-07-18 09:09:29 +00:00
|
|
|
static void apply(const IColumn & x, const IColumn & y, Result & result)
|
2019-07-18 08:07:24 +00:00
|
|
|
{
|
|
|
|
if (const auto column = typeid_cast<const ColumnVector<Type> *>(&x))
|
2019-07-18 09:09:29 +00:00
|
|
|
FastApplierImpl<Op>::template apply<Type>(*column, y, result);
|
2019-07-18 08:07:24 +00:00
|
|
|
else
|
2019-07-18 09:09:29 +00:00
|
|
|
TypedExecutorInvoker<Op, Types ...>::apply(x, y, result);
|
2019-07-18 08:07:24 +00:00
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <typename Op>
|
|
|
|
struct TypedExecutorInvoker<Op>
|
|
|
|
{
|
|
|
|
template <typename T, typename Result>
|
2019-07-18 09:09:29 +00:00
|
|
|
static void apply(const ColumnVector<T> &, const IColumn & y, Result &)
|
2019-07-18 08:07:24 +00:00
|
|
|
{
|
|
|
|
throw Exception(std::string("Unknown numeric column y of type: ") + demangle(typeid(y).name()), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Result>
|
2019-07-18 09:09:29 +00:00
|
|
|
static void apply(const IColumn & x, const IColumn &, Result &)
|
2019-07-18 08:07:24 +00:00
|
|
|
{
|
|
|
|
throw Exception(std::string("Unknown numeric column x of type: ") + demangle(typeid(x).name()), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2019-07-16 20:57:11 +00:00
|
|
|
template <class Op>
|
|
|
|
static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & result_info, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
/// Combine all constant columns into a single constant value.
|
|
|
|
UInt8 const_val = 0;
|
|
|
|
bool has_consts = extractConstColumns<Op>(arguments, const_val);
|
|
|
|
|
|
|
|
/// If the constant value uniquely determines the result, return it.
|
|
|
|
if (has_consts && (arguments.empty() || Op::apply(const_val, 0) == Op::apply(const_val, 1)))
|
|
|
|
{
|
|
|
|
if (!arguments.empty())
|
|
|
|
const_val = Op::apply(const_val, 0);
|
|
|
|
result_info.column = DataTypeUInt8().createColumnConst(input_rows_count, toField(const_val));
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
/// If the constant value is a neutral element, let's forget about it.
|
|
|
|
if (has_consts && Op::apply(const_val, 0) == 0 && Op::apply(const_val, 1) == 1)
|
|
|
|
has_consts = false;
|
|
|
|
|
2020-01-19 06:22:01 +00:00
|
|
|
auto col_res = has_consts ?
|
|
|
|
ColumnUInt8::create(input_rows_count, const_val) : ColumnUInt8::create(input_rows_count);
|
2019-07-16 20:57:11 +00:00
|
|
|
|
2019-07-18 09:09:29 +00:00
|
|
|
/// FastPath detection goes in here
|
|
|
|
if (arguments.size() == (has_consts ? 1 : 2))
|
|
|
|
{
|
|
|
|
if (has_consts)
|
2020-01-19 06:22:01 +00:00
|
|
|
FastApplierImpl<Op>::apply(*arguments[0], *col_res, col_res->getData());
|
2019-07-18 09:09:29 +00:00
|
|
|
else
|
2020-01-19 06:22:01 +00:00
|
|
|
FastApplierImpl<Op>::apply(*arguments[0], *arguments[1], col_res->getData());
|
2019-07-18 09:09:29 +00:00
|
|
|
|
|
|
|
result_info.column = std::move(col_res);
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2019-07-16 20:57:11 +00:00
|
|
|
/// Convert all columns to UInt8
|
2020-01-19 06:22:01 +00:00
|
|
|
UInt8ColumnPtrs uint8_args;
|
|
|
|
Columns converted_columns_holder;
|
2019-07-16 20:57:11 +00:00
|
|
|
for (const IColumn * column : arguments)
|
|
|
|
{
|
|
|
|
if (auto uint8_column = checkAndGetColumn<ColumnUInt8>(column))
|
|
|
|
uint8_args.push_back(uint8_column);
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto converted_column = ColumnUInt8::create(input_rows_count);
|
|
|
|
convertColumnToUInt8(column, converted_column->getData());
|
|
|
|
uint8_args.push_back(converted_column.get());
|
2020-01-19 06:22:01 +00:00
|
|
|
converted_columns_holder.emplace_back(std::move(converted_column));
|
2019-07-16 20:57:11 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-19 06:22:01 +00:00
|
|
|
OperationApplier<Op, AssociativeApplierImpl>::apply(uint8_args, col_res->getData(), has_consts);
|
2019-07-16 20:57:11 +00:00
|
|
|
|
|
|
|
result_info.column = std::move(col_res);
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Impl, typename Name>
|
|
|
|
DataTypePtr FunctionAnyArityLogical<Impl, Name>::getReturnTypeImpl(const DataTypes & arguments) const
|
|
|
|
{
|
|
|
|
if (arguments.size() < 2)
|
|
|
|
throw Exception("Number of arguments for function \"" + getName() + "\" should be at least 2: passed "
|
|
|
|
+ toString(arguments.size()),
|
|
|
|
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION);
|
|
|
|
|
|
|
|
bool has_nullable_arguments = false;
|
|
|
|
for (size_t i = 0; i < arguments.size(); ++i)
|
|
|
|
{
|
|
|
|
const auto & arg_type = arguments[i];
|
|
|
|
|
|
|
|
if (!has_nullable_arguments)
|
|
|
|
{
|
|
|
|
has_nullable_arguments = arg_type->isNullable();
|
|
|
|
if (has_nullable_arguments && !Impl::specialImplementationForNulls())
|
|
|
|
throw Exception("Logical error: Unexpected type of argument for function \"" + getName() + "\": "
|
|
|
|
" argument " + toString(i + 1) + " is of type " + arg_type->getName(), ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (!(isNativeNumber(arg_type)
|
|
|
|
|| (Impl::specialImplementationForNulls() && (arg_type->onlyNull() || isNativeNumber(removeNullable(arg_type))))))
|
|
|
|
throw Exception("Illegal type ("
|
|
|
|
+ arg_type->getName()
|
|
|
|
+ ") of " + toString(i + 1) + " argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
|
|
|
|
|
|
|
auto result_type = std::make_shared<DataTypeUInt8>();
|
|
|
|
return has_nullable_arguments
|
|
|
|
? makeNullable(result_type)
|
|
|
|
: result_type;
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename Impl, typename Name>
|
|
|
|
void FunctionAnyArityLogical<Impl, Name>::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count)
|
|
|
|
{
|
|
|
|
ColumnRawPtrs args_in;
|
|
|
|
for (const auto arg_index : arguments)
|
|
|
|
args_in.push_back(block.getByPosition(arg_index).column.get());
|
|
|
|
|
|
|
|
auto & result_info = block.getByPosition(result_index);
|
|
|
|
if (result_info.type->isNullable())
|
|
|
|
executeForTernaryLogicImpl<Impl>(std::move(args_in), result_info, input_rows_count);
|
|
|
|
else
|
|
|
|
basicExecuteImpl<Impl>(std::move(args_in), result_info, input_rows_count);
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
template <typename A, typename Op>
|
|
|
|
struct UnaryOperationImpl
|
|
|
|
{
|
|
|
|
using ResultType = typename Op::ResultType;
|
|
|
|
using ArrayA = typename ColumnVector<A>::Container;
|
|
|
|
using ArrayC = typename ColumnVector<ResultType>::Container;
|
|
|
|
|
|
|
|
static void NO_INLINE vector(const ArrayA & a, ArrayC & c)
|
|
|
|
{
|
|
|
|
std::transform(
|
|
|
|
a.cbegin(), a.cend(), c.begin(),
|
|
|
|
[](const auto x) { return Op::apply(x); });
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <template <typename> class Impl, typename Name>
|
|
|
|
DataTypePtr FunctionUnaryLogical<Impl, Name>::getReturnTypeImpl(const DataTypes & arguments) const
|
|
|
|
{
|
|
|
|
if (!isNativeNumber(arguments[0]))
|
|
|
|
throw Exception("Illegal type ("
|
|
|
|
+ arguments[0]->getName()
|
|
|
|
+ ") of argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
return std::make_shared<DataTypeUInt8>();
|
|
|
|
}
|
|
|
|
|
|
|
|
template <template <typename> class Impl, typename T>
|
|
|
|
bool functionUnaryExecuteType(Block & block, const ColumnNumbers & arguments, size_t result)
|
|
|
|
{
|
|
|
|
if (auto col = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
|
|
|
|
{
|
|
|
|
auto col_res = ColumnUInt8::create();
|
|
|
|
|
|
|
|
typename ColumnUInt8::Container & vec_res = col_res->getData();
|
|
|
|
vec_res.resize(col->getData().size());
|
|
|
|
UnaryOperationImpl<T, Impl<T>>::vector(col->getData(), vec_res);
|
|
|
|
|
|
|
|
block.getByPosition(result).column = std::move(col_res);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
template <template <typename> class Impl, typename Name>
|
|
|
|
void FunctionUnaryLogical<Impl, Name>::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
|
|
|
{
|
|
|
|
if (!(functionUnaryExecuteType<Impl, UInt8>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, UInt16>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, UInt32>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, UInt64>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, Int8>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, Int16>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, Int32>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, Int64>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, Float32>(block, arguments, result)
|
|
|
|
|| functionUnaryExecuteType<Impl, Float64>(block, arguments, result)))
|
|
|
|
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
|
|
|
+ " of argument of function " + getName(),
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
|
2014-08-22 00:57:20 +00:00
|
|
|
}
|