From ae04de1f9fdf50797bdf6033d7cba6152e1172ae Mon Sep 17 00:00:00 2001 From: akazz Date: Sat, 22 Jun 2019 13:49:19 +0300 Subject: [PATCH 01/79] [WIP] Basic working version --- dbms/src/Functions/FunctionsLogical.h | 440 ++++++++++++++++---------- 1 file changed, 281 insertions(+), 159 deletions(-) diff --git a/dbms/src/Functions/FunctionsLogical.h b/dbms/src/Functions/FunctionsLogical.h index 6a32307cf22..540186347c4 100644 --- a/dbms/src/Functions/FunctionsLogical.h +++ b/dbms/src/Functions/FunctionsLogical.h @@ -30,10 +30,12 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; } + +// TODO: This comment is to be rewritten /** Behaviour in presence of NULLs: * * Functions AND, XOR, NOT use default implementation for NULLs: @@ -44,6 +46,95 @@ namespace ErrorCodes * For example, 1 OR NULL returns 1, not NULL. */ +using UInt8Container = ColumnUInt8::Container; +using UInt8ColumnPtrs = std::vector; +using Columns3vPtrs = std::vector; + +namespace { +namespace Trivalent +{ + const UInt8 False = 0; + const UInt8 True = -1; + const UInt8 Null = 1; +} + +inline UInt8 toTrivalent(bool value, bool is_null = false) +{ + if (is_null) + return Trivalent::Null; + return value ? Trivalent::True : Trivalent::False; +} + +MutableColumnPtr makeColumnFromTrivalentData(const UInt8Container & trivalent_data, bool is_nullable) +{ + const size_t rows_count = trivalent_data.size(); + + auto new_column = ColumnUInt8::create(rows_count); + for (size_t i = 0; i < rows_count; ++i) + new_column->getData()[i] = (trivalent_data[i] == Trivalent::True); + + if (!is_nullable) + return new_column; + + auto null_column = ColumnUInt8::create(rows_count); + for (size_t i = 0; i < rows_count; ++i) + null_column->getData()[i] = (trivalent_data[i] == Trivalent::Null); + + return ColumnNullable::create(std::move(new_column), std::move(null_column)); +} + +template +bool convertTypeToTrivalent(const IColumn * column, ConstNullMapPtr null_map_ptr, UInt8Container & res) +{ + auto col = checkAndGetColumn>(column); + if (!col) + return false; + + const size_t n = res.size(); + const auto & vec = col->getData(); + + if (null_map_ptr != nullptr) + { + for (size_t i = 0; i < n; ++i) + res[i] = toTrivalent(!!vec[i], (*null_map_ptr)[i]); + } + else + { + for (size_t i = 0; i < n; ++i) + res[i] = toTrivalent(!!vec[i]); + } + + return true; +} + +ColumnUInt8::MutablePtr columnTrivalentFromColumnVector(const IColumn * column) +{ + ConstNullMapPtr null_map_ptr = nullptr; + if (const auto column_nullable = checkAndGetColumn(column)) + { + null_map_ptr = &column_nullable->getNullMapData(); + column = column_nullable->getNestedColumnPtr().get(); + } + + auto result_column = ColumnUInt8::create(column->size()); + auto & result_data = result_column->getData(); + + if (!convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data) && + !convertTypeToTrivalent(column, null_map_ptr, result_data)) + throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN); + + return result_column; +} +} + struct AndImpl { static inline constexpr bool isSaturable() @@ -51,17 +142,17 @@ struct AndImpl return true; } - static inline constexpr bool isSaturatedValue(bool a) + static inline constexpr bool isSaturatedValue(UInt8 a) { - return !a; + return a == Trivalent::False; } - static inline constexpr bool apply(bool a, bool b) + static inline constexpr UInt8 apply(UInt8 a, UInt8 b) { - return a && b; + return a & b; } - static inline constexpr bool specialImplementationForNulls() { return false; } + static inline constexpr bool specialImplementationForNulls() { return true; } }; struct OrImpl @@ -71,14 +162,14 @@ struct OrImpl return true; } - static inline constexpr bool isSaturatedValue(bool a) + static inline constexpr bool isSaturatedValue(UInt8 a) { - return a; + return a == Trivalent::True; } - static inline constexpr bool apply(bool a, bool b) + static inline constexpr UInt8 apply(UInt8 a, UInt8 b) { - return a || b; + return a | b; } static inline constexpr bool specialImplementationForNulls() { return true; } @@ -96,9 +187,9 @@ struct XorImpl return false; } - static inline constexpr bool apply(bool a, bool b) + static inline constexpr UInt8 apply(UInt8 a, UInt8 b) { - return a != b; + return (a != b) ? Trivalent::True : Trivalent::False; } static inline constexpr bool specialImplementationForNulls() { return false; } @@ -129,11 +220,7 @@ struct NotImpl #endif }; - -using UInt8Container = ColumnUInt8::Container; -using UInt8ColumnPtrs = std::vector; - - +/* template struct AssociativeOperationImpl { @@ -146,13 +233,19 @@ struct AssociativeOperationImpl return; } - AssociativeOperationImpl operation(in); + AssociativeOperationImpl operation(in); in.erase(in.end() - N, in.end()); - size_t n = result.size(); - for (size_t i = 0; i < n; ++i) + const size_t n = result.size(); + if (Op::isSaturable()) { - result[i] = operation.apply(i); + for (size_t i = 0; i < n; ++i) + result[i] = Op::isSaturatedValue(result[i]) ? result[i] : Op::apply(result[i], operation.apply(i)); + } + else + { + for (size_t i = 0; i < n; ++i) + result[i] = Op::apply(result[i], operation.apply(i)); } } @@ -164,38 +257,134 @@ struct AssociativeOperationImpl : vec(in[in.size() - N]->getData()), continuation(in) {} /// Returns a combination of values in the i-th row of all columns stored in the constructor. + inline UInt8 applySaturable(size_t i) const + { + return Op::isSaturatedValue(vec[i]) ? vec[i] : Op::apply(vec[i], continuation.applySaturable(i)); + } inline UInt8 apply(size_t i) const { - if (Op::isSaturable()) - { - UInt8 a = vec[i]; - return Op::isSaturatedValue(a) ? a : continuation.apply(i); - } - else - { - return Op::apply(vec[i], continuation.apply(i)); - } + return Op::apply(vec[i], continuation.apply(i)); } }; template struct AssociativeOperationImpl { - static void execute(UInt8ColumnPtrs &, UInt8Container &) + static void execute(UInt8ColumnPtrs & in, UInt8Container & result) { - throw Exception("Logical error: AssociativeOperationImpl::execute called", ErrorCodes::LOGICAL_ERROR); + if (in.size() != 1) + { + throw Exception("Logical error: AssociativeOperationImpl::execute called", ErrorCodes::LOGICAL_ERROR); + } + + AssociativeOperationImpl operation(in); + + const size_t n = result.size(); + for (size_t i = 0; i < n; ++i) + result[i] = Op::apply(result[i], in[0]); + + in.clear(); } const UInt8Container & vec; AssociativeOperationImpl(UInt8ColumnPtrs & in) - : vec(in[in.size() - 1]->getData()) {} + : vec(in.back()->getData()) {} inline UInt8 apply(size_t i) const { return vec[i]; } }; +*/ + +template +struct AssociativeOperationImpl +{ + /// Erases the N last columns from `in` (if there are less, then all) and puts into `result` their combination. + static void NO_INLINE execute(Columns3vPtrs & in, UInt8Container & result) + { + if (N > in.size()) + { + AssociativeOperationImpl::execute(in, result); + return; + } + + AssociativeOperationImpl operation(in); + in.erase(in.end() - N, in.end()); + + size_t n = result.size(); + for (size_t i = 0; i < n; ++i) + result[i] = operation.apply(result[i], i); + + } + + const UInt8Container & vec; + AssociativeOperationImpl continuation; + + /// Remembers the last N columns from `in`. + AssociativeOperationImpl(Columns3vPtrs & in) + : vec(in[in.size() - N]->getData()), continuation(in) {} + + /// Returns a combination of values in the i-th row of all columns stored in the constructor. + inline UInt8 apply(UInt8 a, size_t i) const + { + if (Op::isSaturable()) + return Op::isSaturatedValue(a) ? a : Op::apply(a, continuation.apply(vec[i], i)); + else + return Op::apply(a, continuation.apply(vec[i], i)); + } +}; + +template +struct AssociativeOperationImpl +{ + static void execute(Columns3vPtrs & in, UInt8Container & result) + { + if (in.size() != 1) + { + throw Exception("Logical error: AssociativeOperationImpl::execute called", ErrorCodes::LOGICAL_ERROR); + } + + AssociativeOperationImpl operation(in); + + const size_t n = result.size(); + for (size_t i = 0; i < n; ++i) + result[i] = operation.apply(result[i], i); + + in.clear(); + } + + const UInt8Container & vec; + + AssociativeOperationImpl(Columns3vPtrs & in) + : vec(in.back()->getData()) {} + + inline UInt8 apply(UInt8 a, size_t i) const + { + return Op::apply(a, vec[i]); + } +}; + +template +UInt8Container associativeOperationApplyAll(Columns3vPtrs & args) +{ + if (args.size() < 2) + return {}; + + UInt8Container result_data = std::move(args.back()->getData()); + args.pop_back(); + + /// Effeciently combine all the columns of the correct type. + while (args.size() > 0) + { + /// With a large block size, combining 10 columns per pass is the fastest. + /// When small - more, is faster. + AssociativeOperationImpl::execute(args, result_data); + } + + return result_data; +} template @@ -206,7 +395,7 @@ public: static FunctionPtr create(const Context &) { return std::make_shared(); } private: - bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res) + static bool extractConstColumnsTrivalent(ColumnRawPtrs & in, UInt8 & res_3v) { bool has_res = false; for (int i = static_cast(in.size()) - 1; i >= 0; --i) @@ -214,16 +403,19 @@ private: if (!in[i]->isColumnConst()) continue; - Field value = (*in[i])[0]; + const auto field_value = (*in[i])[0]; + + UInt8 value_3v = field_value.isNull() + ? toTrivalent(false, true) + : toTrivalent(applyVisitor(FieldVisitorConvertToNumber(), field_value)); - UInt8 x = !value.isNull() && applyVisitor(FieldVisitorConvertToNumber(), value); if (has_res) { - res = Impl::apply(res, x); + res_3v = Impl::apply(res_3v, value_3v); } else { - res = x; + res_3v = value_3v; has_res = true; } @@ -232,63 +424,6 @@ private: return has_res; } - template - bool convertTypeToUInt8(const IColumn * column, UInt8Container & res) - { - auto col = checkAndGetColumn>(column); - if (!col) - return false; - const auto & vec = col->getData(); - size_t n = res.size(); - for (size_t i = 0; i < n; ++i) - res[i] = !!vec[i]; - - return true; - } - - template - bool convertNullableTypeToUInt8(const IColumn * column, UInt8Container & res) - { - auto col_nullable = checkAndGetColumn(column); - - auto col = checkAndGetColumn>(&col_nullable->getNestedColumn()); - if (!col) - return false; - - const auto & vec = col->getData(); - const auto & null_map = col_nullable->getNullMapData(); - - size_t n = res.size(); - for (size_t i = 0; i < n; ++i) - res[i] = !!vec[i] && !null_map[i]; - - return true; - } - - void convertToUInt8(const IColumn * column, UInt8Container & res) - { - if (!convertTypeToUInt8(column, res) && - !convertTypeToUInt8(column, res) && - !convertTypeToUInt8(column, res) && - !convertTypeToUInt8(column, res) && - !convertTypeToUInt8(column, res) && - !convertTypeToUInt8(column, res) && - !convertTypeToUInt8(column, res) && - !convertTypeToUInt8(column, res) && - !convertTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res) && - !convertNullableTypeToUInt8(column, res)) - throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN); - } - public: String getName() const override { @@ -304,93 +439,80 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() < 2) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be at least 2.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + 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) - if (!(isNativeNumber(arguments[i]) - || (Impl::specialImplementationForNulls() && (arguments[i]->onlyNull() || isNativeNumber(removeNullable(arguments[i])))))) + { + const auto & arg_type = arguments[i]; + + if (!has_nullable_arguments) + has_nullable_arguments = arg_type->isNullable(); + + if (!(isNativeNumber(arg_type) + || (Impl::specialImplementationForNulls() && (arg_type->onlyNull() || isNativeNumber(removeNullable(arg_type)))))) throw Exception("Illegal type (" - + arguments[i]->getName() + + arg_type->getName() + ") of " + toString(i + 1) + " argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } - return std::make_shared(); + /// Put this check inside the loop + if (has_nullable_arguments && !Impl::specialImplementationForNulls()) + throw Exception( + "Function \"" + getName() + "\" does not support Nullable arguments", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto result_type = std::make_shared(); + return has_nullable_arguments + ? makeNullable(result_type) + : result_type; } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) override { - size_t num_arguments = arguments.size(); - ColumnRawPtrs in(num_arguments); - for (size_t i = 0; i < num_arguments; ++i) - in[i] = block.getByPosition(arguments[i]).column.get(); + auto & result_info = block.getByPosition(result_index); - size_t rows = in[0]->size(); + ColumnRawPtrs args_in; + for (const auto arg_index : arguments) + { + const auto & arg_info = block.getByPosition(arg_index); + args_in.push_back(arg_info.column.get()); + } /// Combine all constant columns into a single value. - UInt8 const_val = 0; - bool has_consts = extractConstColumns(in, const_val); + UInt8 const_3v_value = 0; + bool has_consts = extractConstColumnsTrivalent(args_in, const_3v_value); - // If this value uniquely determines the result, return it. - if (has_consts && (in.empty() || Impl::apply(const_val, 0) == Impl::apply(const_val, 1))) + /// If this value uniquely determines result, return it. + if (has_consts && (args_in.empty() || (Impl::isSaturable() && Impl::isSaturatedValue(const_3v_value)))) { - if (!in.empty()) - const_val = Impl::apply(const_val, 0); - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(rows, toField(const_val)); + result_info.column = ColumnConst::create( + makeColumnFromTrivalentData(UInt8Container({const_3v_value}), result_info.type->isNullable()), + input_rows_count + ); return; } - /// If this value is a neutral element, let's forget about it. - if (has_consts && Impl::apply(const_val, 0) == 0 && Impl::apply(const_val, 1) == 1) - has_consts = false; - - auto col_res = ColumnUInt8::create(); - UInt8Container & vec_res = col_res->getData(); + // TODO: What if arguments are duplicated? + /// Prepare Trivalent representation for all columns + Columns3vPtrs args_in_3v; if (has_consts) - { - vec_res.assign(rows, const_val); - in.push_back(col_res.get()); - } - else - { - vec_res.resize(rows); - } + args_in_3v.push_back(ColumnUInt8::create(input_rows_count, const_3v_value)); + for (const auto arg_column_ptr : args_in) + args_in_3v.push_back(columnTrivalentFromColumnVector(arg_column_ptr)); - /// Convert all columns to UInt8 - UInt8ColumnPtrs uint8_in; - Columns converted_columns; + const auto result_data = associativeOperationApplyAll(args_in_3v); - for (const IColumn * column : in) - { - if (auto uint8_column = checkAndGetColumn(column)) - uint8_in.push_back(uint8_column); - else - { - auto converted_column = ColumnUInt8::create(rows); - convertToUInt8(column, converted_column->getData()); - uint8_in.push_back(converted_column.get()); - converted_columns.emplace_back(std::move(converted_column)); - } - } - - /// Effeciently combine all the columns of the correct type. - while (uint8_in.size() > 1) - { - /// With a large block size, combining 6 columns per pass is the fastest. - /// When small - more, is faster. - AssociativeOperationImpl::execute(uint8_in, vec_res); - uint8_in.push_back(col_res.get()); - } - - /// This is possible if there is exactly one non-constant among the arguments, and it is of type UInt8. - if (uint8_in[0] != col_res.get()) - vec_res.assign(uint8_in[0]->getData()); - - block.getByPosition(result).column = std::move(col_res); + result_info.column = makeColumnFromTrivalentData(result_data, result_info.type->isNullable()); } + /// TODO: Rework this part !!! + /// TODO: Rework this part !!! + /// TODO: Rework this part !!! #if USE_EMBEDDED_COMPILER bool isCompilableImpl(const DataTypes &) const override { return true; } From df2d347617adec8ab0f1378056f57782e5452aff Mon Sep 17 00:00:00 2001 From: akazz Date: Tue, 16 Jul 2019 23:57:11 +0300 Subject: [PATCH 02/79] WIP: Candidate1 - Much less code. Several TODO's are still there --- dbms/src/Functions/FunctionsLogical.cpp | 562 ++++++++++++++++++++++++ dbms/src/Functions/FunctionsLogical.h | 541 +++-------------------- dbms/src/Functions/equals.cpp | 1 + 3 files changed, 615 insertions(+), 489 deletions(-) diff --git a/dbms/src/Functions/FunctionsLogical.cpp b/dbms/src/Functions/FunctionsLogical.cpp index 29035366909..566a8e4dc16 100644 --- a/dbms/src/Functions/FunctionsLogical.cpp +++ b/dbms/src/Functions/FunctionsLogical.cpp @@ -1,6 +1,20 @@ #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + namespace DB { @@ -12,4 +26,552 @@ void registerFunctionsLogical(FunctionFactory & factory) factory.registerFunction(); } +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; + + +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 +bool tryConvertColumnToUInt8(const IColumn * column, UInt8Container & res) +{ + const auto col = checkAndGetColumn>(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(column, res) && + !tryConvertColumnToUInt8(column, res) && + !tryConvertColumnToUInt8(column, res) && + !tryConvertColumnToUInt8(column, res) && + !tryConvertColumnToUInt8(column, res) && + !tryConvertColumnToUInt8(column, res) && + !tryConvertColumnToUInt8(column, res) && + !tryConvertColumnToUInt8(column, res) && + !tryConvertColumnToUInt8(column, res)) + throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN); +} + + +template +class AssociativeApplierImpl +{ + using ResultValueType = typename Op::ResultType; + +public: + /// Remembers the last N columns from `in`. + AssociativeApplierImpl(const UInt8ColumnPtrs & in) + : 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. + inline ResultValueType apply(const ResultValueType a, const size_t i) const + { + if constexpr (Op::isSaturable()) + return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(vec[i], i)); + else + return Op::apply(a, next.apply(vec[i], i)); + } + +private: + const UInt8Container & vec; + const AssociativeApplierImpl next; +}; + +template +class AssociativeApplierImpl +{ + using ResultValueType = typename Op::ResultType; + +public: + AssociativeApplierImpl(const UInt8ColumnPtrs & in) + : vec(in[in.size() - 1]->getData()) {} + + inline ResultValueType apply(const ResultValueType a, const size_t i) const + { + return Op::apply(a, vec[i]); + } + +private: + const UInt8Container & vec; +}; + + +template +static bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res) +{ + bool has_res = false; + for (int i = static_cast(in.size()) - 1; i >= 0; --i) + { + if (!in[i]->isColumnConst()) + continue; + + Field value = (*in[i])[0]; + + UInt8 x = !value.isNull() && applyVisitor(FieldVisitorConvertToNumber(), value); + if (has_res) + { + res = Op::apply(res, x); + } + else + { + res = x; + has_res = true; + } + + in.erase(in.begin() + i); + } + return has_res; +} + + +/// TODO: Add a good comment here about what this is +using TValueGetter = std::function; + +template +struct ValueGetterBuilderImpl; + +template +struct ValueGetterBuilderImpl +{ + static TValueGetter build(const IColumn * x) + { + if (const auto nullable_column = typeid_cast(x)) + { + if (const auto nested_column = typeid_cast *>(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::build(x); + } + else if (const auto column = typeid_cast *>(x)) + return [&column_data = column->getData()](size_t i) { return Ternary::makeValue(column_data[i]); }; + else + return ValueGetterBuilderImpl::build(x); + } +}; + +template <> +struct ValueGetterBuilderImpl<> +{ + static TValueGetter build(const IColumn * x) + { + throw Exception( + std::string("Unknown numeric column of type: ") + demangle(typeid(x).name()), + ErrorCodes::LOGICAL_ERROR); + } +}; + +using ValueGetterBuilder = + ValueGetterBuilderImpl; + +template +class AssociativeGenericApplierImpl +{ + using ResultValueType = typename Op::ResultType; + +public: + /// Remembers the last N columns from `in`. + AssociativeGenericApplierImpl(const ColumnRawPtrs & in) + : 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. + inline ResultValueType apply(const ResultValueType a, const size_t i) const + { + if constexpr (Op::isSaturable()) + return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(i)); + else + return Op::apply(a, next.apply(i)); + + } + inline ResultValueType apply(const size_t i) const + { + return apply(val_getter(i), i); + } + +private: + const TValueGetter val_getter; + const AssociativeGenericApplierImpl next; +}; + + +template +class AssociativeGenericApplierImpl +{ + using ResultValueType = typename Op::ResultType; + +public: + /// Remembers the last N columns from `in`. + AssociativeGenericApplierImpl(const ColumnRawPtrs & in) + : val_getter_L{ValueGetterBuilder::build(in[in.size() - 2])} + , val_getter_R{ValueGetterBuilder::build(in[in.size() - 1])} {} + + inline ResultValueType apply(const size_t i) const + { + return Op::apply(val_getter_L(i), val_getter_R(i)); + } + +private: + const TValueGetter val_getter_L; + const TValueGetter val_getter_R; +}; + +template +class AssociativeGenericApplierImpl +{ + using ResultValueType = typename Op::ResultType; + +public: + /// Remembers the last N columns from `in`. + AssociativeGenericApplierImpl(const ColumnRawPtrs & in) + : val_getter{ValueGetterBuilder::build(in[in.size() - 1])} {} + + inline ResultValueType apply(const size_t i) const { return val_getter(i); } + +private: + const TValueGetter val_getter; +}; + + +template +bool extractConstColumnsTernary(ColumnRawPtrs & in, UInt8 & res_3v) +{ + bool has_res = false; + for (int i = static_cast(in.size()) - 1; i >= 0; --i) + { + if (!in[i]->isColumnConst()) + continue; + + const auto field_value = (*in[i])[0]; + + UInt8 value_3v = field_value.isNull() + ? Ternary::makeValue(false, true) + : Ternary::makeValue(applyVisitor(FieldVisitorConvertToNumber(), field_value)); + + if (has_res) + { + res_3v = Op::apply(res_3v, value_3v); + } + else + { + res_3v = value_3v; + has_res = true; + } + + in.erase(in.begin() + i); + } + return has_res; +} + + +template < + template typename OperationApplierImpl, typename Op, + size_t N, bool use_result_as_input> +struct OperationApplier +{ + template + static void NO_INLINE run(Columns & in, Result & result) + { + if (N > in.size()) + { + OperationApplier::run(in, result); + return; + } + + const OperationApplierImpl operationApplierImpl(in); + size_t i = 0; + for (auto & res : result) + if constexpr (use_result_as_input) + res = operationApplierImpl.apply(res, i++); + else + res = operationApplierImpl.apply(i++); + + in.erase(in.end() - N, in.end()); + } +}; + +template < + template typename OperationApplierImpl, typename Op> +struct OperationApplier +{ + template + static void NO_INLINE run(Columns &, Result &) + { + throw Exception( + "AssociativeOperationImpl::execute(...): not enough arguments to run this method", + ErrorCodes::LOGICAL_ERROR); + } +}; + +template < + template typename OperationApplierImpl, typename Op> +struct OperationApplier +{ + template + static void NO_INLINE run(Columns &, Result &) + { + throw Exception( + "AssociativeOperationImpl::execute(...): not enough arguments to run this method", + ErrorCodes::LOGICAL_ERROR); + } +}; + + +template +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(arguments, const_3v_value); + + /// If the constant value uniquely determines the result, return it. + if (has_consts && (arguments.empty() || (Op::isSaturable() && Op::isSaturatedValue(const_3v_value)))) + { + result_info.column = ColumnConst::create( + convertFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()), + input_rows_count + ); + return; + } + + const auto result_column = ColumnUInt8::create(input_rows_count); + + /// Efficiently combine all the columns of the correct type. + while (arguments.size() > 1) + { + /// Combining 10 columns per pass is the fastest for large block sizes. + /// For small block sizes - more columns is faster. + OperationApplier::run(arguments, result_column->getData()); + arguments.insert(arguments.cbegin(), result_column.get()); + } + + result_info.column = convertFromTernaryData(result_column->getData(), result_info.type->isNullable()); +} + + +template +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(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; + + auto col_res = ColumnUInt8::create(); + UInt8Container & vec_res = col_res->getData(); + + if (has_consts) + { + vec_res.assign(input_rows_count, const_val); + arguments.push_back(col_res.get()); + } + else + { + vec_res.resize(input_rows_count); + } + + /// Convert all columns to UInt8 + UInt8ColumnPtrs uint8_args; + Columns converted_columns; + + for (const IColumn * column : arguments) + { + if (auto uint8_column = checkAndGetColumn(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()); + converted_columns.emplace_back(std::move(converted_column)); + } + } + + /// Effeciently combine all the columns of the correct type. + while (uint8_args.size() > 1) + { + /// With a large block size, combining 10 columns per pass is the fastest. + /// When small - more, is faster. + OperationApplier::run(uint8_args, vec_res); + uint8_args.push_back(col_res.get()); + } + + /// This is possible if there is exactly one non-constant among the arguments, and it is of type UInt8. + if (uint8_args[0] != col_res.get()) + vec_res.assign(uint8_args[0]->getData()); + + result_info.column = std::move(col_res); +} + +} + +template +DataTypePtr FunctionAnyArityLogical::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(); + return has_nullable_arguments + ? makeNullable(result_type) + : result_type; +} + +template +void FunctionAnyArityLogical::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(std::move(args_in), result_info, input_rows_count); + else + basicExecuteImpl(std::move(args_in), result_info, input_rows_count); +} + + +template +struct UnaryOperationImpl +{ + using ResultType = typename Op::ResultType; + using ArrayA = typename ColumnVector::Container; + using ArrayC = typename ColumnVector::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); }); + } + + static void constant(A a, ResultType & c) + { + c = Op::apply(a); + } +}; + +template