ClickHouse/src/Interpreters/SetVariants.cpp

175 lines
5.5 KiB
C++
Raw Normal View History

#include <Columns/ColumnString.h>
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
#include <Columns/ColumnConst.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
#include <Interpreters/SetVariants.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2017-04-07 16:13:25 +00:00
template <typename Variant>
void SetVariantsTemplate<Variant>::init(Type type_)
{
type = type_;
switch (type)
{
case Type::EMPTY: break;
#define M(NAME) \
2020-03-18 02:02:24 +00:00
case Type::NAME: (NAME) = std::make_unique<typename decltype(NAME)::element_type>(); break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
}
2017-04-07 16:13:25 +00:00
template <typename Variant>
size_t SetVariantsTemplate<Variant>::getTotalRowCount() const
{
switch (type)
{
case Type::EMPTY: return 0;
#define M(NAME) \
2020-03-18 02:02:24 +00:00
case Type::NAME: return (NAME)->data.size();
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
__builtin_unreachable();
}
2017-04-07 16:13:25 +00:00
template <typename Variant>
size_t SetVariantsTemplate<Variant>::getTotalByteCount() const
{
switch (type)
{
case Type::EMPTY: return 0;
#define M(NAME) \
2020-03-18 02:02:24 +00:00
case Type::NAME: return (NAME)->data.getBufferSizeInBytes();
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
__builtin_unreachable();
}
2017-04-07 16:13:25 +00:00
template <typename Variant>
typename SetVariantsTemplate<Variant>::Type SetVariantsTemplate<Variant>::chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes)
{
/// Check if at least one of the specified keys is nullable.
/// Create a set of nested key columns from the corresponding key columns.
/// Here "nested" means that, if a key column is nullable, we take its nested
/// column; otherwise we take the key column as is.
ColumnRawPtrs nested_key_columns;
nested_key_columns.reserve(key_columns.size());
bool has_nullable_key = false;
for (const auto & col : key_columns)
{
2020-04-22 06:01:33 +00:00
if (const auto * nullable = checkAndGetColumn<ColumnNullable>(*col))
{
2019-06-26 17:20:33 +00:00
nested_key_columns.push_back(&nullable->getNestedColumn());
has_nullable_key = true;
}
else
nested_key_columns.push_back(col);
}
size_t keys_size = nested_key_columns.size();
bool all_fixed = true;
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j)
{
if (!nested_key_columns[j]->isFixedAndContiguous())
{
all_fixed = false;
break;
}
key_sizes[j] = nested_key_columns[j]->sizeOfValueIfFixed();
keys_bytes += key_sizes[j];
}
if (has_nullable_key)
{
/// At least one key is nullable. Therefore we choose a method
/// that takes into account this fact.
if ((keys_size == 1) && (nested_key_columns[0]->isNumeric()))
{
/// We have exactly one key and it is nullable. We shall add it a tag
/// which specifies whether its value is null or not.
size_t size_of_field = nested_key_columns[0]->sizeOfValueIfFixed();
if ((size_of_field == 1) || (size_of_field == 2) || (size_of_field == 4) || (size_of_field == 8))
2017-04-07 16:13:25 +00:00
return Type::nullable_keys128;
/// Pass to more generic method
}
if (all_fixed)
{
/// Pack if possible all the keys along with information about which key values are nulls
/// into a fixed 16- or 32-byte blob.
if (keys_bytes > (std::numeric_limits<size_t>::max() - std::tuple_size<KeysNullMap<UInt128>>::value))
throw Exception{"Aggregator: keys sizes overflow", ErrorCodes::LOGICAL_ERROR};
if ((std::tuple_size<KeysNullMap<UInt128>>::value + keys_bytes) <= 16)
2017-04-07 16:13:25 +00:00
return Type::nullable_keys128;
2021-01-27 00:54:57 +00:00
if ((std::tuple_size<KeysNullMap<UInt256>>::value + keys_bytes) <= 32)
2017-04-07 16:13:25 +00:00
return Type::nullable_keys256;
}
/// Fallback case.
2017-04-07 16:13:25 +00:00
return Type::hashed;
}
2017-04-02 17:37:49 +00:00
/// If there is one numeric key that fits into 64 bits
if (keys_size == 1 && nested_key_columns[0]->isNumeric() && !nested_key_columns[0]->lowCardinality())
{
size_t size_of_field = nested_key_columns[0]->sizeOfValueIfFixed();
if (size_of_field == 1)
2017-04-07 16:13:25 +00:00
return Type::key8;
if (size_of_field == 2)
2017-04-07 16:13:25 +00:00
return Type::key16;
if (size_of_field == 4)
2017-04-07 16:13:25 +00:00
return Type::key32;
if (size_of_field == 8)
2017-04-07 16:13:25 +00:00
return Type::key64;
if (size_of_field == 16)
return Type::keys128;
if (size_of_field == 32)
return Type::keys256;
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.", ErrorCodes::LOGICAL_ERROR);
}
2017-04-02 17:37:49 +00:00
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys
if (all_fixed && keys_bytes <= 16)
2017-04-07 16:13:25 +00:00
return Type::keys128;
if (all_fixed && keys_bytes <= 32)
2017-04-07 16:13:25 +00:00
return Type::keys256;
/// If there is single string key, use hash table of it's values.
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
if (keys_size == 1
&& (typeid_cast<const ColumnString *>(nested_key_columns[0])
|| (isColumnConst(*nested_key_columns[0]) && typeid_cast<const ColumnString *>(&assert_cast<const ColumnConst *>(nested_key_columns[0])->getDataColumn()))))
2017-04-07 16:13:25 +00:00
return Type::key_string;
if (keys_size == 1 && typeid_cast<const ColumnFixedString *>(nested_key_columns[0]))
2017-04-07 16:13:25 +00:00
return Type::key_fixed_string;
/// Otherwise, will use set of cryptographic hashes of unambiguously serialized values.
2017-04-07 16:13:25 +00:00
return Type::hashed;
}
2017-04-08 00:59:53 +00:00
template struct SetVariantsTemplate<NonClearableSet>;
template struct SetVariantsTemplate<ClearableSet>;
}