#pragma once #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int ZERO_ARRAY_OR_TUPLE_INDEX; extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS; } /** Функции по работе с массивами: * * array(с1, с2, ...) - создать массив из констант. * arrayElement(arr, i) - получить элемент массива по индексу. * Индекс начинается с 1. Также индекс может быть отрицательным - тогда он считается с конца массива. * has(arr, x) - есть ли в массиве элемент x. * indexOf(arr, x) - возвращает индекс элемента x (начиная с 1), если он есть в массиве, или 0, если его нет. * arrayEnumerate(arr) - возаращает массив [1,2,3,..., length(arr)] * * arrayUniq(arr) - считает количество разных элементов в массиве, * arrayUniq(arr1, arr2, ...) - считает количество разных кортежей из элементов на соответствующих позициях в нескольких массивах. * * arrayEnumerateUniq(arr) * - возаращает массив, параллельный данному, где для каждого элемента указано, * какой он по счету среди элементов с таким значением. * Например: arrayEnumerateUniq([10, 20, 10, 30]) = [1, 1, 2, 1] * arrayEnumerateUniq(arr1, arr2...) * - для кортежей из элементов на соответствующих позициях в нескольких массивах. * * emptyArrayToSingle(arr) - заменить пустые массивы на массивы из одного элемента со значением "по-умолчанию". * * arrayReduce('agg', arr1, ...) - применить агрегатную функцию agg к массивам arr1... */ class FunctionArray : public IFunction { public: static constexpr auto name = "array"; static FunctionPtr create(const Context & context); FunctionArray(const Context & context); void setCaseMode(); bool hasSpecialSupportForNulls() const override { return true; } /// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; /// Выполнить функцию над блоком. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; private: /// Получить имя функции. String getName() const override; bool addField(DataTypePtr type_res, const Field & f, Array & arr) const; static const DataTypePtr & getScalarType(const DataTypePtr & type); DataTypeTraits::EnrichedDataTypePtr getLeastCommonType(const DataTypes & arguments) const; private: const Context & context; bool is_case_mode = false; }; namespace ArrayImpl { class NullMapBuilder; } class FunctionArrayElement : public IFunction { public: static constexpr auto name = "arrayElement"; static FunctionPtr create(const Context & context); /// Получить имя функции. String getName() const override; bool hasSpecialSupportForNulls() const override { return true; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; /// Выполнить функцию над блоком. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; private: void perform(Block & block, const ColumnNumbers & arguments, size_t result, ArrayImpl::NullMapBuilder & builder); template bool executeNumberConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, ArrayImpl::NullMapBuilder & builder); template bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, ArrayImpl::NullMapBuilder & builder); bool executeStringConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, ArrayImpl::NullMapBuilder & builder); template bool executeString(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, ArrayImpl::NullMapBuilder & builder); bool executeGenericConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, ArrayImpl::NullMapBuilder & builder); template bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, ArrayImpl::NullMapBuilder & builder); bool executeConstConst(Block & block, const ColumnNumbers & arguments, size_t result, const Field & index, ArrayImpl::NullMapBuilder & builder); template bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result, const PaddedPODArray & indices, ArrayImpl::NullMapBuilder & builder); template bool executeArgument(Block & block, const ColumnNumbers & arguments, size_t result, ArrayImpl::NullMapBuilder & builder); /** Для массива кортежей функция вычисляется покомпонентно - для каждого элемента кортежа. */ bool executeTuple(Block & block, const ColumnNumbers & arguments, size_t result); }; /// For has. struct IndexToOne { using ResultType = UInt8; static bool apply(size_t j, ResultType & current) { current = 1; return false; } }; /// For indexOf. struct IndexIdentity { using ResultType = UInt64; /// Индекс возвращается начиная с единицы. static bool apply(size_t j, ResultType & current) { current = j + 1; return false; } }; /// For countEqual. struct IndexCount { using ResultType = UInt32; static bool apply(size_t j, ResultType & current) { ++current; return true; } }; template struct ArrayIndexNumImpl { #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wsign-compare" /// compares `lhs` against `i`-th element of `rhs` static bool compare(const T & lhs, const PaddedPODArray & rhs, const std::size_t i ) { return lhs == rhs[i]; } /// compares `lhs against `rhs`, third argument unused static bool compare(const T & lhs, const U & rhs, std::size_t) { return lhs == rhs; } #pragma GCC diagnostic pop static bool hasNull(const PaddedPODArray & value, const PaddedPODArray & null_map, size_t i) { return null_map[i] == 1; } static bool hasNull(const U & value, const PaddedPODArray & null_map, size_t i) { throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; } /// Both function arguments are ordinary. template static void vectorCase1( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, const ScalarOrVector & value, PaddedPODArray & result) { size_t size = offsets.size(); result.resize(size); ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { size_t array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; for (size_t j = 0; j < array_size; ++j) { if (compare(data[current_offset + j], value, i)) { if (!IndexConv::apply(j, current)) break; } } result[i] = current; current_offset = offsets[i]; } } /// The 2nd function argument is nullable. template static void vectorCase2( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, const ScalarOrVector & value, PaddedPODArray & result, const PaddedPODArray & null_map_item) { size_t size = offsets.size(); result.resize(size); ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { size_t array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; for (size_t j = 0; j < array_size; ++j) { if (!hasNull(value, null_map_item, i) && compare(data[current_offset + j], value, i)) { if (!IndexConv::apply(j, current)) break; } } result[i] = current; current_offset = offsets[i]; } } /// The 1st function argument is a non-constant array of nullable values. template static void vectorCase3( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, const ScalarOrVector & value, PaddedPODArray & result, const PaddedPODArray & null_map_data) { size_t size = offsets.size(); result.resize(size); ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { size_t array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; for (size_t j = 0; j < array_size; ++j) { if (null_map_data[current_offset + j] == 1) { } else if (compare(data[current_offset + j], value, i)) { if (!IndexConv::apply(j, current)) break; } } result[i] = current; current_offset = offsets[i]; } } /// The 1st function argument is a non-constant array of nullable values. /// The 2nd function argument is nullable. template static void vectorCase4( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, const ScalarOrVector & value, PaddedPODArray & result, const PaddedPODArray & null_map_data, const PaddedPODArray & null_map_item) { size_t size = offsets.size(); result.resize(size); ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { size_t array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; for (size_t j = 0; j < array_size; ++j) { bool hit = false; if (null_map_data[current_offset + j] == 1) { if (hasNull(value, null_map_item, i)) hit = true; } else if (compare(data[current_offset + j], value, i)) hit = true; if (hit) { if (!IndexConv::apply(j, current)) break; } } result[i] = current; current_offset = offsets[i]; } } template static void vector( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, const ScalarOrVector & value, PaddedPODArray & result, const PaddedPODArray * null_map_data, const PaddedPODArray * null_map_item) { /// Processing is split into 4 cases. if ((null_map_data == nullptr) && (null_map_item == nullptr)) vectorCase1(data, offsets, value, result); else if ((null_map_data == nullptr) && (null_map_item != nullptr)) vectorCase2(data, offsets, value, result, *null_map_item); else if ((null_map_data != nullptr) && (null_map_item == nullptr)) vectorCase3(data, offsets, value, result, *null_map_data); else vectorCase4(data, offsets, value, result, *null_map_data, *null_map_item); } }; /// Specialization that catches internal errors. template struct ArrayIndexNumImpl { template static void vector( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, const ScalarOrVector & value, PaddedPODArray & result, const PaddedPODArray * null_map_data, const PaddedPODArray * null_map_item) { throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR}; } }; /// Implementation for arrays of numbers when the 2nd function argument /// is a NULL value. template struct ArrayIndexNumNullImpl { static void vector( const PaddedPODArray & data, const ColumnArray::Offsets_t & offsets, PaddedPODArray & result, const PaddedPODArray * null_map_data) { size_t size = offsets.size(); result.resize(size); if (null_map_data == nullptr) return; const auto & null_map_ref = *null_map_data; ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { size_t array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; for (size_t j = 0; j < array_size; ++j) { if (null_map_ref[current_offset + j] == 1) { if (!IndexConv::apply(j, current)) break; } } result[i] = current; current_offset = offsets[i]; } } }; /// Implementation for arrays of strings when the 2nd function argument /// is a NULL value. template struct ArrayIndexStringNullImpl { static void vector_const( const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets, PaddedPODArray & result, const PaddedPODArray * null_map_data) { const auto size = offsets.size(); result.resize(size); if (null_map_data == nullptr) return; const auto & null_map_ref = *null_map_data; ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { const auto array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; for (size_t j = 0; j < array_size; ++j) { size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1; if (null_map_ref[k] == 1) { if (!IndexConv::apply(j, current)) break; } } result[i] = current; current_offset = offsets[i]; } } }; template struct ArrayIndexStringImpl { static void vector_const( const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets, const String & value, PaddedPODArray & result, const PaddedPODArray * null_map_data) { const auto size = offsets.size(); const auto value_size = value.size(); result.resize(size); ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { const auto array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; for (size_t j = 0; j < array_size; ++j) { ColumnArray::Offset_t string_pos = current_offset == 0 && j == 0 ? 0 : string_offsets[current_offset + j - 1]; ColumnArray::Offset_t string_size = string_offsets[current_offset + j] - string_pos; size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1; if (null_map_data && ((*null_map_data)[k] == 1)) { } else if (string_size == value_size + 1 && 0 == memcmp(value.data(), &data[string_pos], value_size)) { if (!IndexConv::apply(j, current)) break; } } result[i] = current; current_offset = offsets[i]; } } static void vector_vector( const ColumnString::Chars_t & data, const ColumnArray::Offsets_t & offsets, const ColumnString::Offsets_t & string_offsets, const ColumnString::Chars_t & item_values, const ColumnString::Offsets_t & item_offsets, PaddedPODArray & result, const PaddedPODArray * null_map_data, const PaddedPODArray * null_map_item) { const auto size = offsets.size(); result.resize(size); ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { const auto array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; const auto value_pos = 0 == i ? 0 : item_offsets[i - 1]; const auto value_size = item_offsets[i] - value_pos; for (size_t j = 0; j < array_size; ++j) { ColumnArray::Offset_t string_pos = current_offset == 0 && j == 0 ? 0 : string_offsets[current_offset + j - 1]; ColumnArray::Offset_t string_size = string_offsets[current_offset + j] - string_pos; bool hit = false; size_t k = (current_offset == 0 && j == 0) ? 0 : current_offset + j - 1; if (null_map_data && ((*null_map_data)[k] == 1)) { if (null_map_item && ((*null_map_item)[i] == 1)) hit = true; } else if (string_size == value_size && 0 == memcmp(&item_values[value_pos], &data[string_pos], value_size)) hit = true; if (hit) { if (!IndexConv::apply(j, current)) break; } } result[i] = current; current_offset = offsets[i]; } } }; /** Catch-all implementation for arrays of arbitary type. */ template struct ArrayIndexGenericImpl { /** To compare with constant value, create non-constant column with single element, * and pass is_value_has_single_element_to_compare = true. */ static void vector( const IColumn & data, const ColumnArray::Offsets_t & offsets, const IColumn & value, PaddedPODArray & result) { size_t size = offsets.size(); result.resize(size); ColumnArray::Offset_t current_offset = 0; for (size_t i = 0; i < size; ++i) { size_t array_size = offsets[i] - current_offset; typename IndexConv::ResultType current = 0; for (size_t j = 0; j < array_size; ++j) if (0 == data.compareAt(current_offset + j, is_value_has_single_element_to_compare ? 0 : i, value, 1)) if (!IndexConv::apply(j, current)) break; result[i] = current; current_offset = offsets[i]; } } }; template class FunctionArrayIndex : public IFunction { public: static constexpr auto name = Name::name; static FunctionPtr create(const Context & context) { return std::make_shared(); } private: using ResultColumnType = ColumnVector; template bool executeNumber(Block & block, const ColumnNumbers & arguments, size_t result) { return executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result) || executeNumberNumber(block, arguments, result); } template bool executeNumberNumber(Block & block, const ColumnNumbers & arguments, size_t result) { const ColumnArray * col_array = typeid_cast(block.getByPosition(arguments[0]).column.get()); if (!col_array) return false; const ColumnVector * col_nested = typeid_cast *>(&col_array->getData()); if (!col_nested) return false; /// Null maps of the 1st and second function arguments, /// if it applies. const PaddedPODArray * null_map_data = nullptr; const PaddedPODArray * null_map_item = nullptr; if (arguments.size() > 2) { const auto & null_map1 = block.getByPosition(arguments[2]).column; if (null_map1) null_map_data = &static_cast(*null_map1).getData(); const auto & null_map2 = block.getByPosition(arguments[3]).column; if (null_map2) null_map_item = &static_cast(*null_map2).getData(); } const auto item_arg = block.getByPosition(arguments[1]).column.get(); if (item_arg->isNull()) { const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; ArrayIndexNumNullImpl::vector(col_nested->getData(), col_array->getOffsets(), col_res->getData(), null_map_data); } else if (const auto item_arg_const = typeid_cast *>(item_arg)) { const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; if (item_arg->isNull()) ArrayIndexNumNullImpl::vector(col_nested->getData(), col_array->getOffsets(), col_res->getData(), null_map_data); else ArrayIndexNumImpl::vector(col_nested->getData(), col_array->getOffsets(), item_arg_const->getData(), col_res->getData(), null_map_data, nullptr); } else if (const auto item_arg_vector = typeid_cast *>(item_arg)) { const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; ArrayIndexNumImpl::vector(col_nested->getData(), col_array->getOffsets(), item_arg_vector->getData(), col_res->getData(), null_map_data, null_map_item); } else return false; return true; } bool executeString(Block & block, const ColumnNumbers & arguments, size_t result) { const ColumnArray * col_array = typeid_cast(block.getByPosition(arguments[0]).column.get()); if (!col_array) return false; const ColumnString * col_nested = typeid_cast(&col_array->getData()); if (!col_nested) return false; /// Null maps of the 1st and second function arguments, /// if it applies. const PaddedPODArray * null_map_data = nullptr; const PaddedPODArray * null_map_item = nullptr; if (arguments.size() > 2) { const auto & col1 = block.getByPosition(arguments[2]).column; if (col1) null_map_data = &static_cast(*col1).getData(); const auto & col2 = block.getByPosition(arguments[3]).column; if (col2) null_map_item = &static_cast(*col2).getData(); } const auto item_arg = block.getByPosition(arguments[1]).column.get(); if (item_arg->isNull()) { const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; ArrayIndexStringNullImpl::vector_const(col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), col_res->getData(), null_map_data); } else if (const auto item_arg_const = typeid_cast *>(item_arg)) { const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; ArrayIndexStringImpl::vector_const(col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), item_arg_const->getData(), col_res->getData(), null_map_data); } else if (const auto item_arg_vector = typeid_cast(item_arg)) { const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; ArrayIndexStringImpl::vector_vector(col_nested->getChars(), col_array->getOffsets(), col_nested->getOffsets(), item_arg_vector->getChars(), item_arg_vector->getOffsets(), col_res->getData(), null_map_data, null_map_item); } else return false; return true; } bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result) { const ColumnConstArray * col_array = typeid_cast(block.getByPosition(arguments[0]).column.get()); if (!col_array) return false; const Array & arr = col_array->getData(); const auto item_arg = block.getByPosition(arguments[1]).column.get(); if (item_arg->isConst()) { typename IndexConv::ResultType current = 0; const auto & value = (*item_arg)[0]; for (size_t i = 0, size = arr.size(); i < size; ++i) { if (apply_visitor(FieldVisitorAccurateEquals(), arr[i], value)) { if (!IndexConv::apply(i, current)) break; } } block.getByPosition(result).column = block.getByPosition(result).type->createConstColumn( item_arg->size(), static_cast::Type>(current)); } else { /// Null map of the 2nd function argument, if it applies. const PaddedPODArray * null_map = nullptr; if (arguments.size() > 2) { const auto & col = block.getByPosition(arguments[3]).column; if (col) null_map = &static_cast(*col).getData(); } const auto size = item_arg->size(); const auto col_res = std::make_shared(size); block.getByPosition(result).column = col_res; auto & data = col_res->getData(); for (size_t row = 0; row < size; ++row) { const auto & value = (*item_arg)[row]; data[row] = 0; for (size_t i = 0, size = arr.size(); i < size; ++i) { bool hit = false; if (arr[i].isNull()) { if (null_map && ((*null_map)[row] == 1)) hit = true; } else if (apply_visitor(FieldVisitorAccurateEquals(), arr[i], value)) hit = true; if (hit) { if (!IndexConv::apply(i, data[row])) break; } } } } return true; } bool executeGeneric(Block & block, const ColumnNumbers & arguments, size_t result) { const ColumnArray * col_array = typeid_cast(block.getByPosition(arguments[0]).column.get()); if (!col_array) return false; const IColumn & col_nested = col_array->getData(); const IColumn & item_arg = *block.getByPosition(arguments[1]).column; const auto col_res = std::make_shared(); block.getByPosition(result).column = col_res; if (item_arg.isConst()) { ArrayIndexGenericImpl::vector(col_nested, col_array->getOffsets(), *item_arg.cut(0, 1)->convertToFullColumnIfConst(), col_res->getData()); } else { /// If item_arg is tuple and have constants. if (auto materialized_tuple = item_arg.convertToFullColumnIfConst()) { ArrayIndexGenericImpl::vector( col_nested, col_array->getOffsets(), *materialized_tuple, col_res->getData()); } else ArrayIndexGenericImpl::vector( col_nested, col_array->getOffsets(), item_arg, col_res->getData()); } return true; } public: /// Get function name. String getName() const override { return name; } bool hasSpecialSupportForNulls() const override { return true; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. 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 2.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const DataTypeArray * array_type = typeid_cast(arguments[0].get()); if (!array_type) throw Exception("First argument for function " + getName() + " must be an array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!arguments[1]->isNull()) { const IDataType * observed_type0 = DataTypeTraits::removeNullable(array_type->getNestedType()).get(); const IDataType * observed_type1 = DataTypeTraits::removeNullable(arguments[1]).get(); if (!(observed_type0->behavesAsNumber() && observed_type1->behavesAsNumber()) && observed_type0->getName() != observed_type1->getName()) throw Exception("Types of array and 2nd argument of function " + getName() + " must be identical up to nullability. Passed: " + arguments[0]->getName() + " and " + arguments[1]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } return std::make_shared::Type>(); } /// Perform function on the given block. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { /// If one or both arguments passed to this function are nullable, /// we create a new block that contains non-nullable parameters: /// - if the 1st argument is a non-constant array of nullable values, /// it is turned into a non-constant array of ordinary values + a null /// byte map; /// - if the 2nd argument is a nullable value, it is turned into an /// ordinary value + a null byte map. /// Note that since constant arrays have quite a specific structure /// (they are vectors of Fields, which may represent the NULL value), /// they do not require any preprocessing /// Check if the 1st function argument is a non-constant array of nullable /// values. bool is_nullable; const ColumnArray * col_array = nullptr; col_array = typeid_cast(block.getByPosition(arguments[0]).column.get()); if (col_array) is_nullable = col_array->getData().isNullable(); else is_nullable = false; /// Check nullability of the 2nd function argument. bool is_arg_nullable = block.getByPosition(arguments[1]).column->isNullable(); if (!is_nullable && !is_arg_nullable) { /// Simple case: no nullable value is passed. perform(block, arguments, result); } else { /// Template of the block on which we will actually apply the function. /// Its elements will be filled later. Block source_block = { /// 1st function argument (data) { }, /// 2nd function argument { }, /// 1st argument null map { }, /// 2nd argument null map { }, /// Function result. { nullptr, block.getByPosition(result).type, "" } }; if (is_nullable) { const auto & nullable_col = static_cast(col_array->getData()); const auto & nested_col = nullable_col.getNestedColumn(); auto & data = source_block.unsafeGetByPosition(0); data.column = std::make_shared(nested_col, col_array->getOffsetsColumn()); data.type = static_cast(*block.getByPosition(arguments[0]).type).getNestedType(); auto & null_map = source_block.unsafeGetByPosition(2); null_map.column = nullable_col.getNullValuesByteMap(); null_map.type = std::make_shared(); } else { auto & data = source_block.unsafeGetByPosition(0); data = block.getByPosition(arguments[0]); } if (is_arg_nullable) { const auto & col = block.getByPosition(arguments[1]).column; const auto & nullable_col = static_cast(*col); auto & arg = source_block.unsafeGetByPosition(1); arg.column = nullable_col.getNestedColumn(); arg.type = static_cast(*block.getByPosition(arguments[1]).type).getNestedType(); auto & null_map = source_block.unsafeGetByPosition(3); null_map.column = nullable_col.getNullValuesByteMap(); null_map.type = std::make_shared(); } else { auto & arg = source_block.unsafeGetByPosition(1); arg = block.getByPosition(arguments[1]); } /// Now perform the function. perform(source_block, {0, 1, 2, 3}, 4); /// Move the result to its final position. const ColumnWithTypeAndName & source_col = source_block.unsafeGetByPosition(4); ColumnWithTypeAndName & dest_col = block.unsafeGetByPosition(result); dest_col.column = std::move(source_col.column); } } private: /// Perform function on the given block. Internal version. void perform(Block & block, const ColumnNumbers & arguments, size_t result) { if (!(executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeNumber(block, arguments, result) || executeConst(block, arguments, result) || executeString(block, arguments, result) || executeGeneric(block, arguments, result))) throw Exception{ "Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } }; class FunctionArrayEnumerate : public IFunction { public: static constexpr auto name = "arrayEnumerate"; static FunctionPtr create(const Context & context); /// Получить имя функции. String getName() const override; /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; /// Выполнить функцию над блоком. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; }; /// Считает количество разных элементов в массиве, или количество разных кортежей из элементов на соответствующих позициях в нескольких массивах. /// NOTE Реализация частично совпадает с arrayEnumerateUniq. class FunctionArrayUniq : public IFunction { public: static constexpr auto name = "arrayUniq"; static FunctionPtr create(const Context & context); /// Получить имя функции. String getName() const override; bool hasSpecialSupportForNulls() const override { return true; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; /// Выполнить функцию над блоком. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; private: /// Изначально выделить кусок памяти для 512 элементов. static constexpr size_t INITIAL_SIZE_DEGREE = 9; template bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container_t & res_values); bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container_t & res_values); bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result); bool execute128bit( const ColumnArray::Offsets_t & offsets, const ConstColumnPlainPtrs & columns, const ConstColumnPlainPtrs & null_maps, ColumnUInt32::Container_t & res_values, bool has_nullable_columns); void executeHashed( const ColumnArray::Offsets_t & offsets, const ConstColumnPlainPtrs & columns, ColumnUInt32::Container_t & res_values); }; class FunctionArrayEnumerateUniq : public IFunction { public: static constexpr auto name = "arrayEnumerateUniq"; static FunctionPtr create(const Context & context); /// Получить имя функции. String getName() const override; bool hasSpecialSupportForNulls() const override { return true; } /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; /// Выполнить функцию над блоком. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; private: /// Изначально выделить кусок памяти для 512 элементов. static constexpr size_t INITIAL_SIZE_DEGREE = 9; template bool executeNumber(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container_t & res_values); bool executeString(const ColumnArray * array, const IColumn * null_map, ColumnUInt32::Container_t & res_values); bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result); bool execute128bit( const ColumnArray::Offsets_t & offsets, const ConstColumnPlainPtrs & columns, const ConstColumnPlainPtrs & null_maps, ColumnUInt32::Container_t & res_values, bool has_nullable_columns); void executeHashed( const ColumnArray::Offsets_t & offsets, const ConstColumnPlainPtrs & columns, ColumnUInt32::Container_t & res_values); }; template struct TypeToColumnType { using ColumnType = ColumnVector; }; template <> struct TypeToColumnType { using ColumnType = ColumnString; }; template struct DataTypeToName : TypeName { }; template <> struct DataTypeToName { static std::string get() { return "Date"; } }; template <> struct DataTypeToName { static std::string get() { return "DateTime"; } }; template struct FunctionEmptyArray : public IFunction { static constexpr auto base_name = "emptyArray"; static const String name; static FunctionPtr create(const Context & context) { return std::make_shared(); } private: String getName() const override { return name; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() != 0) throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 0.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return std::make_shared(std::make_shared()); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { using UnderlyingColumnType = typename TypeToColumnType::ColumnType; block.getByPosition(result).column = std::make_shared( std::make_shared(), std::make_shared(block.rowsInFirstColumn(), 0)); } }; template const String FunctionEmptyArray::name = FunctionEmptyArray::base_name + DataTypeToName::get(); class FunctionRange : public IFunction { public: static constexpr auto max_elements = 100000000; static constexpr auto name = "range"; static FunctionPtr create(const Context &) { return std::make_shared(); } private: String getName() const override { return name; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() != 1) throw Exception{ "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH }; const auto arg = arguments.front().get(); if (!typeid_cast(arg) && !typeid_cast(arg) && !typeid_cast(arg) & !typeid_cast(arg)) { throw Exception{ "Illegal type " + arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT }; } return std::make_shared(arg->clone()); } template bool executeInternal(Block & block, const IColumn * const arg, const size_t result) { if (const auto in = typeid_cast *>(arg)) { const auto & in_data = in->getData(); const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), std::size_t{}, [this] (const std::size_t lhs, const std::size_t rhs) { const auto sum = lhs + rhs; if (sum < lhs) throw Exception{ "A call to function " + getName() + " overflows, investigate the values of arguments you are passing", ErrorCodes::ARGUMENT_OUT_OF_BOUND }; return sum; }); if (total_values > max_elements) throw Exception{ "A call to function " + getName() + " would produce " + std::to_string(total_values) + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), ErrorCodes::ARGUMENT_OUT_OF_BOUND }; const auto data_col = std::make_shared>(total_values); const auto out = std::make_shared( data_col, std::make_shared(in->size())); block.getByPosition(result).column = out; auto & out_data = data_col->getData(); auto & out_offsets = out->getOffsets(); IColumn::Offset_t offset{}; for (const auto i : ext::range(0, in->size())) { std::copy(ext::make_range_iterator(T{}), ext::make_range_iterator(in_data[i]), &out_data[offset]); offset += in_data[i]; out_offsets[i] = offset; } return true; } else if (const auto in = typeid_cast *>(arg)) { const auto & in_data = in->getData(); if (in->size() > std::numeric_limits::max() / in_data) throw Exception{ "A call to function " + getName() + " overflows, investigate the values of arguments you are passing", ErrorCodes::ARGUMENT_OUT_OF_BOUND }; const std::size_t total_values = in->size() * in_data; if (total_values > max_elements) throw Exception{ "A call to function " + getName() + " would produce " + std::to_string(total_values) + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), ErrorCodes::ARGUMENT_OUT_OF_BOUND }; const auto data_col = std::make_shared>(total_values); const auto out = std::make_shared( data_col, std::make_shared(in->size())); block.getByPosition(result).column = out; auto & out_data = data_col->getData(); auto & out_offsets = out->getOffsets(); IColumn::Offset_t offset{}; for (const auto i : ext::range(0, in->size())) { std::copy(ext::make_range_iterator(T{}), ext::make_range_iterator(in_data), &out_data[offset]); offset += in_data; out_offsets[i] = offset; } return true; } return false; } void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto col = block.getByPosition(arguments[0]).column.get(); if (!executeInternal(block, col, result) && !executeInternal(block, col, result) && !executeInternal(block, col, result) && !executeInternal(block, col, result)) { throw Exception{ "Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN }; } } }; class FunctionEmptyArrayToSingle : public IFunction { public: static constexpr auto name = "emptyArrayToSingle"; static FunctionPtr create(const Context & context); /// Получить имя функции. String getName() const override; /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; /// Выполнить функцию над блоком. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; private: bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result); template bool executeNumber( const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets, IColumn & res_data_col, ColumnArray::Offsets_t & res_offsets); bool executeFixedString( const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets, IColumn & res_data_col, ColumnArray::Offsets_t & res_offsets); bool executeString( const IColumn & src_data, const ColumnArray::Offsets_t & src_array_offsets, IColumn & res_data_col, ColumnArray::Offsets_t & res_array_offsets); }; class FunctionArrayReverse : public IFunction { public: static constexpr auto name = "reverse"; static FunctionPtr create(const Context & context); /// Получить имя функции. String getName() const override; /// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; /// Выполнить функцию над блоком. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; private: bool executeConst(Block & block, const ColumnNumbers & arguments, size_t result); template bool executeNumber( const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets, IColumn & res_data_col); bool executeFixedString( const IColumn & src_data, const ColumnArray::Offsets_t & src_offsets, IColumn & res_data_col); bool executeString( const IColumn & src_data, const ColumnArray::Offsets_t & src_array_offsets, IColumn & res_data_col); }; /** Применяет к массиву агрегатную функцию и возвращает её результат. * Также может быть применена к нескольким массивам одинаковых размеров, если агрегатная функция принимает несколько аргументов. */ class FunctionArrayReduce : public IFunction { public: static constexpr auto name = "arrayReduce"; static FunctionPtr create(const Context & context); /// Получить имя функции. String getName() const override; void getReturnTypeAndPrerequisitesImpl( const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, std::vector & out_prerequisites) override; void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; private: AggregateFunctionPtr aggregate_function; }; struct NameHas { static constexpr auto name = "has"; }; struct NameIndexOf { static constexpr auto name = "indexOf"; }; struct NameCountEqual { static constexpr auto name = "countEqual"; }; using FunctionHas = FunctionArrayIndex; using FunctionIndexOf = FunctionArrayIndex; using FunctionCountEqual = FunctionArrayIndex; using FunctionEmptyArrayUInt8 = FunctionEmptyArray; using FunctionEmptyArrayUInt16 = FunctionEmptyArray; using FunctionEmptyArrayUInt32 = FunctionEmptyArray; using FunctionEmptyArrayUInt64 = FunctionEmptyArray; using FunctionEmptyArrayInt8 = FunctionEmptyArray; using FunctionEmptyArrayInt16 = FunctionEmptyArray; using FunctionEmptyArrayInt32 = FunctionEmptyArray; using FunctionEmptyArrayInt64 = FunctionEmptyArray; using FunctionEmptyArrayFloat32 = FunctionEmptyArray; using FunctionEmptyArrayFloat64 = FunctionEmptyArray; using FunctionEmptyArrayDate = FunctionEmptyArray; using FunctionEmptyArrayDateTime = FunctionEmptyArray; using FunctionEmptyArrayString = FunctionEmptyArray; }