提交 2eb35dd6 编写于 作者: N Nikolai Kochetov

Cache AggregateDataPtr for single low cardinality column. [#CLICKHOUSE-3903]

上级 787e814e
......@@ -91,6 +91,8 @@ public:
index.setColumn(getRawColumnPtr());
}
const UInt64 * tryGetSavedHash() const override { return index.tryGetSavedHash(); }
private:
ColumnPtr column_holder;
......
......@@ -509,6 +509,19 @@ UInt64 ColumnWithDictionary::Index::getMaxPositionForCurrentType() const
return value;
}
size_t ColumnWithDictionary::Index::getPositionAt(size_t row) const
{
size_t pos;
auto getPosition = [&](auto type)
{
using CurIndexType = decltype(type);
pos = getPositionsData<CurIndexType>()[row];
};
callForType(std::move(getPosition), size_of_type);
return pos;
}
void ColumnWithDictionary::Index::insertPosition(UInt64 position)
{
while (position > getMaxPositionForCurrentType())
......
......@@ -144,6 +144,8 @@ public:
/// IColumn & getIndexes() { return idx.getPositions()->assumeMutableRef(); }
const IColumn & getIndexes() const { return *idx.getPositions(); }
const ColumnPtr & getIndexesPtr() const { return idx.getPositions(); }
size_t getIndexAt(size_t row) const { return idx.getPositionAt(row); }
size_t getSizeOfIndexType() const { return idx.getSizeOfIndexType(); }
///void setIndexes(MutableColumnPtr && indexes_) { indexes = std::move(indexes_); }
......@@ -176,6 +178,7 @@ public:
const ColumnPtr & getPositions() const { return positions; }
ColumnPtr & getPositionsPtr() { return positions; }
size_t getPositionAt(size_t row) const;
void insertPosition(UInt64 position);
void insertPositionsRange(const IColumn & column, size_t offset, size_t limit);
......@@ -185,6 +188,7 @@ public:
UInt64 getMaxPositionForCurrentType() const;
static size_t getSizeOfIndexType(const IColumn & column, size_t hint);
size_t getSizeOfIndexType() const { return size_of_type; }
void check(size_t max_dictionary_size);
void checkSizeOfType();
......
#pragma once
#include <Columns/IColumn.h>
#include <Columns/ColumnsNumber.h>
namespace DB
{
......@@ -16,6 +17,8 @@ public:
/// The same as getNestedColumn, but removes null map if nested column is nullable.
virtual const ColumnPtr & getNestedNotNullableColumn() const = 0;
virtual const UInt64 * tryGetSavedHash() const = 0;
size_t size() const override { return getNestedColumn()->size(); }
/// Appends new value at the end of column (column's size is increased by 1).
......
......@@ -261,6 +261,17 @@ public:
ColumnType * getColumn() const { return column; }
size_t size() const;
const UInt64 * tryGetSavedHash() const
{
if (!use_saved_hash)
return nullptr;
if (!saved_hash)
calcHashes();
return &saved_hash->getData()[0];
}
size_t allocatedBytes() const { return index ? index->getBufferSizeInBytes() : 0; }
private:
......@@ -272,7 +283,7 @@ private:
/// Lazy initialized.
std::unique_ptr<IndexMapType> index;
ColumnUInt64::MutablePtr saved_hash;
mutable ColumnUInt64::MutablePtr saved_hash;
void buildIndex();
......@@ -287,6 +298,8 @@ private:
else
return StringRefHash()(ref);
}
void calcHashes() const;
};
......@@ -295,7 +308,10 @@ template <typename IndexType, typename ColumnType>
void ReverseIndex<IndexType, ColumnType>:: setColumn(ColumnType * column_)
{
if (column != column_)
{
index = nullptr;
saved_hash = nullptr;
}
column = column_;
}
......@@ -322,7 +338,13 @@ void ReverseIndex<IndexType, ColumnType>::buildIndex()
index = std::make_unique<IndexMapType>(size);
if constexpr (use_saved_hash)
saved_hash = ColumnUInt64::create(size);
{
if (!saved_hash)
{
saved_hash = ColumnUInt64::create(size);
calcHashes();
}
}
auto & state = index->getState();
state.index_column = column;
......@@ -336,10 +358,11 @@ void ReverseIndex<IndexType, ColumnType>::buildIndex()
for (auto row : ext::range(num_prefix_rows_to_skip, size))
{
auto hash = getHash(column->getDataAt(row));
UInt64 hash;
if constexpr (use_saved_hash)
saved_hash->getElement(row) = hash;
hash = saved_hash->getElement(row);
else
hash = getHash(column->getDataAt(row));
index->emplace(row + base_index, iterator, inserted, hash);
......@@ -348,6 +371,19 @@ void ReverseIndex<IndexType, ColumnType>::buildIndex()
}
}
template <typename IndexType, typename ColumnType>
void ReverseIndex<IndexType, ColumnType>::calcHashes() const
{
if (!column)
throw Exception("ReverseIndex can't build index because index column wasn't set.", ErrorCodes::LOGICAL_ERROR);
auto size = column->size();
saved_hash = ColumnUInt64::create(size);
for (auto row : ext::range(0, size))
saved_hash->getElement(row) = getHash(column->getDataAt(row));
};
template <typename IndexType, typename ColumnType>
UInt64 ReverseIndex<IndexType, ColumnType>::insert(UInt64 from_position)
{
......
......@@ -252,9 +252,8 @@ public:
}
iterator ALWAYS_INLINE find(Key x)
iterator ALWAYS_INLINE find(Key x, size_t hash_value)
{
size_t hash_value = hash(x);
size_t buck = getBucketFromHash(hash_value);
typename Impl::iterator found = impls[buck].find(x, hash_value);
......@@ -264,9 +263,8 @@ public:
}
const_iterator ALWAYS_INLINE find(Key x) const
const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const
{
size_t hash_value = hash(x);
size_t buck = getBucketFromHash(hash_value);
typename Impl::const_iterator found = impls[buck].find(x, hash_value);
......@@ -276,6 +274,10 @@ public:
}
iterator ALWAYS_INLINE find(Key x) { return find(x, hash(x)); }
iterator ALWAYS_INLINE find(Key x) const { return find(x, hash(x)); }
void write(DB::WriteBuffer & wb) const
{
for (size_t i = 0; i < NUM_BUCKETS; ++i)
......
......@@ -577,28 +577,6 @@ void NO_INLINE Aggregator::executeImpl(
executeImplCase<true>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, key_sizes, keys, overflow_row);
}
template <typename Method>
void NO_INLINE Aggregator::executeLowCardinalityImpl(
Method & method,
Arena * aggregates_pool,
size_t rows,
ColumnRawPtrs & key_columns,
ColumnRawPtrs & key_counts,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
bool no_more_keys,
AggregateDataPtr overflow_row) const
{
typename Method::State state;
state.init(key_columns);
if (!no_more_keys)
executeLowCardinalityImplCase<false>(method, state, aggregates_pool, rows, key_columns, key_counts, aggregate_instructions, key_sizes, keys, overflow_row);
else
executeLowCardinalityImplCase<true>(method, state, aggregates_pool, rows, key_columns, key_counts, aggregate_instructions, key_sizes, keys, overflow_row);
}
#ifndef __clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
......@@ -619,15 +597,19 @@ void NO_INLINE Aggregator::executeImplCase(
/// NOTE When editing this code, also pay attention to SpecializedAggregator.h.
/// For all rows.
typename Method::iterator it;
typename Method::Key prev_key;
AggregateDataPtr value = nullptr;
for (size_t i = 0; i < rows; ++i)
{
bool inserted; /// Inserted a new key, or was this key already?
bool overflow = false; /// The new key did not fit in the hash table because of no_more_keys.
bool inserted = false; /// Inserted a new key, or was this key already?
/// Get the key to insert into the hash table.
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
typename Method::Key key;
if constexpr (!Method::low_cardinality_optimization)
key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
AggregateDataPtr * aggregate_data = nullptr;
typename Method::iterator it; /// Is not used if Method::low_cardinality_optimization
if (!no_more_keys) /// Insert.
{
......@@ -637,7 +619,6 @@ void NO_INLINE Aggregator::executeImplCase(
if (i != 0 && key == prev_key)
{
/// Add values to the aggregate functions.
AggregateDataPtr value = Method::getAggregateData(it->second);
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool);
......@@ -648,19 +629,32 @@ void NO_INLINE Aggregator::executeImplCase(
prev_key = key;
}
method.data.emplace(key, it, inserted);
if constexpr (Method::low_cardinality_optimization)
aggregate_data = state.emplaceKeyFromRow(method.data, i, inserted, params.keys_size, keys, *aggregates_pool);
else
{
method.data.emplace(key, it, inserted);
aggregate_data = &Method::getAggregateData(it->second);
}
}
else
{
/// Add only if the key already exists.
inserted = false;
it = method.data.find(key);
if (method.data.end() == it)
overflow = true;
if constexpr (Method::low_cardinality_optimization)
aggregate_data = state.findFromRow(method.data, i);
else
{
it = method.data.find(key);
if (method.data.end() != it)
aggregate_data = &Method::getAggregateData(it->second);
}
}
/// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys.
/// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do.
if (no_more_keys && overflow && !overflow_row)
if (!aggregate_data && !overflow_row)
{
method.onExistingKey(key, keys, *aggregates_pool);
continue;
......@@ -669,21 +663,23 @@ void NO_INLINE Aggregator::executeImplCase(
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
if (inserted)
{
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
/// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
aggregate_data = nullptr;
*aggregate_data = nullptr;
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
if constexpr (!Method::low_cardinality_optimization)
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);
aggregate_data = place;
*aggregate_data = place;
if constexpr (Method::low_cardinality_optimization)
state.cacheAggregateData(i, place);
}
else
method.onExistingKey(key, keys, *aggregates_pool);
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
value = aggregate_data ? *aggregate_data : overflow_row;
/// Add values to the aggregate functions.
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
......@@ -691,100 +687,6 @@ void NO_INLINE Aggregator::executeImplCase(
}
}
template <bool no_more_keys, typename Method>
void NO_INLINE Aggregator::executeLowCardinalityImplCase(
Method & method,
typename Method::State & state,
Arena * aggregates_pool,
size_t rows,
ColumnRawPtrs & key_columns,
ColumnRawPtrs & key_counts,
AggregateFunctionInstruction * aggregate_instructions,
const Sizes & key_sizes,
StringRefs & keys,
AggregateDataPtr overflow_row) const
{
/// NOTE When editing this code, also pay attention to SpecializedAggregator.h.
auto & counts_data = static_cast<const ColumnUInt64 *>(key_counts[0])->getData();
/// For all rows.
typename Method::iterator it;
typename Method::Key prev_key;
for (size_t i = 0; i < rows; ++i)
{
/// Get the key to insert into the hash table.
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
size_t num_repeats = counts_data[i];
for (size_t repeat = 0; repeat < num_repeats; ++repeat)
{
bool inserted; /// Inserted a new key, or was this key already?
bool overflow = false; /// The new key did not fit in the hash table because of no_more_keys.
if (!no_more_keys) /// Insert.
{
/// Optimization for consecutive identical keys.
if (!Method::no_consecutive_keys_optimization)
{
if (i != 0 && (repeat || key == prev_key))
{
/// Add values to the aggregate functions.
AggregateDataPtr value = Method::getAggregateData(it->second);
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool);
method.onExistingKey(key, keys, *aggregates_pool);
continue;
}
else
prev_key = key;
}
method.data.emplace(key, it, inserted);
}
else
{
/// Add only if the key already exists.
inserted = false;
it = method.data.find(key);
if (method.data.end() == it)
overflow = true;
}
/// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do.
if (no_more_keys && overflow && !overflow_row)
{
method.onExistingKey(key, keys, *aggregates_pool);
continue;
}
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
if (inserted)
{
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
/// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
aggregate_data = nullptr;
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);
aggregate_data = place;
}
else
method.onExistingKey(key, keys, *aggregates_pool);
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
/// Add values to the aggregate functions.
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool);
}
}
}
#ifndef __clang__
#pragma GCC diagnostic pop
#endif
......@@ -2039,27 +1941,41 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
for (size_t i = 0; i < rows; ++i)
{
typename Table::iterator it;
AggregateDataPtr * aggregate_data = nullptr;
bool inserted; /// Inserted a new key, or was this key already?
bool overflow = false; /// The new key did not fit in the hash table because of no_more_keys.
bool inserted = false; /// Inserted a new key, or was this key already?
/// Get the key to insert into the hash table.
auto key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
typename Method::Key key;
if constexpr (!Method::low_cardinality_optimization)
key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
if (!no_more_keys)
{
data.emplace(key, it, inserted);
if constexpr (Method::low_cardinality_optimization)
aggregate_data = state.emplaceKeyFromRow(data, i, inserted, params.keys_size, keys, *aggregates_pool);
else
{
data.emplace(key, it, inserted);
aggregate_data = &Method::getAggregateData(it->second);
}
}
else
{
inserted = false;
it = data.find(key);
if (data.end() == it)
overflow = true;
if constexpr (Method::low_cardinality_optimization)
aggregate_data = state.findFromRow(data, i);
else
{
it = data.find(key);
if (data.end() != it)
aggregate_data = &Method::getAggregateData(it->second);
}
}
/// aggregate_date == nullptr means that the new key did not fit in the hash table because of no_more_keys.
/// If the key does not fit, and the data does not need to be aggregated into a separate row, then there's nothing to do.
if (no_more_keys && overflow && !overflow_row)
if (!aggregate_data && !overflow_row)
{
method.onExistingKey(key, keys, *aggregates_pool);
continue;
......@@ -2068,19 +1984,22 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
if (inserted)
{
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
aggregate_data = nullptr;
*aggregate_data = nullptr;
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
if constexpr (!Method::low_cardinality_optimization)
method.onNewKey(*it, params.keys_size, keys, *aggregates_pool);
AggregateDataPtr place = aggregates_pool->alloc(total_size_of_aggregate_states);
createAggregateStates(place);
aggregate_data = place;
*aggregate_data = place;
if constexpr (Method::low_cardinality_optimization)
state.cacheAggregateData(i, place);
}
else
method.onExistingKey(key, keys, *aggregates_pool);
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
AggregateDataPtr value = aggregate_data ? *aggregate_data : overflow_row;
/// Merge state of aggregate functions.
for (size_t j = 0; j < params.aggregates_size; ++j)
......
......@@ -148,7 +148,9 @@ struct AggregationMethodOneNumber
/** Do not use optimization for consecutive keys.
*/
static const bool no_consecutive_keys_optimization = false;
static const bool no_consecutive_keys_optimization = true;
/// Use optimization for low cardinality.
static constexpr bool low_cardinality_optimization = false;
/** Insert the key from the hash table into columns.
*/
......@@ -219,6 +221,7 @@ struct AggregationMethodString
static void onExistingKey(const Key & /*key*/, StringRefs & /*keys*/, Arena & /*pool*/) {}
static const bool no_consecutive_keys_optimization = false;
static constexpr bool low_cardinality_optimization = false;
static StringRef getRef(const typename Data::value_type & value)
{
......@@ -285,6 +288,7 @@ struct AggregationMethodFixedString
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
static const bool no_consecutive_keys_optimization = false;
static constexpr bool low_cardinality_optimization = false;
static StringRef getRef(const typename Data::value_type & value)
{
......@@ -311,7 +315,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod
using iterator = typename Base::iterator;
using const_iterator = typename Base::const_iterator;
Data data;
using Base::data;
AggregationMethodSingleLowCardinalityColumn() = default;
......@@ -321,34 +325,118 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod
struct State : public BaseState
{
ColumnRawPtrs key;
const ColumnWithDictionary * column;
const IColumn * positions = nullptr;
const UInt64 * saved_hash;
PaddedPODArray<AggregateDataPtr> aggregate_data_cache;
size_t size_of_index_type = 0;
/** Called at the start of each block processing.
* Sets the variables needed for the other methods called in inner loops.
*/
void init(ColumnRawPtrs & key_columns)
{
column = typeid_cast<const ColumnWithDictionary *>(key_columns[0]);
auto column = typeid_cast<const ColumnWithDictionary *>(key_columns[0]);
if (!column)
throw Exception("Invalid aggregation key type for AggregationMethodSingleLowCardinalityColumn method. "
"Excepted LowCardinality, got " + key_columns[0]->getName(), ErrorCodes::LOGICAL_ERROR);
key = {column->getDictionary().getNestedColumn().get()};
positions = column->getIndexesPtr().get();
saved_hash = column->getDictionary().tryGetSavedHash();
size_of_index_type = column->getSizeOfIndexType();
BaseState::init(key);
AggregateDataPtr default_data = nullptr;
aggregate_data_cache.assign(key[0]->size(), default_data);
}
size_t getIndexAt(size_t row) const
{
switch (size_of_index_type)
{
case sizeof(UInt8): return static_cast<const ColumnUInt8 *>(positions)->getElement(row);
case sizeof(UInt16): return static_cast<const ColumnUInt16 *>(positions)->getElement(row);
case sizeof(UInt32): return static_cast<const ColumnUInt32 *>(positions)->getElement(row);
case sizeof(UInt64): return static_cast<const ColumnUInt64 *>(positions)->getElement(row);
default: throw Exception("Unexpected size of index type for low cardinality column.", ErrorCodes::LOGICAL_ERROR);
}
}
/// Get the key from the key columns for insertion into the hash table.
Key getKey(
const ColumnRawPtrs & /*key_columns*/,
size_t /*keys_size*/,
size_t i,
const Sizes & key_sizes,
StringRefs & keys,
Arena & pool) const
const ColumnRawPtrs & /*key_columns*/,
size_t /*keys_size*/,
size_t i,
const Sizes & key_sizes,
StringRefs & keys,
Arena & pool) const
{
size_t row = column->getIndexes().getUInt(i);
size_t row = getIndexAt(i);
return BaseState::getKey(key, 1, row, key_sizes, keys, pool);
}
template <typename D>
AggregateDataPtr * emplaceKeyFromRow(
D & data,
size_t i,
bool & inserted,
size_t keys_size,
StringRefs & keys,
Arena & pool)
{
size_t row = getIndexAt(i);
if (aggregate_data_cache[row])
{
inserted = false;
return &aggregate_data_cache[row];
}
else
{
ColumnRawPtrs key_columns;
Sizes key_sizes;
auto key = getKey(key_columns, 0, i, key_sizes, keys, pool);
typename D::iterator it;
if (saved_hash)
data.emplace(key, it, inserted, saved_hash[row]);
else
data.emplace(key, it, inserted);
if (inserted)
Base::onNewKey(*it, keys_size, keys, pool);
else
aggregate_data_cache[row] = Base::getAggregateData(it->second);
return &Base::getAggregateData(it->second);
}
}
void cacheAggregateData(size_t i, AggregateDataPtr data)
{
size_t row = getIndexAt(i);
aggregate_data_cache[row] = data;
}
template <typename D>
AggregateDataPtr * findFromRow(D & data, size_t i)
{
size_t row = getIndexAt(i);
if (!aggregate_data_cache[row])
{
ColumnRawPtrs key_columns;
Sizes key_sizes;
StringRefs keys;
Arena pool;
auto key = getKey(key_columns, 0, i, key_sizes, keys, pool);
typename D::iterator it;
if (saved_hash)
it = data.find(key, saved_hash[row]);
else
it = data.find(key);
if (it != data.end())
aggregate_data_cache[row] = Base::getAggregateData(it->second);
}
return &aggregate_data_cache[row];
}
};
static AggregateDataPtr & getAggregateData(Mapped & value) { return Base::getAggregateData(value); }
......@@ -364,7 +452,8 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod
return Base::onExistingKey(key, keys, pool);
}
using Base::no_consecutive_keys_optimization;
static const bool no_consecutive_keys_optimization = true;
static const bool low_cardinality_optimization = true;
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t /*keys_size*/, const Sizes & /*key_sizes*/)
{
......@@ -528,6 +617,7 @@ struct AggregationMethodKeysFixed
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
static const bool no_consecutive_keys_optimization = false;
static constexpr bool low_cardinality_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes & key_sizes)
{
......@@ -629,6 +719,7 @@ struct AggregationMethodConcat
/// If the key already was, then it is removed from the pool (overwritten), and the next key can not be compared with it.
static const bool no_consecutive_keys_optimization = true;
static constexpr bool low_cardinality_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes & key_sizes)
{
......@@ -714,6 +805,7 @@ struct AggregationMethodSerialized
/// If the key already was, it is removed from the pool (overwritten), and the next key can not be compared with it.
static const bool no_consecutive_keys_optimization = true;
static constexpr bool low_cardinality_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes &)
{
......@@ -770,6 +862,7 @@ struct AggregationMethodHashed
static void onExistingKey(const Key &, StringRefs &, Arena &) {}
static const bool no_consecutive_keys_optimization = false;
static constexpr bool low_cardinality_optimization = false;
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t keys_size, const Sizes &)
{
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册