mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #41457 from ClickHouse/remove-trash-5
Remove trash from Field
This commit is contained in:
commit
d4b9fe41be
@ -5,8 +5,9 @@
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
/** \brief Returns value `from` converted to type `To` while retaining bit representation.
|
||||
* `To` and `From` must satisfy `CopyConstructible`.
|
||||
/** Returns value `from` converted to type `To` while retaining bit representation.
|
||||
* `To` and `From` must satisfy `CopyConstructible`.
|
||||
* In contrast to std::bit_cast can cast types of different width.
|
||||
*/
|
||||
template <typename To, typename From>
|
||||
std::decay_t<To> bit_cast(const From & from)
|
||||
@ -15,13 +16,3 @@ std::decay_t<To> bit_cast(const From & from)
|
||||
memcpy(static_cast<void*>(&res), &from, std::min(sizeof(res), sizeof(from)));
|
||||
return res;
|
||||
}
|
||||
|
||||
/** \brief Returns value `from` converted to type `To` while retaining bit representation.
|
||||
* `To` and `From` must satisfy `CopyConstructible`.
|
||||
*/
|
||||
template <typename To, typename From>
|
||||
std::decay_t<To> safe_bit_cast(const From & from)
|
||||
{
|
||||
static_assert(sizeof(To) == sizeof(From), "bit cast on types of different width");
|
||||
return bit_cast<To, From>(from);
|
||||
}
|
||||
|
@ -137,9 +137,41 @@ Field QueryFuzzer::fuzzField(Field field)
|
||||
break;
|
||||
}
|
||||
}
|
||||
else if (type == Field::Types::Array || type == Field::Types::Tuple)
|
||||
else if (type == Field::Types::Array)
|
||||
{
|
||||
auto & arr = field.reinterpret<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())
|
||||
{
|
||||
|
@ -1,4 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Columns/IColumnUnique.h>
|
||||
#include <Columns/IColumnImpl.h>
|
||||
#include <Columns/ReverseIndex.h>
|
||||
@ -7,16 +8,17 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <base/range.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
#include <base/range.h>
|
||||
#include <base/unaligned.h>
|
||||
#include "Columns/ColumnConst.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -305,17 +307,52 @@ size_t ColumnUnique<ColumnType>::getNullValueIndex() const
|
||||
return 0;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
class FieldVisitorGetData : public StaticVisitor<>
|
||||
{
|
||||
public:
|
||||
StringRef res;
|
||||
|
||||
[[noreturn]] static void throwUnsupported()
|
||||
{
|
||||
throw Exception("Unsupported field type", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
[[noreturn]] void operator() (const Null &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const Array &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const Tuple &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const Map &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const Object &) { throwUnsupported(); }
|
||||
[[noreturn]] void operator() (const AggregateFunctionStateData &) { throwUnsupported(); }
|
||||
void operator() (const String & x) { res = {x.data(), x.size()}; }
|
||||
void operator() (const UInt64 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const UInt128 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const UInt256 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Int64 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Int128 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Int256 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const UUID & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const Float64 & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal32> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal64> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal128> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const DecimalField<Decimal256> & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
void operator() (const bool & x) { res = {reinterpret_cast<const char *>(&x), sizeof(x)}; }
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
||||
{
|
||||
if (x.isNull())
|
||||
return getNullValueIndex();
|
||||
|
||||
if (valuesHaveFixedSize())
|
||||
return uniqueInsertData(&x.reinterpret<char>(), size_of_value_if_fixed);
|
||||
|
||||
const auto & val = x.get<String>();
|
||||
return uniqueInsertData(val.data(), val.size());
|
||||
FieldVisitorGetData visitor;
|
||||
applyVisitor(visitor, x);
|
||||
return uniqueInsertData(visitor.res.data, visitor.res.size);
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
|
@ -94,21 +94,7 @@ public:
|
||||
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);
|
||||
}
|
||||
return x.getValue().template convertTo<T>() / x.getScaleMultiplier().template convertTo<T>();
|
||||
else
|
||||
return (x.getValue() / x.getScaleMultiplier()). template convertTo<T>();
|
||||
}
|
||||
@ -134,4 +120,3 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -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() (UInt64 & x) const
|
||||
{
|
||||
x += rhs.reinterpret<UInt64>();
|
||||
x += applyVisitor(FieldVisitorConvertToNumber<UInt64>(), rhs);
|
||||
return x != 0;
|
||||
}
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -41,7 +42,7 @@ public:
|
||||
requires is_big_int_v<T>
|
||||
bool operator() (T & x) const
|
||||
{
|
||||
x += rhs.reinterpret<T>();
|
||||
x += applyVisitor(FieldVisitorConvertToNumber<T>(), rhs);
|
||||
return x != T(0);
|
||||
}
|
||||
};
|
||||
|
@ -425,16 +425,6 @@ public:
|
||||
bool isNegativeInfinity() const { return which == Types::Null && get<Null>().isNegativeInfinity(); }
|
||||
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)
|
||||
{
|
||||
const Types::Which requested = TypeToEnum<std::decay_t<T>>::value;
|
||||
@ -552,7 +542,7 @@ public:
|
||||
case Types::Float64:
|
||||
{
|
||||
// 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::String: return get<String>() == rhs.get<String>();
|
||||
@ -843,30 +833,6 @@ auto & Field::safeGet()
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
T & Field::reinterpret()
|
||||
{
|
||||
assert(which != Types::String); // See specialization for char
|
||||
using ValueType = std::decay_t<T>;
|
||||
ValueType * MAY_ALIAS ptr = reinterpret_cast<ValueType *>(&storage);
|
||||
return *ptr;
|
||||
}
|
||||
|
||||
// Specialize reinterpreting to char (used in ColumnUnique) to make sure Strings are reinterpreted correctly
|
||||
// inline to avoid multiple definitions
|
||||
template <>
|
||||
inline char & Field::reinterpret<char>()
|
||||
{
|
||||
if (which == Types::String)
|
||||
{
|
||||
// For String we want to return a pointer to the data, not the start of the class
|
||||
// as the layout of std::string depends on the STD version and options
|
||||
char * ptr = reinterpret_cast<String *>(&storage)->data();
|
||||
return *ptr;
|
||||
}
|
||||
return *reinterpret_cast<char *>(&storage);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
Field::Field(T && rhs, enable_if_not_field_or_bool_or_stringlike_t<T>) //-V730
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <mutex>
|
||||
#include <base/bit_cast.h>
|
||||
|
||||
#include <Common/FieldVisitorDump.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
@ -920,8 +921,7 @@ private:
|
||||
ColumnString::Offset current_dst_default_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
Field key = src[i];
|
||||
const auto * it = table.find(key.reinterpret<UInt64>());
|
||||
const auto * it = table.find(bit_cast<UInt64>(src[i]));
|
||||
StringRef ref;
|
||||
|
||||
if (it)
|
||||
@ -1081,6 +1081,22 @@ private:
|
||||
|
||||
mutable Cache cache;
|
||||
|
||||
|
||||
static UInt64 bitCastToUInt64(const Field & x)
|
||||
{
|
||||
switch (x.getType())
|
||||
{
|
||||
case Field::Types::UInt64: return x.get<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.
|
||||
void initialize(const Array & from, const Array & to, const ColumnsWithTypeAndName & arguments) const
|
||||
{
|
||||
@ -1151,20 +1167,8 @@ private:
|
||||
if (key.isNull())
|
||||
continue;
|
||||
|
||||
// Field may be of Float type, but for the purpose of bitwise
|
||||
// equality we can treat them as UInt64, hence the reinterpret().
|
||||
if (to[0].getType() ==Field::Types::Decimal32)
|
||||
{
|
||||
table[key.reinterpret<Decimal32>()] = (*used_to)[i].reinterpret<Decimal32>();
|
||||
}
|
||||
else if (to[0].getType() ==Field::Types::Decimal64)
|
||||
{
|
||||
table[key.reinterpret<Decimal32>()] = (*used_to)[i].reinterpret<Decimal64>();
|
||||
}
|
||||
else
|
||||
{
|
||||
table[key.reinterpret<UInt64>()] = (*used_to)[i].reinterpret<UInt64>();
|
||||
}
|
||||
/// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64
|
||||
table[bitCastToUInt64(key)] = bitCastToUInt64((*used_to)[i]);
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -1179,7 +1183,7 @@ private:
|
||||
|
||||
const String & str_to = to[i].get<const String &>();
|
||||
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 +1197,7 @@ private:
|
||||
{
|
||||
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};
|
||||
table[ref] = (*used_to)[i].reinterpret<UInt64>();
|
||||
table[ref] = bitCastToUInt64((*used_to)[i]);
|
||||
}
|
||||
}
|
||||
else
|
||||
|
@ -6,26 +6,24 @@
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeObject.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
|
||||
#include <Core/AccurateComparison.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Common/FieldVisitorToString.h>
|
||||
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -223,7 +221,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
|
||||
&& (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDate() || which_from_type.isDate32() || which_from_type.isDateTime() || which_from_type.isDateTime64()))
|
||||
{
|
||||
const auto scale = static_cast<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));
|
||||
}
|
||||
}
|
||||
|
@ -471,9 +471,8 @@ size_t ORCBlockOutputFormat::getMaxColumnSize(Chunk & chunk)
|
||||
size_t columns_num = chunk.getNumColumns();
|
||||
size_t max_column_size = 0;
|
||||
for (size_t i = 0; i != columns_num; ++i)
|
||||
{
|
||||
max_column_size = std::max(max_column_size, getColumnSize(*chunk.getColumns()[i], data_types[i]));
|
||||
}
|
||||
|
||||
return max_column_size;
|
||||
}
|
||||
|
||||
@ -481,18 +480,22 @@ void ORCBlockOutputFormat::consume(Chunk chunk)
|
||||
{
|
||||
if (!writer)
|
||||
prepareWriter();
|
||||
|
||||
size_t columns_num = chunk.getNumColumns();
|
||||
size_t rows_num = chunk.getNumRows();
|
||||
|
||||
/// getMaxColumnSize is needed to write arrays.
|
||||
/// The size of the batch must be no less than total amount of array elements.
|
||||
ORC_UNIQUE_PTR<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);
|
||||
|
||||
auto columns = chunk.detachColumns();
|
||||
for (auto & column : columns)
|
||||
column = recursiveRemoveLowCardinality(column);
|
||||
|
||||
for (size_t i = 0; i != columns_num; ++i)
|
||||
writeColumn(*root.fields[i], *columns[i], data_types[i], nullptr);
|
||||
|
||||
root.numElements = rows_num;
|
||||
writer->add(*batch);
|
||||
}
|
||||
|
@ -8,11 +8,13 @@
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <orc/OrcFile.hh>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
|
||||
/// orc::Writer writes only in orc::OutputStream
|
||||
class ORCOutputStream : public orc::OutputStream
|
||||
{
|
||||
@ -21,7 +23,7 @@ public:
|
||||
|
||||
uint64_t getLength() const override;
|
||||
uint64_t getNaturalWriteSize() const override;
|
||||
void write(const void* buf, size_t length) override;
|
||||
void write(const void * buf, size_t length) override;
|
||||
|
||||
void close() override {}
|
||||
const std::string& getName() const override { return name; }
|
||||
@ -31,6 +33,7 @@ private:
|
||||
std::string name = "ORCOutputStream";
|
||||
};
|
||||
|
||||
|
||||
class ORCBlockOutputFormat : public IOutputFormat
|
||||
{
|
||||
public:
|
||||
@ -42,7 +45,7 @@ private:
|
||||
void consume(Chunk chunk) override;
|
||||
void finalizeImpl() override;
|
||||
|
||||
ORC_UNIQUE_PTR<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
|
||||
/// converted to unsigned char (bugprone-signed-char-misuse in clang).
|
||||
@ -75,8 +78,8 @@ private:
|
||||
const FormatSettings format_settings;
|
||||
ORCOutputStream output_stream;
|
||||
DataTypes data_types;
|
||||
ORC_UNIQUE_PTR<orc::Writer> writer;
|
||||
ORC_UNIQUE_PTR<orc::Type> schema;
|
||||
std::unique_ptr<orc::Writer> writer;
|
||||
std::unique_ptr<orc::Type> schema;
|
||||
orc::WriterOptions options;
|
||||
};
|
||||
|
||||
|
@ -242,11 +242,21 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl
|
||||
DataTypePtr const_type;
|
||||
if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type))
|
||||
{
|
||||
if (const_value.getType() == Field::Types::UInt64 || const_value.getType() == Field::Types::Int64 ||
|
||||
const_value.getType() == Field::Types::Float64)
|
||||
if (const_value.getType() == Field::Types::UInt64)
|
||||
{
|
||||
/// Zero in all types is represented in memory the same way as in UInt64.
|
||||
out.function = const_value.reinterpret<UInt64>() ? RPNElement::ALWAYS_TRUE : RPNElement::ALWAYS_FALSE;
|
||||
out.function = const_value.get<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;
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1 @@
|
||||
4908278
|
@ -0,0 +1,8 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_LOCAL} --structure 'x LowCardinality(FixedString(2454139))' --input-format Values --output-format TSV --query "SELECT * FROM table" <<< '(1)' | wc -c
|
Loading…
Reference in New Issue
Block a user