mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Minor changes
This commit is contained in:
parent
cae9b25074
commit
447d7bb8cd
@ -1,7 +1,7 @@
|
||||
#include <Access/SettingsConstraints.h>
|
||||
#include <Access/AccessControlManager.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
@ -9,7 +9,8 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
||||
|
@ -2,8 +2,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Interpreters/castColumn.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include "AggregateFunctionFactory.h"
|
||||
|
@ -13,7 +13,7 @@
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorSum.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
#include <map>
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionTopK.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/FactoryHelpers.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <AggregateFunctions/Helpers.h>
|
||||
#include <AggregateFunctions/AggregateFunctionUniqUpTo.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <vector>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <IO/WriteBufferFromArena.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
129
src/Common/FieldVisitorConvertToNumber.h
Normal file
129
src/Common/FieldVisitorConvertToNumber.h
Normal file
@ -0,0 +1,129 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <common/demangle.h>
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_CONVERT_TYPE;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
/** Converts numeric value of any type to specified type. */
|
||||
template <typename T>
|
||||
class FieldVisitorConvertToNumber : public StaticVisitor<T>
|
||||
{
|
||||
public:
|
||||
T operator() (const Null &) const
|
||||
{
|
||||
throw Exception("Cannot convert NULL to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const String &) const
|
||||
{
|
||||
throw Exception("Cannot convert String to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const Array &) const
|
||||
{
|
||||
throw Exception("Cannot convert Array to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const Tuple &) const
|
||||
{
|
||||
throw Exception("Cannot convert Tuple to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const Map &) const
|
||||
{
|
||||
throw Exception("Cannot convert Map to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const UInt64 & x) const { return T(x); }
|
||||
T operator() (const Int64 & x) const { return T(x); }
|
||||
T operator() (const Int128 & x) const { return T(x); }
|
||||
T operator() (const UUID & x) const { return T(x.toUnderType()); }
|
||||
|
||||
T operator() (const Float64 & x) const
|
||||
{
|
||||
if constexpr (!std::is_floating_point_v<T>)
|
||||
{
|
||||
if (!isFinite(x))
|
||||
{
|
||||
/// When converting to bool it's ok (non-zero converts to true, NaN including).
|
||||
if (std::is_same_v<T, bool>)
|
||||
return true;
|
||||
|
||||
/// Conversion of infinite values to integer is undefined.
|
||||
throw Exception("Cannot convert infinite value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
else if (x > std::numeric_limits<T>::max() || x < std::numeric_limits<T>::lowest())
|
||||
{
|
||||
throw Exception("Cannot convert out of range floating point value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (std::is_same_v<Decimal256, T>)
|
||||
{
|
||||
return Int256(x);
|
||||
}
|
||||
else
|
||||
{
|
||||
return T(x);
|
||||
}
|
||||
}
|
||||
|
||||
T operator() (const UInt128 &) const
|
||||
{
|
||||
throw Exception("Cannot convert UInt128 to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
template <typename U>
|
||||
T operator() (const DecimalField<U> & x) const
|
||||
{
|
||||
if constexpr (std::is_floating_point_v<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
|
||||
return (x.getValue() / x.getScaleMultiplier()). template convertTo<T>();
|
||||
}
|
||||
|
||||
T operator() (const AggregateFunctionStateData &) const
|
||||
{
|
||||
throw Exception("Cannot convert AggregateFunctionStateData to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
template <typename U, typename = std::enable_if_t<is_big_int_v<U>> >
|
||||
T operator() (const U & x) const
|
||||
{
|
||||
if constexpr (IsDecimalNumber<T>)
|
||||
return static_cast<T>(static_cast<typename T::NativeType>(x));
|
||||
else if constexpr (std::is_same_v<T, UInt128>)
|
||||
throw Exception("No conversion to old UInt128 from " + demangle(typeid(U).name()), ErrorCodes::NOT_IMPLEMENTED);
|
||||
else
|
||||
return static_cast<T>(x);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
109
src/Common/FieldVisitorDump.cpp
Normal file
109
src/Common/FieldVisitorDump.cpp
Normal file
@ -0,0 +1,109 @@
|
||||
#include <Common/FieldVisitorDump.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
static inline String formatQuotedWithPrefix(T x, const char * prefix)
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
writeCString(prefix, wb);
|
||||
writeQuoted(x, wb);
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
|
||||
{
|
||||
writeChar('\'', buf);
|
||||
writeText(x.getValue(), x.getScale(), buf);
|
||||
writeChar('\'', buf);
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Null &) const { return "NULL"; }
|
||||
String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); }
|
||||
String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); }
|
||||
String FieldVisitorDump::operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); }
|
||||
String FieldVisitorDump::operator() (const DecimalField<Decimal32> & x) const { return formatQuotedWithPrefix(x, "Decimal32_"); }
|
||||
String FieldVisitorDump::operator() (const DecimalField<Decimal64> & x) const { return formatQuotedWithPrefix(x, "Decimal64_"); }
|
||||
String FieldVisitorDump::operator() (const DecimalField<Decimal128> & x) const { return formatQuotedWithPrefix(x, "Decimal128_"); }
|
||||
String FieldVisitorDump::operator() (const DecimalField<Decimal256> & x) const { return formatQuotedWithPrefix(x, "Decimal256_"); }
|
||||
String FieldVisitorDump::operator() (const UInt128 & x) const { return formatQuotedWithPrefix(x, "UInt128_"); }
|
||||
String FieldVisitorDump::operator() (const UInt256 & x) const { return formatQuotedWithPrefix(x, "UInt256_"); }
|
||||
String FieldVisitorDump::operator() (const Int128 & x) const { return formatQuotedWithPrefix(x, "Int128_"); }
|
||||
String FieldVisitorDump::operator() (const Int256 & x) const { return formatQuotedWithPrefix(x, "Int256_"); }
|
||||
String FieldVisitorDump::operator() (const UUID & x) const { return formatQuotedWithPrefix(x, "UUID_"); }
|
||||
|
||||
|
||||
String FieldVisitorDump::operator() (const String & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
writeQuoted(x, wb);
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Array & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << "Array_[";
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ']';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Tuple & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << "Tuple_(";
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ')';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Map & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << "Map_(";
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ')';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const AggregateFunctionStateData & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
wb << "AggregateFunctionState_(";
|
||||
writeQuoted(x.name, wb);
|
||||
wb << ", ";
|
||||
writeQuoted(x.data, wb);
|
||||
wb << ')';
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
33
src/Common/FieldVisitorDump.h
Normal file
33
src/Common/FieldVisitorDump.h
Normal file
@ -0,0 +1,33 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Print readable and unique text dump of field type and value. */
|
||||
class FieldVisitorDump : public StaticVisitor<String>
|
||||
{
|
||||
public:
|
||||
String operator() (const Null & x) const;
|
||||
String operator() (const UInt64 & x) const;
|
||||
String operator() (const UInt128 & x) const;
|
||||
String operator() (const UInt256 & x) const;
|
||||
String operator() (const Int64 & x) const;
|
||||
String operator() (const Int128 & x) const;
|
||||
String operator() (const Int256 & x) const;
|
||||
String operator() (const UUID & x) const;
|
||||
String operator() (const Float64 & x) const;
|
||||
String operator() (const String & x) const;
|
||||
String operator() (const Array & x) const;
|
||||
String operator() (const Tuple & x) const;
|
||||
String operator() (const Map & x) const;
|
||||
String operator() (const DecimalField<Decimal32> & x) const;
|
||||
String operator() (const DecimalField<Decimal64> & x) const;
|
||||
String operator() (const DecimalField<Decimal128> & x) const;
|
||||
String operator() (const DecimalField<Decimal256> & x) const;
|
||||
String operator() (const AggregateFunctionStateData & x) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
149
src/Common/FieldVisitorHash.cpp
Normal file
149
src/Common/FieldVisitorHash.cpp
Normal file
@ -0,0 +1,149 @@
|
||||
#include <Common/FieldVisitorHash.h>
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
FieldVisitorHash::FieldVisitorHash(SipHash & hash_) : hash(hash_) {}
|
||||
|
||||
void FieldVisitorHash::operator() (const Null &) const
|
||||
{
|
||||
UInt8 type = Field::Types::Null;
|
||||
hash.update(type);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UInt64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UInt64;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UInt128 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UInt128;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Int64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Int64;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Int128 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Int128;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UUID & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UUID;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Float64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Float64;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const String & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::String;
|
||||
hash.update(type);
|
||||
hash.update(x.size());
|
||||
hash.update(x.data(), x.size());
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Tuple & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Tuple;
|
||||
hash.update(type);
|
||||
hash.update(x.size());
|
||||
|
||||
for (const auto & elem : x)
|
||||
applyVisitor(*this, elem);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Map & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Map;
|
||||
hash.update(type);
|
||||
hash.update(x.size());
|
||||
|
||||
for (const auto & elem : x)
|
||||
applyVisitor(*this, elem);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Array & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Array;
|
||||
hash.update(type);
|
||||
hash.update(x.size());
|
||||
|
||||
for (const auto & elem : x)
|
||||
applyVisitor(*this, elem);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const DecimalField<Decimal32> & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Decimal32;
|
||||
hash.update(type);
|
||||
hash.update(x.getValue().value);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const DecimalField<Decimal64> & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Decimal64;
|
||||
hash.update(type);
|
||||
hash.update(x.getValue().value);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const DecimalField<Decimal128> & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Decimal128;
|
||||
hash.update(type);
|
||||
hash.update(x.getValue().value);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const DecimalField<Decimal256> & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Decimal256;
|
||||
hash.update(type);
|
||||
hash.update(x.getValue().value);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const AggregateFunctionStateData & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::AggregateFunctionState;
|
||||
hash.update(type);
|
||||
hash.update(x.name.size());
|
||||
hash.update(x.name.data(), x.name.size());
|
||||
hash.update(x.data.size());
|
||||
hash.update(x.data.data(), x.data.size());
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UInt256 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UInt256;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Int256 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Int256;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
}
|
38
src/Common/FieldVisitorHash.h
Normal file
38
src/Common/FieldVisitorHash.h
Normal file
@ -0,0 +1,38 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
class SipHash;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Updates SipHash by type and value of Field */
|
||||
class FieldVisitorHash : public StaticVisitor<>
|
||||
{
|
||||
private:
|
||||
SipHash & hash;
|
||||
public:
|
||||
FieldVisitorHash(SipHash & hash_);
|
||||
|
||||
void operator() (const Null & x) const;
|
||||
void operator() (const UInt64 & x) const;
|
||||
void operator() (const UInt128 & x) const;
|
||||
void operator() (const UInt256 & x) const;
|
||||
void operator() (const Int64 & x) const;
|
||||
void operator() (const Int128 & x) const;
|
||||
void operator() (const Int256 & x) const;
|
||||
void operator() (const UUID & x) const;
|
||||
void operator() (const Float64 & x) const;
|
||||
void operator() (const String & x) const;
|
||||
void operator() (const Array & x) const;
|
||||
void operator() (const Tuple & x) const;
|
||||
void operator() (const Map & x) const;
|
||||
void operator() (const DecimalField<Decimal32> & x) const;
|
||||
void operator() (const DecimalField<Decimal64> & x) const;
|
||||
void operator() (const DecimalField<Decimal128> & x) const;
|
||||
void operator() (const DecimalField<Decimal256> & x) const;
|
||||
void operator() (const AggregateFunctionStateData & x) const;
|
||||
};
|
||||
|
||||
}
|
37
src/Common/FieldVisitorSum.cpp
Normal file
37
src/Common/FieldVisitorSum.cpp
Normal file
@ -0,0 +1,37 @@
|
||||
#include <Common/FieldVisitorSum.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
FieldVisitorSum::FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
|
||||
|
||||
// We can add all ints as unsigned regardless of their actual signedness.
|
||||
bool FieldVisitorSum::operator() (Int64 & x) const { return this->operator()(reinterpret_cast<UInt64 &>(x)); }
|
||||
bool FieldVisitorSum::operator() (UInt64 & x) const
|
||||
{
|
||||
x += rhs.reinterpret<UInt64>();
|
||||
return x != 0;
|
||||
}
|
||||
|
||||
bool FieldVisitorSum::operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
|
||||
|
||||
bool FieldVisitorSum::operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Tuple &) const { throw Exception("Cannot sum Tuples", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (Map &) const { throw Exception("Cannot sum Maps", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool FieldVisitorSum::operator() (UUID &) const { throw Exception("Cannot sum UUIDs", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool FieldVisitorSum::operator() (AggregateFunctionStateData &) const
|
||||
{
|
||||
throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
}
|
||||
|
47
src/Common/FieldVisitorSum.h
Normal file
47
src/Common/FieldVisitorSum.h
Normal file
@ -0,0 +1,47 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Implements `+=` operation.
|
||||
* Returns false if the result is zero.
|
||||
*/
|
||||
class FieldVisitorSum : public StaticVisitor<bool>
|
||||
{
|
||||
private:
|
||||
const Field & rhs;
|
||||
public:
|
||||
explicit FieldVisitorSum(const Field & rhs_);
|
||||
|
||||
// We can add all ints as unsigned regardless of their actual signedness.
|
||||
bool operator() (Int64 & x) const;
|
||||
bool operator() (UInt64 & x) const;
|
||||
bool operator() (Float64 & x) const;
|
||||
bool operator() (Null &) const;
|
||||
bool operator() (String &) const;
|
||||
bool operator() (Array &) const;
|
||||
bool operator() (Tuple &) const;
|
||||
bool operator() (Map &) const;
|
||||
bool operator() (UUID &) const;
|
||||
bool operator() (AggregateFunctionStateData &) const;
|
||||
|
||||
template <typename T>
|
||||
bool operator() (DecimalField<T> & x) const
|
||||
{
|
||||
x += get<DecimalField<T>>(rhs);
|
||||
return x.getValue() != T(0);
|
||||
}
|
||||
|
||||
template <typename T, typename = std::enable_if_t<is_big_int_v<T>> >
|
||||
bool operator() (T & x) const
|
||||
{
|
||||
x += rhs.reinterpret<T>();
|
||||
return x != T(0);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
130
src/Common/FieldVisitorToString.cpp
Normal file
130
src/Common/FieldVisitorToString.cpp
Normal file
@ -0,0 +1,130 @@
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER;
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
static inline String formatQuoted(T x)
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
writeQuoted(x, wb);
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
|
||||
{
|
||||
writeChar('\'', buf);
|
||||
writeText(x.getValue(), x.getScale(), buf);
|
||||
writeChar('\'', buf);
|
||||
}
|
||||
|
||||
/** In contrast to writeFloatText (and writeQuoted),
|
||||
* even if number looks like integer after formatting, prints decimal point nevertheless (for example, Float64(1) is printed as 1.).
|
||||
* - because resulting text must be able to be parsed back as Float64 by query parser (otherwise it will be parsed as integer).
|
||||
*
|
||||
* Trailing zeros after decimal point are omitted.
|
||||
*
|
||||
* NOTE: Roundtrip may lead to loss of precision.
|
||||
*/
|
||||
static String formatFloat(const Float64 x)
|
||||
{
|
||||
DoubleConverter<true>::BufferType buffer;
|
||||
double_conversion::StringBuilder builder{buffer, sizeof(buffer)};
|
||||
|
||||
const auto result = DoubleConverter<true>::instance().ToShortest(x, &builder);
|
||||
|
||||
if (!result)
|
||||
throw Exception("Cannot print float or double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
|
||||
|
||||
return { buffer, buffer + builder.position() };
|
||||
}
|
||||
|
||||
|
||||
String FieldVisitorToString::operator() (const Null &) const { return "NULL"; }
|
||||
String FieldVisitorToString::operator() (const UInt64 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int64 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Float64 & x) const { return formatFloat(x); }
|
||||
String FieldVisitorToString::operator() (const String & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const DecimalField<Decimal32> & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const DecimalField<Decimal64> & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const DecimalField<Decimal128> & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const DecimalField<Decimal256> & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int128 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UInt128 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UInt256 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int256 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UUID & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x.data); }
|
||||
|
||||
String FieldVisitorToString::operator() (const Array & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << '[';
|
||||
for (Array::const_iterator it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb.write(", ", 2);
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ']';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorToString::operator() (const Tuple & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
// For single-element tuples we must use the explicit tuple() function,
|
||||
// or they will be parsed back as plain literals.
|
||||
if (x.size() > 1)
|
||||
{
|
||||
wb << '(';
|
||||
}
|
||||
else
|
||||
{
|
||||
wb << "tuple(";
|
||||
}
|
||||
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ')';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorToString::operator() (const Map & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << '(';
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ')';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
}
|
||||
|
33
src/Common/FieldVisitorToString.h
Normal file
33
src/Common/FieldVisitorToString.h
Normal file
@ -0,0 +1,33 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Prints Field as literal in SQL query */
|
||||
class FieldVisitorToString : public StaticVisitor<String>
|
||||
{
|
||||
public:
|
||||
String operator() (const Null & x) const;
|
||||
String operator() (const UInt64 & x) const;
|
||||
String operator() (const UInt128 & x) const;
|
||||
String operator() (const UInt256 & x) const;
|
||||
String operator() (const Int64 & x) const;
|
||||
String operator() (const Int128 & x) const;
|
||||
String operator() (const Int256 & x) const;
|
||||
String operator() (const UUID & x) const;
|
||||
String operator() (const Float64 & x) const;
|
||||
String operator() (const String & x) const;
|
||||
String operator() (const Array & x) const;
|
||||
String operator() (const Tuple & x) const;
|
||||
String operator() (const Map & x) const;
|
||||
String operator() (const DecimalField<Decimal32> & x) const;
|
||||
String operator() (const DecimalField<Decimal64> & x) const;
|
||||
String operator() (const DecimalField<Decimal128> & x) const;
|
||||
String operator() (const DecimalField<Decimal256> & x) const;
|
||||
String operator() (const AggregateFunctionStateData & x) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
70
src/Common/FieldVisitorWriteBinary.cpp
Normal file
70
src/Common/FieldVisitorWriteBinary.cpp
Normal file
@ -0,0 +1,70 @@
|
||||
#include <Common/FieldVisitorWriteBinary.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Null &, WriteBuffer &) const { }
|
||||
void FieldVisitorWriteBinary::operator() (const UInt64 & x, WriteBuffer & buf) const { writeVarUInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int64 & x, WriteBuffer & buf) const { writeVarInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Float64 & x, WriteBuffer & buf) const { writeFloatBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const String & x, WriteBuffer & buf) const { writeStringBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UInt128 & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int128 & x, WriteBuffer & buf) const { writeVarInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UInt256 & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int256 & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UUID & x, WriteBuffer & buf) const { writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal32> & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal64> & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal128> & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal256> & x, WriteBuffer & buf) const { writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const AggregateFunctionStateData & x, WriteBuffer & buf) const
|
||||
{
|
||||
writeStringBinary(x.name, buf);
|
||||
writeStringBinary(x.data, buf);
|
||||
}
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Array & x, WriteBuffer & buf) const
|
||||
{
|
||||
const size_t size = x.size();
|
||||
writeBinary(size, buf);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 type = x[i].getType();
|
||||
writeBinary(type, buf);
|
||||
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, x[i]);
|
||||
}
|
||||
}
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Tuple & x, WriteBuffer & buf) const
|
||||
{
|
||||
const size_t size = x.size();
|
||||
writeBinary(size, buf);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 type = x[i].getType();
|
||||
writeBinary(type, buf);
|
||||
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, x[i]);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Map & x, WriteBuffer & buf) const
|
||||
{
|
||||
const size_t size = x.size();
|
||||
writeBinary(size, buf);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 type = x[i].getType();
|
||||
writeBinary(type, buf);
|
||||
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, x[i]);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
32
src/Common/FieldVisitorWriteBinary.h
Normal file
32
src/Common/FieldVisitorWriteBinary.h
Normal file
@ -0,0 +1,32 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FieldVisitorWriteBinary
|
||||
{
|
||||
public:
|
||||
void operator() (const Null & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt128 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt256 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int128 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int256 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UUID & x, WriteBuffer & buf) const;
|
||||
void operator() (const Float64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const String & x, WriteBuffer & buf) const;
|
||||
void operator() (const Array & x, WriteBuffer & buf) const;
|
||||
void operator() (const Tuple & x, WriteBuffer & buf) const;
|
||||
void operator() (const Map & x, WriteBuffer & buf) const;
|
||||
void operator() (const DecimalField<Decimal32> & x, WriteBuffer & buf) const;
|
||||
void operator() (const DecimalField<Decimal64> & x, WriteBuffer & buf) const;
|
||||
void operator() (const DecimalField<Decimal128> & x, WriteBuffer & buf) const;
|
||||
void operator() (const DecimalField<Decimal256> & x, WriteBuffer & buf) const;
|
||||
void operator() (const AggregateFunctionStateData & x, WriteBuffer & buf) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,421 +0,0 @@
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/SipHash.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline String formatQuoted(T x)
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
writeQuoted(x, wb);
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline String formatQuotedWithPrefix(T x, const char * prefix)
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
writeCString(prefix, wb);
|
||||
writeQuoted(x, wb);
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeQuoted(const DecimalField<T> & x, WriteBuffer & buf)
|
||||
{
|
||||
writeChar('\'', buf);
|
||||
writeText(x.getValue(), x.getScale(), buf);
|
||||
writeChar('\'', buf);
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Null &) const { return "NULL"; }
|
||||
String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuotedWithPrefix(x, "UInt64_"); }
|
||||
String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); }
|
||||
String FieldVisitorDump::operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); }
|
||||
String FieldVisitorDump::operator() (const DecimalField<Decimal32> & x) const { return formatQuotedWithPrefix(x, "Decimal32_"); }
|
||||
String FieldVisitorDump::operator() (const DecimalField<Decimal64> & x) const { return formatQuotedWithPrefix(x, "Decimal64_"); }
|
||||
String FieldVisitorDump::operator() (const DecimalField<Decimal128> & x) const { return formatQuotedWithPrefix(x, "Decimal128_"); }
|
||||
String FieldVisitorDump::operator() (const DecimalField<Decimal256> & x) const { return formatQuotedWithPrefix(x, "Decimal256_"); }
|
||||
String FieldVisitorDump::operator() (const UInt128 & x) const { return formatQuotedWithPrefix(x, "UInt128_"); }
|
||||
String FieldVisitorDump::operator() (const UInt256 & x) const { return formatQuotedWithPrefix(x, "UInt256_"); }
|
||||
String FieldVisitorDump::operator() (const Int128 & x) const { return formatQuotedWithPrefix(x, "Int128_"); }
|
||||
String FieldVisitorDump::operator() (const Int256 & x) const { return formatQuotedWithPrefix(x, "Int256_"); }
|
||||
String FieldVisitorDump::operator() (const UUID & x) const { return formatQuotedWithPrefix(x, "UUID_"); }
|
||||
|
||||
|
||||
String FieldVisitorDump::operator() (const String & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
writeQuoted(x, wb);
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Array & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << "Array_[";
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ']';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Tuple & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << "Tuple_(";
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ')';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const Map & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << "Map_(";
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ')';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorDump::operator() (const AggregateFunctionStateData & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
wb << "AggregateFunctionState_(";
|
||||
writeQuoted(x.name, wb);
|
||||
wb << ", ";
|
||||
writeQuoted(x.data, wb);
|
||||
wb << ')';
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
/** In contrast to writeFloatText (and writeQuoted),
|
||||
* even if number looks like integer after formatting, prints decimal point nevertheless (for example, Float64(1) is printed as 1.).
|
||||
* - because resulting text must be able to be parsed back as Float64 by query parser (otherwise it will be parsed as integer).
|
||||
*
|
||||
* Trailing zeros after decimal point are omitted.
|
||||
*
|
||||
* NOTE: Roundtrip may lead to loss of precision.
|
||||
*/
|
||||
static String formatFloat(const Float64 x)
|
||||
{
|
||||
DoubleConverter<true>::BufferType buffer;
|
||||
double_conversion::StringBuilder builder{buffer, sizeof(buffer)};
|
||||
|
||||
const auto result = DoubleConverter<true>::instance().ToShortest(x, &builder);
|
||||
|
||||
if (!result)
|
||||
throw Exception("Cannot print float or double number", ErrorCodes::CANNOT_PRINT_FLOAT_OR_DOUBLE_NUMBER);
|
||||
|
||||
return { buffer, buffer + builder.position() };
|
||||
}
|
||||
|
||||
|
||||
String FieldVisitorToString::operator() (const Null &) const { return "NULL"; }
|
||||
String FieldVisitorToString::operator() (const UInt64 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int64 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Float64 & x) const { return formatFloat(x); }
|
||||
String FieldVisitorToString::operator() (const String & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const DecimalField<Decimal32> & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const DecimalField<Decimal64> & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const DecimalField<Decimal128> & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const DecimalField<Decimal256> & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int128 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UInt128 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UInt256 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const Int256 & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const UUID & x) const { return formatQuoted(x); }
|
||||
String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const { return formatQuoted(x.data); }
|
||||
|
||||
String FieldVisitorToString::operator() (const Array & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << '[';
|
||||
for (Array::const_iterator it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb.write(", ", 2);
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ']';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorToString::operator() (const Tuple & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
// For single-element tuples we must use the explicit tuple() function,
|
||||
// or they will be parsed back as plain literals.
|
||||
if (x.size() > 1)
|
||||
{
|
||||
wb << '(';
|
||||
}
|
||||
else
|
||||
{
|
||||
wb << "tuple(";
|
||||
}
|
||||
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ')';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
String FieldVisitorToString::operator() (const Map & x) const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
|
||||
wb << '(';
|
||||
for (auto it = x.begin(); it != x.end(); ++it)
|
||||
{
|
||||
if (it != x.begin())
|
||||
wb << ", ";
|
||||
wb << applyVisitor(*this, *it);
|
||||
}
|
||||
wb << ')';
|
||||
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Null &, WriteBuffer &) const { }
|
||||
void FieldVisitorWriteBinary::operator() (const UInt64 & x, WriteBuffer & buf) const { DB::writeVarUInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int64 & x, WriteBuffer & buf) const { DB::writeVarInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Float64 & x, WriteBuffer & buf) const { DB::writeFloatBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const String & x, WriteBuffer & buf) const { DB::writeStringBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UInt128 & x, WriteBuffer & buf) const { DB::writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int128 & x, WriteBuffer & buf) const { DB::writeVarInt(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UInt256 & x, WriteBuffer & buf) const { DB::writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const Int256 & x, WriteBuffer & buf) const { DB::writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const UUID & x, WriteBuffer & buf) const { DB::writeBinary(x, buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal32> & x, WriteBuffer & buf) const { DB::writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal64> & x, WriteBuffer & buf) const { DB::writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal128> & x, WriteBuffer & buf) const { DB::writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const DecimalField<Decimal256> & x, WriteBuffer & buf) const { DB::writeBinary(x.getValue(), buf); }
|
||||
void FieldVisitorWriteBinary::operator() (const AggregateFunctionStateData & x, WriteBuffer & buf) const
|
||||
{
|
||||
DB::writeStringBinary(x.name, buf);
|
||||
DB::writeStringBinary(x.data, buf);
|
||||
}
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Array & x, WriteBuffer & buf) const
|
||||
{
|
||||
const size_t size = x.size();
|
||||
DB::writeBinary(size, buf);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 type = x[i].getType();
|
||||
DB::writeBinary(type, buf);
|
||||
Field::dispatch([&buf] (const auto & value) { DB::FieldVisitorWriteBinary()(value, buf); }, x[i]);
|
||||
}
|
||||
}
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Tuple & x, WriteBuffer & buf) const
|
||||
{
|
||||
const size_t size = x.size();
|
||||
DB::writeBinary(size, buf);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 type = x[i].getType();
|
||||
DB::writeBinary(type, buf);
|
||||
Field::dispatch([&buf] (const auto & value) { DB::FieldVisitorWriteBinary()(value, buf); }, x[i]);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void FieldVisitorWriteBinary::operator() (const Map & x, WriteBuffer & buf) const
|
||||
{
|
||||
const size_t size = x.size();
|
||||
DB::writeBinary(size, buf);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
const UInt8 type = x[i].getType();
|
||||
writeBinary(type, buf);
|
||||
Field::dispatch([&buf] (const auto & value) { DB::FieldVisitorWriteBinary()(value, buf); }, x[i]);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
FieldVisitorHash::FieldVisitorHash(SipHash & hash_) : hash(hash_) {}
|
||||
|
||||
void FieldVisitorHash::operator() (const Null &) const
|
||||
{
|
||||
UInt8 type = Field::Types::Null;
|
||||
hash.update(type);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UInt64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UInt64;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UInt128 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UInt128;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Int64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Int64;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Int128 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Int128;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UUID & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UUID;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Float64 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Float64;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const String & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::String;
|
||||
hash.update(type);
|
||||
hash.update(x.size());
|
||||
hash.update(x.data(), x.size());
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Tuple & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Tuple;
|
||||
hash.update(type);
|
||||
hash.update(x.size());
|
||||
|
||||
for (const auto & elem : x)
|
||||
applyVisitor(*this, elem);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Map & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Map;
|
||||
hash.update(type);
|
||||
hash.update(x.size());
|
||||
|
||||
for (const auto & elem : x)
|
||||
applyVisitor(*this, elem);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Array & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Array;
|
||||
hash.update(type);
|
||||
hash.update(x.size());
|
||||
|
||||
for (const auto & elem : x)
|
||||
applyVisitor(*this, elem);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const DecimalField<Decimal32> & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Decimal32;
|
||||
hash.update(type);
|
||||
hash.update(x.getValue().value);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const DecimalField<Decimal64> & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Decimal64;
|
||||
hash.update(type);
|
||||
hash.update(x.getValue().value);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const DecimalField<Decimal128> & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Decimal128;
|
||||
hash.update(type);
|
||||
hash.update(x.getValue().value);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const DecimalField<Decimal256> & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Decimal256;
|
||||
hash.update(type);
|
||||
hash.update(x.getValue().value);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const AggregateFunctionStateData & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::AggregateFunctionState;
|
||||
hash.update(type);
|
||||
hash.update(x.name.size());
|
||||
hash.update(x.name.data(), x.name.size());
|
||||
hash.update(x.data.size());
|
||||
hash.update(x.data.data(), x.data.size());
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const UInt256 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::UInt256;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
void FieldVisitorHash::operator() (const Int256 & x) const
|
||||
{
|
||||
UInt8 type = Field::Types::Int256;
|
||||
hash.update(type);
|
||||
hash.update(x);
|
||||
}
|
||||
|
||||
}
|
@ -1,25 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <Core/Field.h>
|
||||
#include <common/demangle.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
|
||||
|
||||
class SipHash;
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_CONVERT_TYPE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
/** StaticVisitor (and its descendants) - class with overloaded operator() for all types of fields.
|
||||
* You could call visitor for field using function 'applyVisitor'.
|
||||
* Also "binary visitor" is supported - its operator() takes two arguments.
|
||||
@ -55,268 +41,4 @@ auto applyVisitor(Visitor && visitor, F1 && field1, F2 && field2)
|
||||
std::forward<F1>(field1));
|
||||
}
|
||||
|
||||
|
||||
/** Prints Field as literal in SQL query */
|
||||
class FieldVisitorToString : public StaticVisitor<String>
|
||||
{
|
||||
public:
|
||||
String operator() (const Null & x) const;
|
||||
String operator() (const UInt64 & x) const;
|
||||
String operator() (const UInt128 & x) const;
|
||||
String operator() (const UInt256 & x) const;
|
||||
String operator() (const Int64 & x) const;
|
||||
String operator() (const Int128 & x) const;
|
||||
String operator() (const Int256 & x) const;
|
||||
String operator() (const UUID & x) const;
|
||||
String operator() (const Float64 & x) const;
|
||||
String operator() (const String & x) const;
|
||||
String operator() (const Array & x) const;
|
||||
String operator() (const Tuple & x) const;
|
||||
String operator() (const Map & x) const;
|
||||
String operator() (const DecimalField<Decimal32> & x) const;
|
||||
String operator() (const DecimalField<Decimal64> & x) const;
|
||||
String operator() (const DecimalField<Decimal128> & x) const;
|
||||
String operator() (const DecimalField<Decimal256> & x) const;
|
||||
String operator() (const AggregateFunctionStateData & x) const;
|
||||
};
|
||||
|
||||
|
||||
class FieldVisitorWriteBinary
|
||||
{
|
||||
public:
|
||||
void operator() (const Null & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt128 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UInt256 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int128 & x, WriteBuffer & buf) const;
|
||||
void operator() (const Int256 & x, WriteBuffer & buf) const;
|
||||
void operator() (const UUID & x, WriteBuffer & buf) const;
|
||||
void operator() (const Float64 & x, WriteBuffer & buf) const;
|
||||
void operator() (const String & x, WriteBuffer & buf) const;
|
||||
void operator() (const Array & x, WriteBuffer & buf) const;
|
||||
void operator() (const Tuple & x, WriteBuffer & buf) const;
|
||||
void operator() (const Map & x, WriteBuffer & buf) const;
|
||||
void operator() (const DecimalField<Decimal32> & x, WriteBuffer & buf) const;
|
||||
void operator() (const DecimalField<Decimal64> & x, WriteBuffer & buf) const;
|
||||
void operator() (const DecimalField<Decimal128> & x, WriteBuffer & buf) const;
|
||||
void operator() (const DecimalField<Decimal256> & x, WriteBuffer & buf) const;
|
||||
void operator() (const AggregateFunctionStateData & x, WriteBuffer & buf) const;
|
||||
};
|
||||
|
||||
|
||||
/** Print readable and unique text dump of field type and value. */
|
||||
class FieldVisitorDump : public StaticVisitor<String>
|
||||
{
|
||||
public:
|
||||
String operator() (const Null & x) const;
|
||||
String operator() (const UInt64 & x) const;
|
||||
String operator() (const UInt128 & x) const;
|
||||
String operator() (const UInt256 & x) const;
|
||||
String operator() (const Int64 & x) const;
|
||||
String operator() (const Int128 & x) const;
|
||||
String operator() (const Int256 & x) const;
|
||||
String operator() (const UUID & x) const;
|
||||
String operator() (const Float64 & x) const;
|
||||
String operator() (const String & x) const;
|
||||
String operator() (const Array & x) const;
|
||||
String operator() (const Tuple & x) const;
|
||||
String operator() (const Map & x) const;
|
||||
String operator() (const DecimalField<Decimal32> & x) const;
|
||||
String operator() (const DecimalField<Decimal64> & x) const;
|
||||
String operator() (const DecimalField<Decimal128> & x) const;
|
||||
String operator() (const DecimalField<Decimal256> & x) const;
|
||||
String operator() (const AggregateFunctionStateData & x) const;
|
||||
};
|
||||
|
||||
|
||||
/** Converts numeric value of any type to specified type. */
|
||||
template <typename T>
|
||||
class FieldVisitorConvertToNumber : public StaticVisitor<T>
|
||||
{
|
||||
public:
|
||||
T operator() (const Null &) const
|
||||
{
|
||||
throw Exception("Cannot convert NULL to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const String &) const
|
||||
{
|
||||
throw Exception("Cannot convert String to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const Array &) const
|
||||
{
|
||||
throw Exception("Cannot convert Array to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const Tuple &) const
|
||||
{
|
||||
throw Exception("Cannot convert Tuple to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const Map &) const
|
||||
{
|
||||
throw Exception("Cannot convert Map to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
T operator() (const UInt64 & x) const { return T(x); }
|
||||
T operator() (const Int64 & x) const { return T(x); }
|
||||
T operator() (const Int128 & x) const { return T(x); }
|
||||
T operator() (const UUID & x) const { return T(x.toUnderType()); }
|
||||
|
||||
T operator() (const Float64 & x) const
|
||||
{
|
||||
if constexpr (!std::is_floating_point_v<T>)
|
||||
{
|
||||
if (!isFinite(x))
|
||||
{
|
||||
/// When converting to bool it's ok (non-zero converts to true, NaN including).
|
||||
if (std::is_same_v<T, bool>)
|
||||
return true;
|
||||
|
||||
/// Conversion of infinite values to integer is undefined.
|
||||
throw Exception("Cannot convert infinite value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
else if (x > std::numeric_limits<T>::max() || x < std::numeric_limits<T>::lowest())
|
||||
{
|
||||
throw Exception("Cannot convert out of range floating point value to integer type", ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (std::is_same_v<Decimal256, T>)
|
||||
{
|
||||
return Int256(x);
|
||||
}
|
||||
else
|
||||
{
|
||||
return T(x);
|
||||
}
|
||||
}
|
||||
|
||||
T operator() (const UInt128 &) const
|
||||
{
|
||||
throw Exception("Cannot convert UInt128 to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
template <typename U>
|
||||
T operator() (const DecimalField<U> & x) const
|
||||
{
|
||||
if constexpr (std::is_floating_point_v<T>)
|
||||
return x.getValue(). template convertTo<T>() / x.getScaleMultiplier(). template convertTo<T>();
|
||||
else if constexpr (std::is_same_v<T, UInt128>)
|
||||
{
|
||||
/// TODO: remove with old UInt128 type
|
||||
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
|
||||
return (x.getValue() / x.getScaleMultiplier()). template convertTo<T>();
|
||||
}
|
||||
|
||||
T operator() (const AggregateFunctionStateData &) const
|
||||
{
|
||||
throw Exception("Cannot convert AggregateFunctionStateData to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE);
|
||||
}
|
||||
|
||||
template <typename U, typename = std::enable_if_t<is_big_int_v<U>> >
|
||||
T operator() (const U & x) const
|
||||
{
|
||||
if constexpr (IsDecimalNumber<T>)
|
||||
return static_cast<T>(static_cast<typename T::NativeType>(x));
|
||||
else if constexpr (std::is_same_v<T, UInt128>)
|
||||
throw Exception("No conversion to old UInt128 from " + demangle(typeid(U).name()), ErrorCodes::NOT_IMPLEMENTED);
|
||||
else
|
||||
return static_cast<T>(x);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/** Updates SipHash by type and value of Field */
|
||||
class FieldVisitorHash : public StaticVisitor<>
|
||||
{
|
||||
private:
|
||||
SipHash & hash;
|
||||
public:
|
||||
FieldVisitorHash(SipHash & hash_);
|
||||
|
||||
void operator() (const Null & x) const;
|
||||
void operator() (const UInt64 & x) const;
|
||||
void operator() (const UInt128 & x) const;
|
||||
void operator() (const UInt256 & x) const;
|
||||
void operator() (const Int64 & x) const;
|
||||
void operator() (const Int128 & x) const;
|
||||
void operator() (const Int256 & x) const;
|
||||
void operator() (const UUID & x) const;
|
||||
void operator() (const Float64 & x) const;
|
||||
void operator() (const String & x) const;
|
||||
void operator() (const Array & x) const;
|
||||
void operator() (const Tuple & x) const;
|
||||
void operator() (const Map & x) const;
|
||||
void operator() (const DecimalField<Decimal32> & x) const;
|
||||
void operator() (const DecimalField<Decimal64> & x) const;
|
||||
void operator() (const DecimalField<Decimal128> & x) const;
|
||||
void operator() (const DecimalField<Decimal256> & x) const;
|
||||
void operator() (const AggregateFunctionStateData & x) const;
|
||||
};
|
||||
|
||||
|
||||
/** Implements `+=` operation.
|
||||
* Returns false if the result is zero.
|
||||
*/
|
||||
class FieldVisitorSum : public StaticVisitor<bool>
|
||||
{
|
||||
private:
|
||||
const Field & rhs;
|
||||
public:
|
||||
explicit FieldVisitorSum(const Field & rhs_) : rhs(rhs_) {}
|
||||
|
||||
// We can add all ints as unsigned regardless of their actual signedness.
|
||||
bool operator() (Int64 & x) const { return this->operator()(reinterpret_cast<UInt64 &>(x)); }
|
||||
bool operator() (UInt64 & x) const
|
||||
{
|
||||
x += rhs.reinterpret<UInt64>();
|
||||
return x != 0;
|
||||
}
|
||||
|
||||
bool operator() (Float64 & x) const { x += get<Float64>(rhs); return x != 0; }
|
||||
|
||||
bool operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Tuple &) const { throw Exception("Cannot sum Tuples", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (Map &) const { throw Exception("Cannot sum Maps", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (UUID &) const { throw Exception("Cannot sum UUIDs", ErrorCodes::LOGICAL_ERROR); }
|
||||
bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); }
|
||||
|
||||
bool operator() (Int128 & x) const
|
||||
{
|
||||
x += get<Int128>(rhs);
|
||||
return x != Int128(0);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool operator() (DecimalField<T> & x) const
|
||||
{
|
||||
x += get<DecimalField<T>>(rhs);
|
||||
return x.getValue() != T(0);
|
||||
}
|
||||
|
||||
template <typename T, typename = std::enable_if_t<is_big_int_v<T>> >
|
||||
bool operator() (T & x) const
|
||||
{
|
||||
x += rhs.reinterpret<T>();
|
||||
return x != T(0);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -44,7 +44,11 @@ SRCS(
|
||||
ErrorCodes.cpp
|
||||
Exception.cpp
|
||||
ExternalLoaderStatus.cpp
|
||||
FieldVisitors.cpp
|
||||
FieldVisitorDump.cpp
|
||||
FieldVisitorHash.cpp
|
||||
FieldVisitorSum.cpp
|
||||
FieldVisitorToString.cpp
|
||||
FieldVisitorWriteBinary.cpp
|
||||
FileChecker.cpp
|
||||
IO.cpp
|
||||
IPv6ToBinary.cpp
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
|
||||
|
@ -6,7 +6,9 @@
|
||||
#include <IO/readDecimalText.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/DecimalComparison.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorDump.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/FieldVisitorWriteBinary.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -95,12 +97,12 @@ void writeBinary(const Array & x, WriteBuffer & buf)
|
||||
DB::writeBinary(size, buf);
|
||||
|
||||
for (const auto & elem : x)
|
||||
Field::dispatch([&buf] (const auto & value) { DB::FieldVisitorWriteBinary()(value, buf); }, elem);
|
||||
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
|
||||
}
|
||||
|
||||
void writeText(const Array & x, WriteBuffer & buf)
|
||||
{
|
||||
DB::String res = applyVisitor(DB::FieldVisitorToString(), DB::Field(x));
|
||||
DB::String res = applyVisitor(FieldVisitorToString(), DB::Field(x));
|
||||
buf.write(res.data(), res.size());
|
||||
}
|
||||
|
||||
@ -126,7 +128,7 @@ void writeBinary(const Tuple & x, WriteBuffer & buf)
|
||||
{
|
||||
const UInt8 type = elem.getType();
|
||||
DB::writeBinary(type, buf);
|
||||
Field::dispatch([&buf] (const auto & value) { DB::FieldVisitorWriteBinary()(value, buf); }, elem);
|
||||
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
|
||||
}
|
||||
}
|
||||
|
||||
@ -157,7 +159,7 @@ void writeBinary(const Map & x, WriteBuffer & buf)
|
||||
{
|
||||
const UInt8 type = elem.getType();
|
||||
DB::writeBinary(type, buf);
|
||||
Field::dispatch([&buf] (const auto & value) { DB::FieldVisitorWriteBinary()(value, buf); }, elem);
|
||||
Field::dispatch([&buf] (const auto & value) { FieldVisitorWriteBinary()(value, buf); }, elem);
|
||||
}
|
||||
}
|
||||
|
||||
@ -194,14 +196,14 @@ template void readQuoted<Decimal256>(DecimalField<Decimal256> & x, ReadBuffer &
|
||||
|
||||
void writeFieldText(const Field & x, WriteBuffer & buf)
|
||||
{
|
||||
DB::String res = Field::dispatch(DB::FieldVisitorToString(), x);
|
||||
String res = Field::dispatch(FieldVisitorToString(), x);
|
||||
buf.write(res.data(), res.size());
|
||||
}
|
||||
|
||||
|
||||
String Field::dump() const
|
||||
{
|
||||
return applyVisitor(DB::FieldVisitorDump(), *this);
|
||||
return applyVisitor(FieldVisitorDump(), *this);
|
||||
}
|
||||
|
||||
Field Field::restoreFromDump(const std::string_view & dump_)
|
||||
|
@ -163,10 +163,10 @@ private:
|
||||
#endif
|
||||
|
||||
template <typename T> constexpr bool is_decimal_field = false;
|
||||
template <> constexpr bool is_decimal_field<DecimalField<Decimal32>> = true;
|
||||
template <> constexpr bool is_decimal_field<DecimalField<Decimal64>> = true;
|
||||
template <> constexpr bool is_decimal_field<DecimalField<Decimal128>> = true;
|
||||
template <> constexpr bool is_decimal_field<DecimalField<Decimal256>> = true;
|
||||
template <> constexpr inline bool is_decimal_field<DecimalField<Decimal32>> = true;
|
||||
template <> constexpr inline bool is_decimal_field<DecimalField<Decimal64>> = true;
|
||||
template <> constexpr inline bool is_decimal_field<DecimalField<Decimal128>> = true;
|
||||
template <> constexpr inline bool is_decimal_field<DecimalField<Decimal256>> = true;
|
||||
|
||||
/// char may be signed or unsigned, and behave identically to signed char or unsigned char,
|
||||
/// but they are always three different types.
|
||||
|
@ -6,10 +6,11 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <common/DateLUT.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Core/MySQL/PacketsGeneric.h>
|
||||
#include <Core/MySQL/PacketsProtocolText.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <iomanip>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
@ -15,7 +15,8 @@
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/COW.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorDump.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include <DataStreams/CheckSortedBlockInputStream.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorDump.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Core/SortDescription.h>
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -8,6 +8,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
@ -47,7 +48,7 @@ std::string DataTypeAggregateFunction::doGetName() const
|
||||
{
|
||||
if (i)
|
||||
stream << ", ";
|
||||
stream << applyVisitor(DB::FieldVisitorToString(), parameters[i]);
|
||||
stream << applyVisitor(FieldVisitorToString(), parameters[i]);
|
||||
}
|
||||
stream << ')';
|
||||
}
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
|
||||
@ -13,6 +13,7 @@
|
||||
|
||||
#include <boost/algorithm/string/join.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -27,6 +28,7 @@ namespace ErrorCodes
|
||||
|
||||
void DataTypeCustomSimpleAggregateFunction::checkSupportedFunctions(const AggregateFunctionPtr & function)
|
||||
{
|
||||
/// TODO Make it sane.
|
||||
static const std::vector<String> supported_functions{"any", "anyLast", "min",
|
||||
"max", "sum", "sumWithOverflow", "groupBitAnd", "groupBitOr", "groupBitXor",
|
||||
"sumMap", "minMap", "maxMap", "groupArrayArray", "groupUniqArrayArray"};
|
||||
@ -51,7 +53,7 @@ String DataTypeCustomSimpleAggregateFunction::getName() const
|
||||
{
|
||||
if (i)
|
||||
stream << ", ";
|
||||
stream << applyVisitor(DB::FieldVisitorToString(), parameters[i]);
|
||||
stream << applyVisitor(FieldVisitorToString(), parameters[i]);
|
||||
}
|
||||
stream << ")";
|
||||
}
|
||||
|
@ -11,11 +11,13 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Parsers/ASTFunctionWithKeyValueArguments.h>
|
||||
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
||||
#include <Dictionaries/DictionaryFactory.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -32,7 +32,6 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <ext/map.h>
|
||||
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/types.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include "Sources.h"
|
||||
#include "Sinks.h"
|
||||
#include <Core/AccurateComparison.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionUnaryArithmetic.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionUnaryArithmetic.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/intExp.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionUnaryArithmetic.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/intExp.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <common/sleep.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <mutex>
|
||||
#include <ext/bit_cast.h>
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
@ -1,9 +1,10 @@
|
||||
#include <Interpreters/AggregateDescription.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <Common/JSONBuilder.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include "CatBoostModel.h"
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <mutex>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
@ -14,6 +14,7 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Common/formatReadable.h>
|
||||
#include <Common/Throttler.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Coordination/KeeperStorageDispatcher.h>
|
||||
#include <Compression/ICompressionCodec.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <Interpreters/FillingRow.h>
|
||||
#include <Common/FieldVisitorSum.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
|
||||
|
||||
|
@ -79,6 +79,7 @@
|
||||
#include <common/types.h>
|
||||
#include <Columns/Collator.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <ext/map.h>
|
||||
|
@ -7,7 +7,7 @@
|
||||
#include <llvm/IR/IRBuilder.h>
|
||||
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/Native.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <Interpreters/WindowDescription.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Parsers/ASTDictionary.h>
|
||||
#include <Poco/String.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
@ -13,6 +14,7 @@
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTWithAlias.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/FieldVisitorHash.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <Core/Field.h>
|
||||
#include <Parsers/ASTWithAlias.h>
|
||||
#include <Parsers/TokenIterator.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorDump.h>
|
||||
|
||||
#include <optional>
|
||||
|
||||
|
@ -1,6 +1,8 @@
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Parsers/formatSettingName.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/FieldVisitorHash.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/SettingsChanges.h>
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <Parsers/ASTSettingsProfileElement.h>
|
||||
#include <Parsers/formatSettingName.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
@ -1,7 +1,6 @@
|
||||
#include <Parsers/ASTWindowDefinition.h>
|
||||
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTRolesOrUsersSet.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <ext/range.h>
|
||||
#include <boost/algorithm/string/predicate.hpp>
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <boost/range/algorithm_ext/erase.hpp>
|
||||
|
||||
|
||||
|
@ -1,13 +1,14 @@
|
||||
#include <Processors/Merges/Algorithms/CollapsingSortedAlgorithm.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
/// Maximum number of messages about incorrect data in the log.
|
||||
#define MAX_ERROR_MESSAGES 10
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorSum.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
|
||||
|
@ -12,6 +12,7 @@
|
||||
#include <Functions/FunctionsConversion.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/FieldVisitorsAccurateComparison.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
#include <Interpreters/Set.h>
|
||||
|
@ -7,6 +7,8 @@
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Common/FieldVisitorHash.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/hex.h>
|
||||
#include <Core/Block.h>
|
||||
|
@ -15,7 +15,7 @@
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
#include <Storages/System/StorageSystemNumbers.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Interpreters/convertFieldToType.h>
|
||||
|
@ -17,10 +17,12 @@
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Parsers/parseQuery.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include <boost/program_options.hpp>
|
||||
|
||||
|
||||
namespace po = boost::program_options;
|
||||
|
||||
using ColumnType = uint32_t;
|
||||
|
Loading…
Reference in New Issue
Block a user