ClickHouse/src/Common/HashTable/HashSet.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

173 lines
5.4 KiB
C++
Raw Normal View History

2014-03-17 02:01:03 +00:00
#pragma once
#include <Common/HashTable/Hash.h>
#include <Common/HashTable/HashTable.h>
#include <Common/HashTable/HashTableAllocator.h>
#include <Common/HashTable/TwoLevelHashTable.h>
2014-03-17 02:01:03 +00:00
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/VarInt.h>
2014-03-17 02:01:03 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
}
/** NOTE HashSet could only be used for memmoveable (position independent) types.
* Example: std::string is not position independent in libstdc++ with C++11 ABI or in libc++.
* Also, key must be of type, that zero bytes is compared equals to zero key.
*/
2014-03-17 02:01:03 +00:00
template <
2014-03-17 02:01:03 +00:00
typename Key,
typename TCell,
2014-03-17 02:01:03 +00:00
typename Hash = DefaultHash<Key>,
typename Grower = HashTableGrowerWithPrecalculation<>,
typename Allocator = HashTableAllocator>
class HashSetTable : public HashTable<Key, TCell, Hash, Grower, Allocator>
2014-03-17 02:01:03 +00:00
{
public:
using Self = HashSetTable;
using Cell = TCell;
using Base = HashTable<Key, TCell, Hash, Grower, Allocator>;
using typename Base::LookupResult;
2014-03-17 02:01:03 +00:00
void merge(const Self & rhs)
{
if (!this->hasZero() && rhs.hasZero())
2014-03-17 02:01:03 +00:00
{
this->setHasZero();
2014-03-17 02:01:03 +00:00
++this->m_size;
}
2014-03-17 02:01:03 +00:00
for (size_t i = 0; i < rhs.grower.bufSize(); ++i)
if (!rhs.buf[i].isZero(*this))
2020-04-20 20:19:03 +00:00
this->insert(rhs.buf[i].getValue());
2014-03-17 02:01:03 +00:00
}
2014-03-17 02:01:03 +00:00
void readAndMerge(DB::ReadBuffer & rb)
{
Cell::State::read(rb);
2014-03-17 02:01:03 +00:00
size_t new_size = 0;
DB::readVarUInt(new_size, rb);
2014-03-17 02:01:03 +00:00
this->resize(new_size);
2014-03-17 02:01:03 +00:00
for (size_t i = 0; i < new_size; ++i)
{
Cell x;
x.read(rb);
2020-04-20 20:19:03 +00:00
this->insert(x.getValue());
2014-03-17 02:01:03 +00:00
}
}
};
template <
typename Key,
typename TCell, /// Supposed to have no state (HashTableNoState)
typename Hash = DefaultHash<Key>,
typename Grower = TwoLevelHashTableGrower<>,
typename Allocator = HashTableAllocator>
class TwoLevelHashSetTable
: public TwoLevelHashTable<Key, TCell, Hash, Grower, Allocator, HashSetTable<Key, TCell, Hash, Grower, Allocator>>
{
public:
using Self = TwoLevelHashSetTable;
using Base = TwoLevelHashTable<Key, TCell, Hash, Grower, Allocator, HashSetTable<Key, TCell, Hash, Grower, Allocator>>;
using Base::Base;
/// Writes its content in a way that it will be correctly read by HashSetTable.
/// Used by uniqExact to preserve backward compatibility.
void writeAsSingleLevel(DB::WriteBuffer & wb) const
{
DB::writeVarUInt(this->size(), wb);
bool zero_written = false;
for (size_t i = 0; i < Base::NUM_BUCKETS; ++i)
{
if (this->impls[i].hasZero())
{
if (zero_written)
throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "No more than one zero value expected");
this->impls[i].zeroValue()->write(wb);
zero_written = true;
}
}
static constexpr HashTableNoState state;
for (auto ptr = this->begin(); ptr != this->end(); ++ptr)
if (!ptr.getPtr()->isZero(state))
ptr.getPtr()->write(wb);
}
};
template <typename Key, typename Hash, typename TState = HashTableNoState>
struct HashSetCellWithSavedHash : public HashTableCell<Key, Hash, TState>
{
using Base = HashTableCell<Key, Hash, TState>;
size_t saved_hash;
2021-05-08 15:20:40 +00:00
HashSetCellWithSavedHash() : Base() {} //-V730
HashSetCellWithSavedHash(const Key & key_, const typename Base::State & state) : Base(key_, state) {} //-V730
2020-08-02 22:20:35 +00:00
bool keyEquals(const Key & key_) const { return bitEquals(this->key, key_); }
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && bitEquals(this->key, key_); }
bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); }
void setHash(size_t hash_value) { saved_hash = hash_value; }
2017-12-01 18:36:55 +00:00
size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
};
template <
typename Key,
typename Hash = DefaultHash<Key>,
typename Grower = HashTableGrowerWithPrecalculation<>,
typename Allocator = HashTableAllocator>
using HashSet = HashSetTable<Key, HashTableCell<Key, Hash>, Hash, Grower, Allocator>;
template <
typename Key,
typename Hash = DefaultHash<Key>,
typename Grower = TwoLevelHashTableGrower<>,
typename Allocator = HashTableAllocator>
using TwoLevelHashSet = TwoLevelHashSetTable<Key, HashTableCell<Key, Hash>, Hash, Grower, Allocator>;
2020-05-26 05:54:04 +00:00
template <typename Key, typename Hash, size_t initial_size_degree>
using HashSetWithStackMemory = HashSet<
Key,
Hash,
HashTableGrower<initial_size_degree>,
HashTableAllocatorWithStackMemory<
(1ULL << initial_size_degree)
* sizeof(HashTableCell<Key, Hash>)>>;
template <
typename Key,
typename Hash = DefaultHash<Key>,
typename Grower = HashTableGrowerWithPrecalculation<>,
typename Allocator = HashTableAllocator>
using HashSetWithSavedHash = HashSetTable<Key, HashSetCellWithSavedHash<Key, Hash>, Hash, Grower, Allocator>;
2020-05-26 05:54:04 +00:00
template <typename Key, typename Hash, size_t initial_size_degree>
using HashSetWithSavedHashWithStackMemory = HashSetWithSavedHash<
Key,
Hash,
HashTableGrower<initial_size_degree>,
HashTableAllocatorWithStackMemory<
(1ULL << initial_size_degree)
* sizeof(HashSetCellWithSavedHash<Key, Hash>)>>;