Remove trash from Field

This commit is contained in:
Alexey Milovidov 2022-09-18 05:16:08 +02:00
parent 7f68f2c7df
commit 791de6592b
10 changed files with 96 additions and 63 deletions

View File

@ -137,9 +137,41 @@ Field QueryFuzzer::fuzzField(Field field)
break; break;
} }
} }
else if (type == Field::Types::Array || type == Field::Types::Tuple) else if (type == Field::Types::Array)
{ {
auto & arr = field.reinterpret<FieldVector>(); auto & arr = field.get<Array>();
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<Tuple>();
if (fuzz_rand() % 5 == 0 && !arr.empty()) if (fuzz_rand() % 5 == 0 && !arr.empty())
{ {

View File

@ -94,21 +94,7 @@ public:
T operator() (const DecimalField<U> & x) const T operator() (const DecimalField<U> & x) const
{ {
if constexpr (std::is_floating_point_v<T>) if constexpr (std::is_floating_point_v<T>)
return x.getValue(). template convertTo<T>() / x.getScaleMultiplier(). template convertTo<T>(); return x.getValue().template convertTo<T>() / x.getScaleMultiplier().template convertTo<T>();
else if constexpr (std::is_same_v<T, UInt128>)
{
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);
}
else else
return (x.getValue() / x.getScaleMultiplier()). template convertTo<T>(); return (x.getValue() / x.getScaleMultiplier()). template convertTo<T>();
} }
@ -134,4 +120,3 @@ public:
}; };
} }

View File

@ -15,7 +15,7 @@ FieldVisitorSum::FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
bool FieldVisitorSum::operator() (Int64 & x) const { return this->operator()(reinterpret_cast<UInt64 &>(x)); } bool FieldVisitorSum::operator() (Int64 & x) const { return this->operator()(reinterpret_cast<UInt64 &>(x)); }
bool FieldVisitorSum::operator() (UInt64 & x) const bool FieldVisitorSum::operator() (UInt64 & x) const
{ {
x += rhs.reinterpret<UInt64>(); x += applyVisitor(FieldVisitorConvertToNumber<UInt64>(), rhs);
return x != 0; return x != 0;
} }

View File

@ -1,6 +1,7 @@
#pragma once #pragma once
#include <Common/FieldVisitors.h> #include <Common/FieldVisitors.h>
#include <Common/FieldVisitorConvertToNumber.h>
namespace DB namespace DB
@ -41,7 +42,7 @@ public:
requires is_big_int_v<T> requires is_big_int_v<T>
bool operator() (T & x) const bool operator() (T & x) const
{ {
x += rhs.reinterpret<T>(); x += applyVisitor(FieldVisitorConvertToNumber<T>(), rhs);
return x != T(0); return x != T(0);
} }
}; };

View File

@ -425,16 +425,6 @@ public:
bool isNegativeInfinity() const { return which == Types::Null && get<Null>().isNegativeInfinity(); } bool isNegativeInfinity() const { return which == Types::Null && get<Null>().isNegativeInfinity(); }
bool isPositiveInfinity() const { return which == Types::Null && get<Null>().isPositiveInfinity(); } bool isPositiveInfinity() const { return which == Types::Null && get<Null>().isPositiveInfinity(); }
template <typename T>
T & reinterpret();
template <typename T>
const T & reinterpret() const
{
auto * mutable_this = const_cast<std::decay_t<decltype(*this)> *>(this);
return mutable_this->reinterpret<T>();
}
template <typename T> bool tryGet(T & result) template <typename T> bool tryGet(T & result)
{ {
const Types::Which requested = TypeToEnum<std::decay_t<T>>::value; const Types::Which requested = TypeToEnum<std::decay_t<T>>::value;
@ -552,7 +542,7 @@ public:
case Types::Float64: case Types::Float64:
{ {
// Compare as UInt64 so that NaNs compare as equal. // Compare as UInt64 so that NaNs compare as equal.
return reinterpret<UInt64>() == rhs.reinterpret<UInt64>(); return std::bit_cast<UInt64>(get<Float64>()) == std::bit_cast<UInt64>(rhs.get<Float64>());
} }
case Types::UUID: return get<UUID>() == rhs.get<UUID>(); case Types::UUID: return get<UUID>() == rhs.get<UUID>();
case Types::String: return get<String>() == rhs.get<String>(); case Types::String: return get<String>() == rhs.get<String>();
@ -843,15 +833,6 @@ auto & Field::safeGet()
} }
template <typename T>
T & Field::reinterpret()
{
assert(which != Types::String);
using ValueType = std::decay_t<T>;
ValueType * MAY_ALIAS ptr = reinterpret_cast<ValueType *>(&storage);
return *ptr;
}
template <typename T> template <typename T>
Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t<T>) //-V730 Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t<T>) //-V730
{ {

View File

@ -920,8 +920,8 @@ private:
ColumnString::Offset current_dst_default_offset = 0; ColumnString::Offset current_dst_default_offset = 0;
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
{ {
Field key = src[i]; T key = src[i];
const auto * it = table.find(key.reinterpret<UInt64>()); const auto * it = table.find(key);
StringRef ref; StringRef ref;
if (it) if (it)
@ -1081,6 +1081,22 @@ private:
mutable Cache cache; mutable Cache cache;
static UInt64 bitCastToUInt64(const Field & x)
{
switch (x.getType())
{
case Field::Types::UInt64: return x.get<UInt64>();
case Field::Types::Int64: return x.get<Int64>();
case Field::Types::Float64: return std::bit_cast<UInt64>(x.get<Float64>());
case Field::Types::Bool: return x.get<bool>();
case Field::Types::Decimal32: return x.get<DecimalField<Decimal32>>().getValue();
case Field::Types::Decimal64: return x.get<DecimalField<Decimal64>>().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. /// 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 void initialize(const Array & from, const Array & to, const ColumnsWithTypeAndName & arguments) const
{ {
@ -1153,17 +1169,17 @@ private:
// Field may be of Float type, but for the purpose of bitwise // Field may be of Float type, but for the purpose of bitwise
// equality we can treat them as UInt64, hence the reinterpret(). // equality we can treat them as UInt64, hence the reinterpret().
if (to[0].getType() ==Field::Types::Decimal32) if (to[0].getType() == Field::Types::Decimal32)
{ {
table[key.reinterpret<Decimal32>()] = (*used_to)[i].reinterpret<Decimal32>(); table[key.get<Decimal32>().getValue()] = (*used_to)[i].get<Decimal32>().getValue();
} }
else if (to[0].getType() ==Field::Types::Decimal64) else if (to[0].getType() == Field::Types::Decimal64)
{ {
table[key.reinterpret<Decimal32>()] = (*used_to)[i].reinterpret<Decimal64>(); table[key.get<Decimal64>().getValue()] = (*used_to)[i].get<Decimal64>().getValue();
} }
else else
{ {
table[key.reinterpret<UInt64>()] = (*used_to)[i].reinterpret<UInt64>(); table[bitCastToUInt64(key)] = bitCastToUInt64((*used_to)[i]);
} }
} }
} }
@ -1179,7 +1195,7 @@ private:
const String & str_to = to[i].get<const String &>(); const String & str_to = to[i].get<const String &>();
StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1}; StringRef ref{cache.string_pool.insert(str_to.data(), str_to.size() + 1), str_to.size() + 1};
table[key.reinterpret<UInt64>()] = ref; table[bitCastToUInt64(key)] = ref;
} }
} }
} }
@ -1193,7 +1209,7 @@ private:
{ {
const String & str_from = from[i].get<const String &>(); const String & str_from = from[i].get<const String &>();
StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1}; StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
table[ref] = (*used_to)[i].reinterpret<UInt64>(); table[ref] = bitCastToUInt64((*used_to)[i]);
} }
} }
else else

View File

@ -18,14 +18,16 @@
#include <DataTypes/DataTypeUUID.h> #include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeLowCardinality.h> #include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Core/AccurateComparison.h> #include <Core/AccurateComparison.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Common/NaNUtils.h> #include <Common/NaNUtils.h>
#include <Common/FieldVisitorToString.h> #include <Common/FieldVisitorToString.h>
#include <Common/FieldVisitorConvertToNumber.h>
#include <Common/DateLUT.h> #include <Common/DateLUT.h>
#include <DataTypes/DataTypeAggregateFunction.h>
namespace DB namespace DB
@ -223,7 +225,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())) && (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<const DataTypeDateTime64 &>(type).getScale(); const auto scale = static_cast<const DataTypeDateTime64 &>(type).getScale();
const auto decimal_value = DecimalUtils::decimalFromComponents<DateTime64>(src.reinterpret<Int64>(), 0, scale); const auto decimal_value = DecimalUtils::decimalFromComponents<DateTime64>(applyVisitor(FieldVisitorConvertToNumber<Int64>(), src), 0, scale);
return Field(DecimalField<DateTime64>(decimal_value, scale)); return Field(DecimalField<DateTime64>(decimal_value, scale));
} }
} }

View File

@ -471,9 +471,8 @@ size_t ORCBlockOutputFormat::getMaxColumnSize(Chunk & chunk)
size_t columns_num = chunk.getNumColumns(); size_t columns_num = chunk.getNumColumns();
size_t max_column_size = 0; size_t max_column_size = 0;
for (size_t i = 0; i != columns_num; ++i) for (size_t i = 0; i != columns_num; ++i)
{
max_column_size = std::max(max_column_size, getColumnSize(*chunk.getColumns()[i], data_types[i])); max_column_size = std::max(max_column_size, getColumnSize(*chunk.getColumns()[i], data_types[i]));
}
return max_column_size; return max_column_size;
} }
@ -481,18 +480,22 @@ void ORCBlockOutputFormat::consume(Chunk chunk)
{ {
if (!writer) if (!writer)
prepareWriter(); prepareWriter();
size_t columns_num = chunk.getNumColumns(); size_t columns_num = chunk.getNumColumns();
size_t rows_num = chunk.getNumRows(); size_t rows_num = chunk.getNumRows();
/// getMaxColumnSize is needed to write arrays. /// getMaxColumnSize is needed to write arrays.
/// The size of the batch must be no less than total amount of array elements. /// The size of the batch must be no less than total amount of array elements.
ORC_UNIQUE_PTR<orc::ColumnVectorBatch> batch = writer->createRowBatch(getMaxColumnSize(chunk)); std::unique_ptr<orc::ColumnVectorBatch> batch = writer->createRowBatch(getMaxColumnSize(chunk));
orc::StructVectorBatch & root = dynamic_cast<orc::StructVectorBatch &>(*batch); orc::StructVectorBatch & root = dynamic_cast<orc::StructVectorBatch &>(*batch);
auto columns = chunk.detachColumns(); auto columns = chunk.detachColumns();
for (auto & column : columns) for (auto & column : columns)
column = recursiveRemoveLowCardinality(column); column = recursiveRemoveLowCardinality(column);
for (size_t i = 0; i != columns_num; ++i) for (size_t i = 0; i != columns_num; ++i)
writeColumn(*root.fields[i], *columns[i], data_types[i], nullptr); writeColumn(*root.fields[i], *columns[i], data_types[i], nullptr);
root.numElements = rows_num; root.numElements = rows_num;
writer->add(*batch); writer->add(*batch);
} }

View File

@ -8,11 +8,13 @@
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <orc/OrcFile.hh> #include <orc/OrcFile.hh>
namespace DB namespace DB
{ {
class WriteBuffer; class WriteBuffer;
/// orc::Writer writes only in orc::OutputStream /// orc::Writer writes only in orc::OutputStream
class ORCOutputStream : public orc::OutputStream class ORCOutputStream : public orc::OutputStream
{ {
@ -21,7 +23,7 @@ public:
uint64_t getLength() const override; uint64_t getLength() const override;
uint64_t getNaturalWriteSize() 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 {} void close() override {}
const std::string& getName() const override { return name; } const std::string& getName() const override { return name; }
@ -31,6 +33,7 @@ private:
std::string name = "ORCOutputStream"; std::string name = "ORCOutputStream";
}; };
class ORCBlockOutputFormat : public IOutputFormat class ORCBlockOutputFormat : public IOutputFormat
{ {
public: public:
@ -42,7 +45,7 @@ private:
void consume(Chunk chunk) override; void consume(Chunk chunk) override;
void finalizeImpl() override; void finalizeImpl() override;
ORC_UNIQUE_PTR<orc::Type> getORCType(const DataTypePtr & type); std::unique_ptr<orc::Type> getORCType(const DataTypePtr & type);
/// ConvertFunc is needed for type UInt8, because firstly UInt8 (char8_t) must be /// ConvertFunc is needed for type UInt8, because firstly UInt8 (char8_t) must be
/// converted to unsigned char (bugprone-signed-char-misuse in clang). /// converted to unsigned char (bugprone-signed-char-misuse in clang).
@ -75,8 +78,8 @@ private:
const FormatSettings format_settings; const FormatSettings format_settings;
ORCOutputStream output_stream; ORCOutputStream output_stream;
DataTypes data_types; DataTypes data_types;
ORC_UNIQUE_PTR<orc::Writer> writer; std::unique_ptr<orc::Writer> writer;
ORC_UNIQUE_PTR<orc::Type> schema; std::unique_ptr<orc::Type> schema;
orc::WriterOptions options; orc::WriterOptions options;
}; };

View File

@ -242,11 +242,21 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl
DataTypePtr const_type; DataTypePtr const_type;
if (KeyCondition::getConstant(node, block_with_constants, const_value, 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 || if (const_value.getType() == Field::Types::UInt64)
const_value.getType() == Field::Types::Float64)
{ {
/// Zero in all types is represented in memory the same way as in UInt64. out.function = const_value.get<UInt64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
out.function = const_value.reinterpret<UInt64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE; return true;
}
if (const_value.getType() == Field::Types::Int64)
{
out.function = const_value.get<Int64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
return true;
}
if (const_value.getType() == Field::Types::Float64)
{
out.function = const_value.get<Float64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
return true; return true;
} }
} }