diff --git a/base/base/bit_cast.h b/base/base/bit_cast.h index d1246b45590..b2b6915764d 100644 --- a/base/base/bit_cast.h +++ b/base/base/bit_cast.h @@ -5,8 +5,9 @@ #include -/** \brief Returns value `from` converted to type `To` while retaining bit representation. - * `To` and `From` must satisfy `CopyConstructible`. +/** Returns value `from` converted to type `To` while retaining bit representation. + * `To` and `From` must satisfy `CopyConstructible`. + * In contrast to std::bit_cast can cast types of different width. */ template std::decay_t bit_cast(const From & from) @@ -15,13 +16,3 @@ std::decay_t bit_cast(const From & from) memcpy(static_cast(&res), &from, std::min(sizeof(res), sizeof(from))); return res; } - -/** \brief Returns value `from` converted to type `To` while retaining bit representation. - * `To` and `From` must satisfy `CopyConstructible`. - */ -template -std::decay_t safe_bit_cast(const From & from) -{ - static_assert(sizeof(To) == sizeof(From), "bit cast on types of different width"); - return bit_cast(from); -} diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index 9b404e7c5b7..5e231108bed 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -137,9 +137,41 @@ Field QueryFuzzer::fuzzField(Field field) break; } } - else if (type == Field::Types::Array || type == Field::Types::Tuple) + else if (type == Field::Types::Array) { - auto & arr = field.reinterpret(); + auto & arr = field.get(); + + if (fuzz_rand() % 5 == 0 && !arr.empty()) + { + size_t pos = fuzz_rand() % arr.size(); + arr.erase(arr.begin() + pos); + std::cerr << "erased\n"; + } + + if (fuzz_rand() % 5 == 0) + { + if (!arr.empty()) + { + size_t pos = fuzz_rand() % arr.size(); + arr.insert(arr.begin() + pos, fuzzField(arr[pos])); + std::cerr << fmt::format("inserted (pos {})\n", pos); + } + else + { + arr.insert(arr.begin(), getRandomField(0)); + std::cerr << "inserted (0)\n"; + } + + } + + for (auto & element : arr) + { + element = fuzzField(element); + } + } + else if (type == Field::Types::Tuple) + { + auto & arr = field.get(); if (fuzz_rand() % 5 == 0 && !arr.empty()) { diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index 8420441aac8..53763e94b0d 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -1,4 +1,5 @@ #pragma once + #include #include #include @@ -7,16 +8,17 @@ #include #include #include +#include #include #include #include #include -#include +#include +#include #include -#include "Columns/ColumnConst.h" namespace DB @@ -305,17 +307,52 @@ size_t ColumnUnique::getNullValueIndex() const return 0; } + +namespace +{ + class FieldVisitorGetData : public StaticVisitor<> + { + public: + StringRef res; + + [[noreturn]] static void throwUnsupported() + { + throw Exception("Unsupported field type", ErrorCodes::LOGICAL_ERROR); + } + + [[noreturn]] void operator() (const Null &) { throwUnsupported(); } + [[noreturn]] void operator() (const Array &) { throwUnsupported(); } + [[noreturn]] void operator() (const Tuple &) { throwUnsupported(); } + [[noreturn]] void operator() (const Map &) { throwUnsupported(); } + [[noreturn]] void operator() (const Object &) { throwUnsupported(); } + [[noreturn]] void operator() (const AggregateFunctionStateData &) { throwUnsupported(); } + void operator() (const String & x) { res = {x.data(), x.size()}; } + void operator() (const UInt64 & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const UInt128 & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const UInt256 & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const Int64 & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const Int128 & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const Int256 & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const UUID & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const Float64 & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const DecimalField & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const DecimalField & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const DecimalField & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const DecimalField & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + void operator() (const bool & x) { res = {reinterpret_cast(&x), sizeof(x)}; } + }; +} + + template size_t ColumnUnique::uniqueInsert(const Field & x) { if (x.isNull()) return getNullValueIndex(); - if (valuesHaveFixedSize()) - return uniqueInsertData(&x.reinterpret(), size_of_value_if_fixed); - - const auto & val = x.get(); - return uniqueInsertData(val.data(), val.size()); + FieldVisitorGetData visitor; + applyVisitor(visitor, x); + return uniqueInsertData(visitor.res.data, visitor.res.size); } template diff --git a/src/Common/FieldVisitorConvertToNumber.h b/src/Common/FieldVisitorConvertToNumber.h index 92da0f89844..466d312406e 100644 --- a/src/Common/FieldVisitorConvertToNumber.h +++ b/src/Common/FieldVisitorConvertToNumber.h @@ -94,21 +94,7 @@ public: T operator() (const DecimalField & x) const { if constexpr (std::is_floating_point_v) - return x.getValue(). template convertTo() / x.getScaleMultiplier(). template convertTo(); - else if constexpr (std::is_same_v) - { - if constexpr (sizeof(U) < 16) - { - return UInt128(0, (x.getValue() / x.getScaleMultiplier()).value); - } - else if constexpr (sizeof(U) == 16) - { - auto tmp = (x.getValue() / x.getScaleMultiplier()).value; - return UInt128(tmp >> 64, UInt64(tmp)); - } - else - throw Exception("No conversion to old UInt128 from " + demangle(typeid(U).name()), ErrorCodes::NOT_IMPLEMENTED); - } + return x.getValue().template convertTo() / x.getScaleMultiplier().template convertTo(); else return (x.getValue() / x.getScaleMultiplier()). template convertTo(); } @@ -134,4 +120,3 @@ public: }; } - diff --git a/src/Common/FieldVisitorSum.cpp b/src/Common/FieldVisitorSum.cpp index bc996ae2298..2c404c33177 100644 --- a/src/Common/FieldVisitorSum.cpp +++ b/src/Common/FieldVisitorSum.cpp @@ -15,7 +15,7 @@ FieldVisitorSum::FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {} bool FieldVisitorSum::operator() (Int64 & x) const { return this->operator()(reinterpret_cast(x)); } bool FieldVisitorSum::operator() (UInt64 & x) const { - x += rhs.reinterpret(); + x += applyVisitor(FieldVisitorConvertToNumber(), rhs); return x != 0; } diff --git a/src/Common/FieldVisitorSum.h b/src/Common/FieldVisitorSum.h index cd8777e7bfb..c28e2058b05 100644 --- a/src/Common/FieldVisitorSum.h +++ b/src/Common/FieldVisitorSum.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -41,7 +42,7 @@ public: requires is_big_int_v bool operator() (T & x) const { - x += rhs.reinterpret(); + x += applyVisitor(FieldVisitorConvertToNumber(), rhs); return x != T(0); } }; diff --git a/src/Core/Field.h b/src/Core/Field.h index 47241aa1f3e..2924ed9f174 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -425,16 +425,6 @@ public: bool isNegativeInfinity() const { return which == Types::Null && get().isNegativeInfinity(); } bool isPositiveInfinity() const { return which == Types::Null && get().isPositiveInfinity(); } - template - T & reinterpret(); - - template - const T & reinterpret() const - { - auto * mutable_this = const_cast *>(this); - return mutable_this->reinterpret(); - } - template bool tryGet(T & result) { const Types::Which requested = TypeToEnum>::value; @@ -552,7 +542,7 @@ public: case Types::Float64: { // Compare as UInt64 so that NaNs compare as equal. - return reinterpret() == rhs.reinterpret(); + return std::bit_cast(get()) == std::bit_cast(rhs.get()); } case Types::UUID: return get() == rhs.get(); case Types::String: return get() == rhs.get(); @@ -843,30 +833,6 @@ auto & Field::safeGet() } -template -T & Field::reinterpret() -{ - assert(which != Types::String); // See specialization for char - using ValueType = std::decay_t; - ValueType * MAY_ALIAS ptr = reinterpret_cast(&storage); - return *ptr; -} - -// Specialize reinterpreting to char (used in ColumnUnique) to make sure Strings are reinterpreted correctly -// inline to avoid multiple definitions -template <> -inline char & Field::reinterpret() -{ - if (which == Types::String) - { - // For String we want to return a pointer to the data, not the start of the class - // as the layout of std::string depends on the STD version and options - char * ptr = reinterpret_cast(&storage)->data(); - return *ptr; - } - return *reinterpret_cast(&storage); -} - template Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t) //-V730 { diff --git a/src/Functions/transform.cpp b/src/Functions/transform.cpp index 3337e8d40a8..62be535be85 100644 --- a/src/Functions/transform.cpp +++ b/src/Functions/transform.cpp @@ -1,6 +1,7 @@ #include #include +#include #include #include #include @@ -920,8 +921,7 @@ private: ColumnString::Offset current_dst_default_offset = 0; for (size_t i = 0; i < size; ++i) { - Field key = src[i]; - const auto * it = table.find(key.reinterpret()); + const auto * it = table.find(bit_cast(src[i])); StringRef ref; if (it) @@ -1081,6 +1081,22 @@ private: mutable Cache cache; + + static UInt64 bitCastToUInt64(const Field & x) + { + switch (x.getType()) + { + case Field::Types::UInt64: return x.get(); + case Field::Types::Int64: return x.get(); + case Field::Types::Float64: return std::bit_cast(x.get()); + case Field::Types::Bool: return x.get(); + case Field::Types::Decimal32: return x.get>().getValue(); + case Field::Types::Decimal64: return x.get>().getValue(); + default: + throw Exception("Unexpected type in function 'transform'", ErrorCodes::BAD_ARGUMENTS); + } + } + /// Can be called from different threads. It works only on the first call. void initialize(const Array & from, const Array & to, const ColumnsWithTypeAndName & arguments) const { @@ -1151,20 +1167,8 @@ private: if (key.isNull()) continue; - // Field may be of Float type, but for the purpose of bitwise - // equality we can treat them as UInt64, hence the reinterpret(). - if (to[0].getType() ==Field::Types::Decimal32) - { - table[key.reinterpret()] = (*used_to)[i].reinterpret(); - } - else if (to[0].getType() ==Field::Types::Decimal64) - { - table[key.reinterpret()] = (*used_to)[i].reinterpret(); - } - else - { - table[key.reinterpret()] = (*used_to)[i].reinterpret(); - } + /// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64 + table[bitCastToUInt64(key)] = bitCastToUInt64((*used_to)[i]); } } else @@ -1179,7 +1183,7 @@ private: const String & str_to = to[i].get(); StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; - table[key.reinterpret()] = ref; + table[bitCastToUInt64(key)] = ref; } } } @@ -1193,7 +1197,7 @@ private: { const String & str_from = from[i].get(); StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; - table[ref] = (*used_to)[i].reinterpret(); + table[ref] = bitCastToUInt64((*used_to)[i]); } } else diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index dd23ad69ae2..4e7562ef451 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -6,26 +6,24 @@ #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 @@ -223,7 +221,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID && (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDate() || which_from_type.isDate32() || which_from_type.isDateTime() || which_from_type.isDateTime64())) { const auto scale = static_cast(type).getScale(); - const auto decimal_value = DecimalUtils::decimalFromComponents(src.reinterpret(), 0, scale); + const auto decimal_value = DecimalUtils::decimalFromComponents(applyVisitor(FieldVisitorConvertToNumber(), src), 0, scale); return Field(DecimalField(decimal_value, scale)); } } diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 1de2acbb3b9..2b20a4a4bc5 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -471,9 +471,8 @@ size_t ORCBlockOutputFormat::getMaxColumnSize(Chunk & chunk) size_t columns_num = chunk.getNumColumns(); size_t max_column_size = 0; for (size_t i = 0; i != columns_num; ++i) - { max_column_size = std::max(max_column_size, getColumnSize(*chunk.getColumns()[i], data_types[i])); - } + return max_column_size; } @@ -481,18 +480,22 @@ void ORCBlockOutputFormat::consume(Chunk chunk) { if (!writer) prepareWriter(); + size_t columns_num = chunk.getNumColumns(); size_t rows_num = chunk.getNumRows(); + /// getMaxColumnSize is needed to write arrays. /// The size of the batch must be no less than total amount of array elements. - ORC_UNIQUE_PTR batch = writer->createRowBatch(getMaxColumnSize(chunk)); + std::unique_ptr batch = writer->createRowBatch(getMaxColumnSize(chunk)); orc::StructVectorBatch & root = dynamic_cast(*batch); + auto columns = chunk.detachColumns(); for (auto & column : columns) column = recursiveRemoveLowCardinality(column); for (size_t i = 0; i != columns_num; ++i) writeColumn(*root.fields[i], *columns[i], data_types[i], nullptr); + root.numElements = rows_num; writer->add(*batch); } diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h index d4a19353915..6467f2148f5 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h @@ -8,11 +8,13 @@ #include #include + namespace DB { class WriteBuffer; + /// orc::Writer writes only in orc::OutputStream class ORCOutputStream : public orc::OutputStream { @@ -21,7 +23,7 @@ public: uint64_t getLength() const override; uint64_t getNaturalWriteSize() const override; - void write(const void* buf, size_t length) override; + void write(const void * buf, size_t length) override; void close() override {} const std::string& getName() const override { return name; } @@ -31,6 +33,7 @@ private: std::string name = "ORCOutputStream"; }; + class ORCBlockOutputFormat : public IOutputFormat { public: @@ -42,7 +45,7 @@ private: void consume(Chunk chunk) override; void finalizeImpl() override; - ORC_UNIQUE_PTR getORCType(const DataTypePtr & type); + std::unique_ptr getORCType(const DataTypePtr & type); /// ConvertFunc is needed for type UInt8, because firstly UInt8 (char8_t) must be /// converted to unsigned char (bugprone-signed-char-misuse in clang). @@ -75,8 +78,8 @@ private: const FormatSettings format_settings; ORCOutputStream output_stream; DataTypes data_types; - ORC_UNIQUE_PTR writer; - ORC_UNIQUE_PTR schema; + std::unique_ptr writer; + std::unique_ptr schema; orc::WriterOptions options; }; diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 9d1a075a63f..dc890910224 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -242,11 +242,21 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl DataTypePtr const_type; if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type)) { - if (const_value.getType() == Field::Types::UInt64 || const_value.getType() == Field::Types::Int64 || - const_value.getType() == Field::Types::Float64) + if (const_value.getType() == Field::Types::UInt64) { - /// Zero in all types is represented in memory the same way as in UInt64. - out.function = const_value.reinterpret() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + return true; + } + + if (const_value.getType() == Field::Types::Int64) + { + out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; + return true; + } + + if (const_value.getType() == Field::Types::Float64) + { + out.function = const_value.get() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true; } } diff --git a/tests/queries/0_stateless/02426_low_cardinality_fixed_string_insert_field.reference b/tests/queries/0_stateless/02426_low_cardinality_fixed_string_insert_field.reference new file mode 100644 index 00000000000..3bfced8d8bd --- /dev/null +++ b/tests/queries/0_stateless/02426_low_cardinality_fixed_string_insert_field.reference @@ -0,0 +1 @@ +4908278 diff --git a/tests/queries/0_stateless/02426_low_cardinality_fixed_string_insert_field.sh b/tests/queries/0_stateless/02426_low_cardinality_fixed_string_insert_field.sh new file mode 100755 index 00000000000..dc9f1ec8ed2 --- /dev/null +++ b/tests/queries/0_stateless/02426_low_cardinality_fixed_string_insert_field.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_LOCAL} --structure 'x LowCardinality(FixedString(2454139))' --input-format Values --output-format TSV --query "SELECT * FROM table" <<< '(1)' | wc -c