mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
Remove trash from LowCardinality
This commit is contained in:
parent
45dc3faeca
commit
7f68f2c7df
@ -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>
|
||||
|
@ -846,27 +846,12 @@ auto & Field::safeGet()
|
||||
template <typename T>
|
||||
T & Field::reinterpret()
|
||||
{
|
||||
assert(which != Types::String); // See specialization for char
|
||||
assert(which != Types::String);
|
||||
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
|
||||
{
|
||||
|
@ -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