#include #include #include #include #include #include #include #include #include #include #include #include /** The function will enumerate distinct values of the passed multidimensional arrays looking inside at the specified depths. * This is very unusual function made as a special order for Yandex.Metrica. * * arrayEnumerateUniqRanked(['hello', 'world', 'hello']) = [1, 1, 2] * - it returns similar structured array containing number of occurence of the corresponding value. * * arrayEnumerateUniqRanked([['hello', 'world'], ['hello'], ['hello']], 1) = [1, 1, 2] * - look at the depth 1 by default. Elements are ['hello', 'world'], ['hello'], ['hello']. * * arrayEnumerateUniqRanked([['hello', 'world'], ['hello'], ['hello']]) = [[1,1],[2],[3]] * - look at the depth 2. Return similar structured array. * arrayEnumerateUniqRanked([['hello', 'world'], ['hello'], ['hello']], 2) = [[1,1],[2],[3]] * - look at the maximum depth by default. * * We may pass multiple array arguments. Their elements will be processed as zipped to tuple. * * arrayEnumerateUniqRanked(['hello', 'hello', 'world', 'world'], ['a', 'b', 'b', 'b']) = [1, 1, 1, 2] * * We may provide arrays of different depths to look at different arguments. * * arrayEnumerateUniqRanked([['hello', 'world'], ['hello'], ['world'], ['world']], ['a', 'b', 'b', 'b']) = [[1,1],[1],[1],[2]] * arrayEnumerateUniqRanked([['hello', 'world'], ['hello'], ['world'], ['world']], 1, ['a', 'b', 'b', 'b'], 1) = [1, 1, 1, 2] * * When depths are different, we process less deep arrays as promoted to deeper arrays of similar structure by duplicating elements. * * arrayEnumerateUniqRanked( * [['hello', 'world'], ['hello'], ['world'], ['world']], * ['a', 'b', 'b', 'b']) * = arrayEnumerateUniqRanked( * [['hello', 'world'], ['hello'], ['world'], ['world']], * [['a', 'a'], ['b'], ['b'], ['b']]) * * Finally, we can provide extra first argument named "clear_depth" (it can be considered as 1 by default). * Array elements at the clear_depth will be enumerated as separate elements (enumeration counter is reset for each new element). * * SELECT arrayEnumerateUniqRanked(1, [['hello', 'world'], ['hello'], ['world'], ['world']]) = [[1,1],[2],[2],[3]] * SELECT arrayEnumerateUniqRanked(2, [['hello', 'world'], ['hello'], ['world'], ['world']]) = [[1,1],[1],[1],[1]] * SELECT arrayEnumerateUniqRanked(1, [['hello', 'world', 'hello'], ['hello'], ['world'], ['world']]) = [[1,1,2],[3],[2],[3]] * SELECT arrayEnumerateUniqRanked(2, [['hello', 'world', 'hello'], ['hello'], ['world'], ['world']]) = [[1,1,2],[1],[1],[1]] */ namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; } class FunctionArrayEnumerateUniqRanked; class FunctionArrayEnumerateDenseRanked; using DepthType = uint32_t; using DepthTypes = std::vector; struct ArraysDepths { /// Enumerate elements at the specified level separately. DepthType clear_depth; /// Effective depth is the array depth by default or lower value, specified as a constant argument following the array. /// f([[1, 2], [3]]) - effective depth is 2. /// f([[1, 2], [3]], 1) - effective depth is 1. DepthTypes depths; /// Maximum effective depth. DepthType max_array_depth; }; /// Return depth info about passed arrays ArraysDepths getArraysDepths(const ColumnsWithTypeAndName & arguments); template class FunctionArrayEnumerateRankedExtended : public IFunction { public: static FunctionPtr create(const Context & /* context */) { return std::make_shared(); } String getName() const override { return Derived::name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() == 0) throw Exception( "Number of arguments for function " + getName() + " doesn't match: passed " + std::to_string(arguments.size()) + ", should be at least 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); const ArraysDepths arrays_depths = getArraysDepths(arguments); /// Return type is the array of the depth as the maximum effective depth of arguments, containing UInt32. DataTypePtr type = std::make_shared(); for (DepthType i = 0; i < arrays_depths.max_array_depth; ++i) type = std::make_shared(type); return type; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; private: /// Initially allocate a piece of memory for 512 elements. NOTE: This is just a guess. static constexpr size_t INITIAL_SIZE_DEGREE = 9; void executeMethodImpl( const std::vector & offsets_by_depth, const ColumnRawPtrs & columns, const ArraysDepths & arrays_depths, ColumnUInt32::Container & res_values); }; /// Hash a set of keys into a UInt128 value. static inline UInt128 ALWAYS_INLINE hash128depths(const std::vector & indices, const ColumnRawPtrs & key_columns) { UInt128 key; SipHash hash; for (size_t j = 0, keys_size = key_columns.size(); j < keys_size; ++j) { // Debug: const auto & field = (*key_columns[j])[indices[j]]; DUMP(j, indices[j], field); key_columns[j]->updateHashWithValue(indices[j], hash); } hash.get128(key.low, key.high); return key; } template void FunctionArrayEnumerateRankedExtended::executeImpl( Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) { size_t num_arguments = arguments.size(); ColumnRawPtrs data_columns; Columns array_holders; ColumnPtr offsets_column; ColumnsWithTypeAndName args; for (size_t i = 0; i < arguments.size(); ++i) args.emplace_back(block.getByPosition(arguments[i])); const ArraysDepths arrays_depths = getArraysDepths(args); /// If the column is Array - return it. If the const Array - materialize it, keep ownership and return. auto get_array_column = [&](const auto & column) -> const DB::ColumnArray * { const ColumnArray * array = checkAndGetColumn(column); if (!array) { const ColumnConst * const_array = checkAndGetColumnConst(column); if (!const_array) return nullptr; array_holders.emplace_back(const_array->convertToFullColumn()); array = checkAndGetColumn(array_holders.back().get()); } return array; }; std::vector offsets_by_depth; std::vector offsetsptr_by_depth; size_t array_num = 0; for (size_t i = 0; i < num_arguments; ++i) { const auto * array = get_array_column(block.getByPosition(arguments[i]).column.get()); if (!array) continue; if (array_num == 0) // TODO check with prev { offsets_by_depth.emplace_back(&array->getOffsets()); offsetsptr_by_depth.emplace_back(array->getOffsetsPtr()); } else { if (*offsets_by_depth[0] != array->getOffsets()) { throw Exception( "Lengths and effective depths of all arrays passed to " + getName() + " must be equal.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); } } DepthType col_depth = 1; for (; col_depth < arrays_depths.depths[array_num]; ++col_depth) { auto sub_array = get_array_column(&array->getData()); if (sub_array) array = sub_array; if (!sub_array) break; if (offsets_by_depth.size() <= col_depth) { offsets_by_depth.emplace_back(&array->getOffsets()); offsetsptr_by_depth.emplace_back(array->getOffsetsPtr()); } else { if (*offsets_by_depth[col_depth] != array->getOffsets()) { throw Exception( "Lengths and effective depths of all arrays passed to " + getName() + " must be equal.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); } } } if (col_depth < arrays_depths.depths[array_num]) { throw Exception( getName() + ": Passed array number " + std::to_string(array_num) + " depth (" + std::to_string(arrays_depths.depths[array_num]) + ") is more than the actual array depth (" + std::to_string(col_depth) + ").", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); } auto * array_data = &array->getData(); data_columns.emplace_back(array_data); ++array_num; } if (offsets_by_depth.empty()) throw Exception("No arrays passed to function " + getName(), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto res_nested = ColumnUInt32::create(); ColumnUInt32::Container & res_values = res_nested->getData(); res_values.resize(offsets_by_depth[arrays_depths.max_array_depth - 1]->back()); executeMethodImpl(offsets_by_depth, data_columns, arrays_depths, res_values); ColumnPtr result_nested_array = std::move(res_nested); for (int depth = arrays_depths.max_array_depth - 1; depth >= 0; --depth) result_nested_array = ColumnArray::create(std::move(result_nested_array), offsetsptr_by_depth[depth]); block.getByPosition(result).column = result_nested_array; } /* (2, [[1,2,3],[2,2,1],[3]], 2, [4,5,6], 1) ; 1 2 3; 2 2 1; 3 4 5 6 ; 4 4 4; 5 5 5; 6 <- (1, [[1,2,3],[2,2,1],[3]], 1, [4,5,6], 1) ;[1,2,3] [2,2,1] [3] 4 5 6 ;4 5 6 <- (1, [[1,2,3],[2,2,1],[3]], 1, [4,5,6], 0) ;[1,2,3] [2,2,1] [3] 4 5 6 ;[4,5,6] [4,5,6] [4,5,6] <- . - get data ; - clean index (1, [[[1,2,3],[1,2,3],[1,2,3]],[[1,2,3],[1,2,3],[1,2,3]],[[1,2]]], 1) ;. . . (1, [[[1,2,3],[1,2,3],[1,2,3]],[[1,2,3],[1,2,3],[1,2,3]],[[1,2]]], 2) ; . . . . . . . (2, [[[1,2,3],[1,2,3],[1,2,3]],[[1,2,3],[1,2,3],[1,2,3]],[[1,2]]], 2) ; . . . ; . . . ; . (1, [[[1,2,3],[1,2,3],[1,2,3]],[[1,2,3],[1,2,3],[1,2,3]],[[1,2]]], 3) ; . . . . . . . . . . . . . . . . . . . . (2, [[[1,2,3],[1,2,3],[1,2,3]],[[1,2,3],[1,2,3],[1,2,3]],[[1,2]]], 3) ; . . . . . . . . . ; . . . . . . . . . ; . . (3, [[[1,2,3],[1,2,3],[1,2,3]],[[1,2,3],[1,2,3],[1,2,3]],[[1,2]]], 3) ; . . . ; . . . ; . . . ; . . . ; . . . ; . . . ; . . */ template void FunctionArrayEnumerateRankedExtended::executeMethodImpl( const std::vector & offsets_by_depth, const ColumnRawPtrs & columns, const ArraysDepths & arrays_depths, ColumnUInt32::Container & res_values) { /// Offsets at the depth we want to look. const size_t current_offset_depth = arrays_depths.max_array_depth; const auto & offsets = *offsets_by_depth[current_offset_depth - 1]; ColumnArray::Offset prev_off = 0; using Map = ClearableHashMap< UInt128, UInt32, UInt128TrivialHash, HashTableGrower, HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>; Map indices; std::vector indices_by_depth(arrays_depths.max_array_depth); std::vector current_offset_n_by_depth(arrays_depths.max_array_depth); UInt32 rank = 0; std::vector columns_indices(columns.size()); for (size_t off : offsets) { bool want_clear = false; /// For each element at the depth we want to look. for (size_t j = prev_off; j < off; ++j) { for (size_t col_n = 0; col_n < columns.size(); ++col_n) columns_indices[col_n] = indices_by_depth[arrays_depths.depths[col_n] - 1]; auto hash = hash128depths(columns_indices, columns); if constexpr (std::is_same_v) { auto idx = ++indices[hash]; res_values[j] = idx; } else // FunctionArrayEnumerateDenseRanked { auto idx = indices[hash]; if (!idx) { idx = ++rank; indices[hash] = idx; } res_values[j] = idx; } // Debug: DUMP(off, prev_off, j, columns_indices, res_values[j], columns); for (int depth = current_offset_depth - 1; depth >= 0; --depth) { ++indices_by_depth[depth]; if (indices_by_depth[depth] == (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]]) { if (static_cast(arrays_depths.clear_depth) == depth + 1) want_clear = true; ++current_offset_n_by_depth[depth]; } else { break; } } } if (want_clear) { want_clear = false; indices.clear(); rank = 0; } prev_off = off; } } }