#include #include #include #include #include #include #include #include #include #include namespace DB { template static void numWidthVector(const PaddedPODArray & a, PaddedPODArray & c) { size_t size = a.size(); for (size_t i = 0; i < size; ++i) if (a[i] >= 0) c[i] = a[i] ? 1 + log10(a[i]) : 1; else if (std::is_signed::value && a[i] == std::numeric_limits::min()) c[i] = 2 + log10(std::numeric_limits::max()); else c[i] = 2 + log10(-a[i]); } template static void numWidthConstant(T a, UInt64 & c) { if (a >= 0) c = a ? 1 + log10(a) : 1; else if (std::is_signed::value && a == std::numeric_limits::min()) c = 2 + log10(std::numeric_limits::max()); else c = 2 + log10(-a); } inline UInt64 floatWidth(const double x) { DoubleConverter::BufferType buffer; double_conversion::StringBuilder builder{buffer, sizeof(buffer)}; const auto result = DoubleConverter::instance().ToShortest(x, &builder); if (!result) throw Exception("Cannot print double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER); return builder.position(); } inline UInt64 floatWidth(const float x) { DoubleConverter::BufferType buffer; double_conversion::StringBuilder builder{buffer, sizeof(buffer)}; const auto result = DoubleConverter::instance().ToShortestSingle(x, &builder); if (!result) throw Exception("Cannot print float number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER); return builder.position(); } template static void floatWidthVector(const PaddedPODArray & a, PaddedPODArray & c) { size_t size = a.size(); for (size_t i = 0; i < size; ++i) c[i] = floatWidth(a[i]); } template static void floatWidthConstant(T a, UInt64 & c) { c = floatWidth(a); } template <> inline void numWidthVector(const PaddedPODArray & a, PaddedPODArray & c) { floatWidthVector(a, c); } template <> inline void numWidthVector(const PaddedPODArray & a, PaddedPODArray & c) { floatWidthVector(a, c); } template <> inline void numWidthConstant(Float64 a, UInt64 & c) { floatWidthConstant(a, c); } template <> inline void numWidthConstant(Float32 a, UInt64 & c) { floatWidthConstant(a, c); } static inline void stringWidthVector(const ColumnString::Chars_t & data, const ColumnString::Offsets_t & offsets, PaddedPODArray & res) { size_t size = offsets.size(); size_t prev_offset = 0; for (size_t i = 0; i < size; ++i) { res[i] = stringWidth(&data[prev_offset], &data[offsets[i] - 1]); prev_offset = offsets[i]; } } static inline void stringWidthFixedVector(const ColumnString::Chars_t & data, size_t n, PaddedPODArray & res) { size_t size = data.size() / n; for (size_t i = 0; i < size; ++i) res[i] = stringWidth(&data[i * n], &data[(i + 1) * n]); } namespace VisibleWidth { template static bool executeConstNumber(Block & block, const ColumnPtr & column, size_t result) { if (const ColumnConst * col = typeid_cast *>(column.get())) { UInt64 res = 0; numWidthConstant(col->getData(), res); block.getByPosition(result).column = std::make_shared(column->size(), res); return true; } else return false; } template static bool executeNumber(Block & block, const ColumnPtr & column, size_t result) { if (const ColumnVector * col = typeid_cast *>(column.get())) { auto res = std::make_shared(column->size()); block.getByPosition(result).column = res; numWidthVector(col->getData(), res->getData()); return true; } else return false; } template static bool executeEnum(Block & block, const DataTypePtr & type_ptr, const ColumnPtr & column, const size_t result) { if (const auto type = typeid_cast(type_ptr.get())) { if (const auto col = typeid_cast(column.get())) { const auto res = std::make_shared(col->size()); block.getByPosition(result).column = res; const auto & in = col->getData(); auto & out = res->getData(); for (const auto & idx_num : ext::enumerate(in)) { StringRef name = type->getNameForValue(idx_num.second); out[idx_num.first] = stringWidth( reinterpret_cast(name.data), reinterpret_cast(name.data) + name.size); } return true; } else if (const auto col = typeid_cast(column.get())) { StringRef name = type->getNameForValue(col->getData()); block.getByPosition(result).column = std::make_shared( col->size(), stringWidth( reinterpret_cast(name.data), reinterpret_cast(name.data) + name.size)); return true; } } return false; } } void FunctionVisibleWidth::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) { auto & element = block.getByPosition(arguments[0]); auto & res_element = block.getByPosition(result); auto & res_col = res_element.column; size_t row_count = block.rowsInFirstColumn(); if (element.column->isNull()) { /// The input column has the Null type. res_col = std::make_shared(row_count, strlen(NullSymbol::Escaped::name)); } else if (element.column->isNullable()) { /// Perform visibleWidth on a block that holds the nested column /// of the input column. auto & nullable_col = static_cast(*element.column); auto & nested_col = nullable_col.getNestedColumn(); auto & nullable_type = static_cast(*element.type); auto & nested_type = nullable_type.getNestedType(); Block block_with_nested_col = { { nested_col, nested_type, element.name }, { nullptr, res_element.type, "" } }; perform(block_with_nested_col, {0, 1}, 1); /// Create the result. If any row of the input column holds a NULL value, /// we assign the corresponding row of the result the length of the NULL /// symbol. res_col = std::make_shared(row_count); auto & res_data = static_cast(*res_col).getData(); const auto & src = static_cast( *block_with_nested_col.unsafeGetByPosition(1).column ).getData(); for (size_t row = 0; row < row_count; ++row) { if (nullable_col.isNullAt(row)) res_data[row] = strlen(NullSymbol::Escaped::name); else res_data[row] = src[row]; } } else { /// The input column has an ordinary type. perform(block, arguments, result); } } void FunctionVisibleWidth::perform(Block & block, const ColumnNumbers & arguments, size_t result) { const ColumnPtr column = block.getByPosition(arguments[0]).column; const DataTypePtr type = block.getByPosition(arguments[0]).type; size_t rows = column->size(); if (typeid_cast(type.get())) { block.getByPosition(result).column = std::make_shared(rows, strlen("0000-00-00")); } else if (typeid_cast(type.get())) { block.getByPosition(result).column = std::make_shared(rows, strlen("0000-00-00 00:00:00")); } else if (VisibleWidth::executeEnum(block, type, column, result) || VisibleWidth::executeEnum(block, type, column, result)) { } else if (VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeConstNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result) || VisibleWidth::executeNumber(block, column, result)) { } else if (const ColumnString * col = typeid_cast(column.get())) { auto res = std::make_shared(rows); block.getByPosition(result).column = res; stringWidthVector(col->getChars(), col->getOffsets(), res->getData()); } else if (const ColumnFixedString * col = typeid_cast(column.get())) { auto res = std::make_shared(rows); block.getByPosition(result).column = res; stringWidthFixedVector(col->getChars(), col->getN(), res->getData()); } else if (const ColumnConstString * col = typeid_cast(column.get())) { UInt64 res = 0; block.getByPosition(result).column = std::make_shared(rows, res); stringWidthConstant(col->getData(), res); } else if (const ColumnArray * col = typeid_cast(column.get())) { /// Вычисляем видимую ширину для значений массива. Block nested_block; ColumnWithTypeAndName nested_values; nested_values.type = typeid_cast(*type).getNestedType(); nested_values.column = col->getDataPtr(); nested_block.insert(nested_values); ColumnWithTypeAndName nested_result; nested_result.type = std::make_shared(); nested_block.insert(nested_result); executeImpl(nested_block, {0}, 1); /// Теперь суммируем и кладём в результат. auto res = std::make_shared(rows); block.getByPosition(result).column = res; ColumnUInt64::Container_t & vec = res->getData(); /// If the elements of the array are nullable, we have to check whether /// an element is a NULL for it is not surrounded by a pair of quotes. const PaddedPODArray * null_map = nullptr; if (nested_values.type->isNullable()) { const auto & nullable_col = static_cast(col->getData()); null_map = &static_cast(*nullable_col.getNullValuesByteMap()).getData(); } const auto & observed_type = DataTypeTraits::removeNullable(nested_values.type); size_t additional_symbols = 0; /// Кавычки. if (typeid_cast(observed_type.get()) || typeid_cast(observed_type.get()) || typeid_cast(observed_type.get()) || typeid_cast(observed_type.get())) additional_symbols = 2; if (ColumnUInt64 * nested_result_column = typeid_cast(nested_block.getByPosition(1).column.get())) { ColumnUInt64::Container_t & nested_res = nested_result_column->getData(); size_t j = 0; for (size_t i = 0; i < rows; ++i) { /** Если пустой массив - то два символа: []; * если непустой - то сначала один символ [, и по одному лишнему символу на значение: , или ]. */ vec[i] = (j == col->getOffsets()[i]) ? 2 : 1; for (; j < col->getOffsets()[i]; ++j) { size_t effective_additional_symbols; if ((null_map != nullptr) && ((*null_map)[j] == 1)) { /// The NULL value is not quoted. effective_additional_symbols = 0; } else effective_additional_symbols = additional_symbols; vec[i] += 1 + effective_additional_symbols + nested_res[j]; } } } else if (ColumnConstUInt64 * nested_result_column = typeid_cast(nested_block.getByPosition(1).column.get())) { size_t j = 0; for (size_t i = 0; i < rows; ++i) { size_t width = 0; for (; j < col->getOffsets()[i]; ++j) { size_t effective_additional_symbols; if ((null_map != nullptr) && ((*null_map)[j] == 1)) { /// The NULL value is not quoted. effective_additional_symbols = 0; } else effective_additional_symbols = additional_symbols; width += 1 + effective_additional_symbols + nested_result_column->getData(); } vec[i] = 1 + std::max(static_cast(1), width); } } } else if (const ColumnTuple * col = typeid_cast(column.get())) { /// Посчитаем видимую ширину для каждого вложенного столбца по отдельности, и просуммируем. Block nested_block = col->getData(); size_t columns = nested_block.columns(); FunctionPlus func_plus; for (size_t i = 0; i < columns; ++i) { nested_block.getByPosition(i).type = static_cast(*type).getElements()[i]; /** nested_block будет состоять из следующих столбцов: * x1, x2, x3... , width1, width2, width1 + width2, width3, width1 + width2 + width3, ... */ ColumnWithTypeAndName nested_result; nested_result.type = std::make_shared(); nested_block.insert(nested_result); ColumnNumbers nested_argument_numbers(1, i); execute(nested_block, nested_argument_numbers, nested_block.columns() - 1); if (i != 0) { ColumnWithTypeAndName plus_result; plus_result.type = std::make_shared(); nested_block.insert(plus_result); ColumnNumbers plus_argument_numbers(2); plus_argument_numbers[0] = nested_block.columns() - 3; plus_argument_numbers[1] = nested_block.columns() - 2; func_plus.execute(nested_block, plus_argument_numbers, nested_block.columns() - 1); } } /// Прибавим ещё количество символов на кавычки и запятые. size_t additional_symbols = columns - 1; /// Запятые. for (size_t i = 0; i < columns; ++i) { if (typeid_cast(nested_block.getByPosition(i).type.get()) || typeid_cast(nested_block.getByPosition(i).type.get()) || typeid_cast(nested_block.getByPosition(i).type.get()) || typeid_cast(nested_block.getByPosition(i).type.get())) additional_symbols += 2; /// Кавычки. } ColumnPtr & nested_result_column = nested_block.getByPosition(nested_block.columns() - 1).column; if (nested_result_column->isConst()) { ColumnConstUInt64 & nested_result_column_const = typeid_cast(*nested_result_column); if (nested_result_column_const.size()) nested_result_column_const.getData() += 2 + additional_symbols; } else { ColumnUInt64 & nested_result_column_vec = typeid_cast(*nested_result_column); ColumnUInt64::Container_t & nested_res = nested_result_column_vec.getData(); for (size_t i = 0; i < rows; ++i) nested_res[i] += 2 + additional_symbols; } block.getByPosition(result).column = nested_result_column; } else if (typeid_cast(column.get()) || typeid_cast(column.get())) { String s; { WriteBufferFromString wb(s); type->serializeTextEscaped(*column->cut(0, 1)->convertToFullColumnIfConst(), 0, wb); } block.getByPosition(result).column = std::make_shared(rows, s.size()); } else if (typeid_cast(column.get())) { /** Return obviously wrong (arbitary) value for states of aggregate functions. * Result of visibleWidth is used for presentation purposes, * and state of aggregate function is presented as unreadable sequence of bytes, * so using wrong calculation of its displayed width don't make presentation much worse. */ block.getByPosition(result).column = std::make_shared(rows, 10); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } } namespace DB { void registerFunctionsMiscellaneous(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction>(); factory.registerFunction>(); factory.registerFunction>(); factory.registerFunction>(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); } }