Merge pull request #13091 from ClickHouse/fixed-hash-map-implicit-zero-4

Slightly improve performance of aggregation by UInt8/UInt16 keys, part 4
This commit is contained in:
alexey-milovidov 2020-08-03 19:07:35 +03:00 committed by GitHub
commit f536836eee
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 177 additions and 25 deletions

View File

@ -151,7 +151,8 @@ public:
virtual void addBatchSinglePlaceNotNull(
size_t batch_size, AggregateDataPtr place, const IColumn ** columns, const UInt8 * null_map, Arena * arena) const = 0;
virtual void addBatchSinglePlaceFromInterval(size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0;
virtual void addBatchSinglePlaceFromInterval(
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0;
/** In addition to addBatch, this method collects multiple rows of arguments into array "places"
* as long as they are between offsets[i-1] and offsets[i]. This is used for arrayReduce and
@ -159,7 +160,24 @@ public:
* "places" contains a large number of same values consecutively.
*/
virtual void addBatchArray(
size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena) const = 0;
size_t batch_size,
AggregateDataPtr * places,
size_t place_offset,
const IColumn ** columns,
const UInt64 * offsets,
Arena * arena) const = 0;
/** The case when the aggregation key is UInt8
* and pointers to aggregation states are stored in AggregateDataPtr[256] lookup table.
*/
virtual void addBatchLookupTable8(
size_t batch_size,
AggregateDataPtr * places,
size_t place_offset,
std::function<void(AggregateDataPtr &)> init,
const UInt8 * key,
const IColumn ** columns,
Arena * arena) const = 0;
/** By default all NULLs are skipped during aggregation.
* If it returns nullptr, the default one will be used.
@ -204,6 +222,24 @@ public:
static_cast<const Derived *>(this)->add(places[i] + place_offset, columns, i, arena);
}
void addBatchLookupTable8(
size_t batch_size,
AggregateDataPtr * places,
size_t place_offset,
std::function<void(AggregateDataPtr &)> init,
const UInt8 * key,
const IColumn ** columns,
Arena * arena) const override
{
for (size_t i = 0; i < batch_size; ++i)
{
AggregateDataPtr & place = places[key[i]];
if (unlikely(!place))
init(place);
static_cast<const Derived *>(this)->add(place + place_offset, columns, i, arena);
}
}
void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override
{
for (size_t i = 0; i < batch_size; ++i)
@ -218,7 +254,8 @@ public:
static_cast<const Derived *>(this)->add(place, columns, i, arena);
}
void addBatchSinglePlaceFromInterval(size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override
void addBatchSinglePlaceFromInterval(
size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override
{
for (size_t i = batch_begin; i < batch_end; ++i)
static_cast<const Derived *>(this)->add(place, columns, i, arena);

View File

@ -64,6 +64,8 @@ struct HashMethodOneNumber
/// Is used for default implementation in HashMethodBase.
FieldType getKeyHolder(size_t row, Arena &) const { return unalignedLoad<FieldType>(vec + row * sizeof(FieldType)); }
const FieldType * getKeyData() const { return reinterpret_cast<const FieldType *>(vec); }
};

View File

@ -34,10 +34,11 @@ struct FixedClearableHashTableCell
template <typename Key, typename Allocator = HashTableAllocator>
class FixedClearableHashSet : public FixedHashTable<Key, FixedClearableHashTableCell<Key>, Allocator>
class FixedClearableHashSet : public FixedHashTable<
Key, FixedClearableHashTableCell<Key>, FixedHashTableStoredSize<FixedClearableHashTableCell<Key>>, Allocator>
{
public:
using Base = FixedHashTable<Key, FixedClearableHashTableCell<Key>, Allocator>;
using Base = FixedHashTable<Key, FixedClearableHashTableCell<Key>, FixedHashTableStoredSize<FixedClearableHashTableCell<Key>>, Allocator>;
using LookupResult = typename Base::LookupResult;
void clear()

View File

@ -94,11 +94,16 @@ struct FixedHashMapImplicitZeroCell
};
template <typename Key, typename Mapped, typename Cell = FixedHashMapCell<Key, Mapped>, typename Allocator = HashTableAllocator>
class FixedHashMap : public FixedHashTable<Key, Cell, Allocator>
template <
typename Key,
typename Mapped,
typename Cell = FixedHashMapCell<Key, Mapped>,
typename Size = FixedHashTableStoredSize<Cell>,
typename Allocator = HashTableAllocator>
class FixedHashMap : public FixedHashTable<Key, Cell, Size, Allocator>
{
public:
using Base = FixedHashTable<Key, Cell, Allocator>;
using Base = FixedHashTable<Key, Cell, Size, Allocator>;
using Self = FixedHashMap;
using LookupResult = typename Base::LookupResult;
@ -155,5 +160,19 @@ public:
}
};
template <typename Key, typename Mapped, typename Cell = FixedHashMapImplicitZeroCell<Key, Mapped>, typename Allocator = HashTableAllocator>
using FixedImplicitZeroHashMap = FixedHashMap<Key, Mapped, Cell, Allocator>;
template <typename Key, typename Mapped, typename Allocator = HashTableAllocator>
using FixedImplicitZeroHashMap = FixedHashMap<
Key,
Mapped,
FixedHashMapImplicitZeroCell<Key, Mapped>,
FixedHashTableStoredSize<FixedHashMapImplicitZeroCell<Key, Mapped>>,
Allocator>;
template <typename Key, typename Mapped, typename Allocator = HashTableAllocator>
using FixedImplicitZeroHashMapWithCalculatedSize = FixedHashMap<
Key,
Mapped,
FixedHashMapImplicitZeroCell<Key, Mapped>,
FixedHashTableCalculatedSize<FixedHashMapImplicitZeroCell<Key, Mapped>>,
Allocator>;

View File

@ -3,11 +3,11 @@
#include <Common/HashTable/FixedHashTable.h>
template <typename Key, typename Allocator = HashTableAllocator>
class FixedHashSet : public FixedHashTable<Key, FixedHashTableCell<Key>, Allocator>
class FixedHashSet : public FixedHashTable<Key, FixedHashTableCell<Key>, FixedHashTableStoredSize<FixedHashTableCell<Key>>, Allocator>
{
public:
using Cell = FixedHashTableCell<Key>;
using Base = FixedHashTable<Key, Cell, Allocator>;
using Base = FixedHashTable<Key, Cell, FixedHashTableStoredSize<Cell>, Allocator>;
using Self = FixedHashSet;
void merge(const Self & rhs)

View File

@ -47,6 +47,47 @@ struct FixedHashTableCell
};
/// How to obtain the size of the table.
template <typename Cell>
struct FixedHashTableStoredSize
{
size_t m_size = 0;
size_t getSize(const Cell *, const typename Cell::State &, size_t) const { return m_size; }
bool isEmpty(const Cell *, const typename Cell::State &, size_t) const { return m_size == 0; }
void increaseSize() { ++m_size; }
void clearSize() { m_size = 0; }
void setSize(size_t to) { m_size = to; }
};
template <typename Cell>
struct FixedHashTableCalculatedSize
{
size_t getSize(const Cell * buf, const typename Cell::State & state, size_t num_cells) const
{
size_t res = 0;
for (const Cell * end = buf + num_cells; buf != end; ++buf)
if (!buf->isZero(state))
++res;
return res;
}
bool isEmpty(const Cell * buf, const typename Cell::State & state, size_t num_cells) const
{
for (const Cell * end = buf + num_cells; buf != end; ++buf)
if (!buf->isZero(state))
return false;
return true;
}
void increaseSize() {}
void clearSize() {}
void setSize(size_t) {}
};
/** Used as a lookup table for small keys such as UInt8, UInt16. It's different
* than a HashTable in that keys are not stored in the Cell buf, but inferred
* inside each iterator. There are a bunch of to make it faster than using
@ -63,8 +104,8 @@ struct FixedHashTableCell
* transfer, key updates (f.g. StringRef) and serde. This will allow
* TwoLevelHashSet(Map) to contain different type of sets(maps).
*/
template <typename Key, typename Cell, typename Allocator>
class FixedHashTable : private boost::noncopyable, protected Allocator, protected Cell::State
template <typename Key, typename Cell, typename Size, typename Allocator>
class FixedHashTable : private boost::noncopyable, protected Allocator, protected Cell::State, protected Size
{
static constexpr size_t NUM_CELLS = 1ULL << (sizeof(Key) * 8);
@ -75,7 +116,6 @@ protected:
using Self = FixedHashTable;
size_t m_size = 0; /// Amount of elements
Cell * buf; /// A piece of memory for all elements.
void alloc() { buf = reinterpret_cast<Cell *>(Allocator::alloc(NUM_CELLS * sizeof(Cell))); }
@ -178,7 +218,7 @@ public:
free();
std::swap(buf, rhs.buf);
std::swap(m_size, rhs.m_size);
this->setSize(rhs.size());
Allocator::operator=(std::move(rhs));
Cell::State::operator=(std::move(rhs));
@ -305,7 +345,7 @@ public:
new (&buf[x]) Cell(x, *this);
inserted = true;
++m_size;
this->increaseSize();
}
std::pair<LookupResult, bool> ALWAYS_INLINE insert(const value_type & x)
@ -335,7 +375,7 @@ public:
void write(DB::WriteBuffer & wb) const
{
Cell::State::write(wb);
DB::writeVarUInt(m_size, wb);
DB::writeVarUInt(size(), wb);
if (!buf)
return;
@ -353,7 +393,7 @@ public:
void writeText(DB::WriteBuffer & wb) const
{
Cell::State::writeText(wb);
DB::writeText(m_size, wb);
DB::writeText(size(), wb);
if (!buf)
return;
@ -374,7 +414,9 @@ public:
{
Cell::State::read(rb);
destroyElements();
size_t m_size;
DB::readVarUInt(m_size, rb);
this->setSize(m_size);
free();
alloc();
@ -392,7 +434,9 @@ public:
{
Cell::State::readText(rb);
destroyElements();
size_t m_size;
DB::readText(m_size, rb);
this->setSize(m_size);
free();
alloc();
@ -408,14 +452,13 @@ public:
}
}
size_t size() const { return m_size; }
bool empty() const { return 0 == m_size; }
size_t size() const { return this->getSize(buf, *this, NUM_CELLS); }
bool empty() const { return this->isEmpty(buf, *this, NUM_CELLS); }
void clear()
{
destroyElements();
m_size = 0;
this->clearSize();
memset(static_cast<void *>(buf), 0, NUM_CELLS * sizeof(*buf));
}
@ -425,7 +468,7 @@ public:
void clearAndShrink()
{
destroyElements();
m_size = 0;
this->clearSize();
free();
}
@ -433,6 +476,9 @@ public:
size_t getBufferSizeInCells() const { return NUM_CELLS; }
const Cell * data() const { return buf; }
Cell * data() { return buf; }
#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
size_t getCollisions() const { return 0; }
#endif

View File

@ -521,6 +521,39 @@ void NO_INLINE Aggregator::executeImplBatch(
size_t rows,
AggregateFunctionInstruction * aggregate_instructions) const
{
/// Optimization for special case when there are no aggregate functions.
if (params.aggregates_size == 0)
{
/// For all rows.
AggregateDataPtr place = aggregates_pool->alloc(0);
for (size_t i = 0; i < rows; ++i)
state.emplaceKey(method.data, i, *aggregates_pool).setMapped(place);
return;
}
/// Optimization for special case when aggregating by 8bit key.
if constexpr (std::is_same_v<Method, typename decltype(AggregatedDataVariants::key8)::element_type>)
{
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
{
inst->batch_that->addBatchLookupTable8(
rows,
reinterpret_cast<AggregateDataPtr *>(method.data.data()),
inst->state_offset,
[&](AggregateDataPtr & aggregate_data)
{
aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
createAggregateStates(aggregate_data);
},
state.getKeyData(),
inst->batch_arguments,
aggregates_pool);
}
return;
}
/// Generic case.
PODArray<AggregateDataPtr> places(rows);
/// For all rows.

View File

@ -68,7 +68,7 @@ class IBlockOutputStream;
using AggregatedDataWithoutKey = AggregateDataPtr;
using AggregatedDataWithUInt8Key = FixedImplicitZeroHashMap<UInt8, AggregateDataPtr>;
using AggregatedDataWithUInt8Key = FixedImplicitZeroHashMapWithCalculatedSize<UInt8, AggregateDataPtr>;
using AggregatedDataWithUInt16Key = FixedImplicitZeroHashMap<UInt16, AggregateDataPtr>;
using AggregatedDataWithUInt32Key = HashMap<UInt32, AggregateDataPtr, HashCRC32<UInt32>>;

View File

@ -37,6 +37,11 @@
<!-- 29.1. Lookup-таблица, помещающаяся в L1-кэш. -->
<query>SELECT number % 10 AS k FROM numbers(100000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 10 AS k FROM numbers_mt(1600000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 256 AS k FROM numbers(100000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 256 AS k FROM numbers_mt(1600000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 10 AS k, count() FROM numbers(100000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 10 AS k, count() FROM numbers_mt(1600000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 256 AS k, count() FROM numbers(100000000) GROUP BY k FORMAT Null</query>
@ -46,19 +51,28 @@
<query>SELECT number % 10 AS k, count(), sum(number), avg(number) FROM numbers_mt(1600000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 256 AS k, count(), sum(number), avg(number) FROM numbers(100000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 256 AS k, count(), sum(number), avg(number) FROM numbers_mt(1600000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 256 AS k, count(), sum(number), avg(number), min(number), max(number), uniq(number), any(number), argMin(number, number), argMax(number, number) FROM numbers_mt(160000000) GROUP BY k FORMAT Null</query>
<!-- 29.2. Lookup-таблица, помещающаяся в L2-кэш. -->
<query>SELECT number % 1000 AS k, count() FROM numbers( 100000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 1000 AS k, count() FROM numbers_mt(1600000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 1000 AS k FROM numbers( 100000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 1000 AS k FROM numbers_mt(1600000000) GROUP BY k FORMAT Null</query>
<!-- 30. Хэш-таблица, помещающаяся в L3-кэш. -->
<query>SELECT number % 100000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 100000 AS k, count() FROM numbers_mt(160000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 100000 AS k FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 100000 AS k FROM numbers_mt(160000000) GROUP BY k FORMAT Null</query>
<!-- 31. Хэш-таблица, наверное помещающаяся в L3-кэш. -->
<query>SELECT number % 1000000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 1000000 AS k, count() FROM numbers_mt(160000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 1000000 AS k FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 1000000 AS k FROM numbers_mt(160000000) GROUP BY k FORMAT Null</query>
<!-- 32. Хэш-таблица, не помещающаяся в L3-кэш. -->
<query>SELECT number % 10000000 AS k, count() FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 10000000 AS k, count() FROM numbers_mt(80000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 10000000 AS k FROM numbers( 10000000) GROUP BY k FORMAT Null</query>
<query>SELECT number % 10000000 AS k FROM numbers_mt(80000000) GROUP BY k FORMAT Null</query>
<!-- 33. Хэш-таблица, требующая кучу оперативки. Возможны интересные эффекты. -->
<!-- For this HT size, a single-threaded query that makes sense would be too slow (tens of seconds).
<query>SELECT number % 100000000 AS k, count() FROM numbers( 100000000) GROUP BY k FORMAT Null</query>