Skip to content

Commit

Permalink
Smart pointers for hash tables -- the tests seem to pass
Browse files Browse the repository at this point in the history
  • Loading branch information
akuzm committed Aug 9, 2019
1 parent 048fe08 commit 16d9136
Show file tree
Hide file tree
Showing 6 changed files with 252 additions and 94 deletions.
5 changes: 4 additions & 1 deletion dbms/src/Common/Arena.h
Expand Up @@ -170,11 +170,14 @@ class Arena : private boost::noncopyable

/** Rollback just performed allocation.
* Must pass size not more that was just allocated.
* Return the resulting head pointer, so that the caller can assert that
* the allocation it intended to roll back was indeed the last one.
*/
void rollback(size_t size)
void * rollback(size_t size)
{
head->pos -= size;
ASAN_POISON_MEMORY_REGION(head->pos, size + pad_right);
return head->pos;
}

/** Begin or expand allocation of contiguous piece of memory without alignment.
Expand Down
165 changes: 123 additions & 42 deletions dbms/src/Common/ColumnsHashing.h
@@ -1,6 +1,7 @@
#pragma once


#include <Common/HashTable/HashTable.h>
#include <Common/ColumnsHashingImpl.h>
#include <Common/Arena.h>
#include <Common/LRUCache.h>
Expand All @@ -16,6 +17,45 @@
namespace DB
{

/**
* This is a key holder that serializes the key to arena.
* For types other than StringRef, it does nothing, because they are stored
* by value.
* StringRef is also technically stored by value inside a hash table, but it is
* a pointer type. We copy the data it points to to the arena, and update the
* pointer.
*/
template <typename Key>
struct ArenaKeyPtr : public NoopKeyPtr<Key>
{
ArenaKeyPtr(Key key_, Arena &) : NoopKeyPtr<Key>(key_) {}
};

template <>
struct ArenaKeyPtr<StringRef>
{
StringRef key;
Arena & pool;

ArenaKeyPtr(StringRef key_, Arena & pool_)
: key(key_), pool(pool_) {}

StringRef & persist()
{
if (key.size)
{
key.data = pool.insert(key.data, key.size);
}
return key;
}

StringRef & operator * ()
{
return key;
}
};


namespace ColumnsHashing
{

Expand Down Expand Up @@ -56,7 +96,7 @@ struct HashMethodOneNumber
using Base::getHash; /// (const Data & data, size_t row, Arena & pool) -> size_t

/// Is used for default implementation in HashMethodBase.
FieldType getKey(size_t row, Arena &) const { return unalignedLoad<FieldType>(vec + row * sizeof(FieldType)); }
NoopKeyPtr<FieldType> getKeyPtr(size_t row, Arena &) const { return unalignedLoad<FieldType>(vec + row * sizeof(FieldType)); }

/// Get StringRef from value which can be inserted into column.
static StringRef getValueRef(const Value & value)
Expand Down Expand Up @@ -85,24 +125,20 @@ struct HashMethodString
chars = column_string.getChars().data();
}

auto getKey(ssize_t row, Arena &) const
// FIXME can we simplify this?
using KeyPtr = typename std::conditional<place_string_to_arena,
ArenaKeyPtr<StringRef>,
NoopKeyPtr<StringRef>>::type;

KeyPtr getKeyPtr(ssize_t row, Arena & pool) const
{
return StringRef(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1);
return KeyPtr(StringRef(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1), pool);
}

static StringRef getValueRef(const Value & value) { return value.first; }

protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;

static ALWAYS_INLINE void onNewKey([[maybe_unused]] StringRef & key, [[maybe_unused]] Arena & pool)
{
if constexpr (place_string_to_arena)
{
if (key.size)
key.data = pool.insert(key.data, key.size);
}
}
};


Expand All @@ -125,17 +161,20 @@ struct HashMethodFixedString
chars = &column_string.getChars();
}

StringRef getKey(size_t row, Arena &) const { return StringRef(&(*chars)[row * n], n); }
// FIXME can we simplify this?
using KeyPtr = typename std::conditional<place_string_to_arena,
ArenaKeyPtr<StringRef>,
NoopKeyPtr<StringRef>>::type;

KeyPtr getKeyPtr(size_t row, [[maybe_unused]] Arena & pool) const
{
return KeyPtr(StringRef(&(*chars)[row * n], n), pool);
}

static StringRef getValueRef(const Value & value) { return value.first; }

protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
static ALWAYS_INLINE void onNewKey([[maybe_unused]] StringRef & key, [[maybe_unused]] Arena & pool)
{
if constexpr (place_string_to_arena)
key.data = pool.insert(key.data, key.size);
}
};


Expand Down Expand Up @@ -316,9 +355,9 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
}

/// Get the key from the key columns for insertion into the hash table.
ALWAYS_INLINE auto getKey(size_t row, Arena & pool) const
ALWAYS_INLINE auto getKeyPtr(size_t row, Arena & pool) const
{
return Base::getKey(getIndexAt(row), pool);
return Base::getKeyPtr(getIndexAt(row), pool);
}

template <typename Data>
Expand Down Expand Up @@ -346,30 +385,23 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
return EmplaceResult(false);
}

auto key = getKey(row_, pool);
auto key_ptr = getKeyPtr(row_, pool);

bool inserted = false;
typename Data::iterator it;
if (saved_hash)
data.emplace(key, it, inserted, saved_hash[row]);
data.emplacePtr(key_ptr, it, inserted, saved_hash[row]);
else
data.emplace(key, it, inserted);
data.emplacePtr(key_ptr, it, inserted);

visit_cache[row] = VisitValue::Found;

if (inserted)
if constexpr (has_mapped)
{
if constexpr (has_mapped)
if (inserted)
{
new(&it->getSecond()) Mapped();
Base::onNewKey(it->getFirstMutable(), pool);
new (&it->getSecond()) Mapped();
}
else
Base::onNewKey(*it, pool);
}

if constexpr (has_mapped)
{
mapped_cache[row] = it->getSecond();
return EmplaceResult(it->getSecond(), mapped_cache[row], inserted);
}
Expand Down Expand Up @@ -406,13 +438,13 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
return FindResult(visit_cache[row] == VisitValue::Found);
}

auto key = getKey(row_, pool);
auto key_ptr = getKeyPtr(row_, pool);

typename Data::iterator it;
if (saved_hash)
it = data.find(key, saved_hash[row]);
it = data.find(*key_ptr, saved_hash[row]);
else
it = data.find(key);
it = data.find(*key_ptr);

bool found = it != data.end();
visit_cache[row] = found ? VisitValue::Found : VisitValue::NotFound;
Expand Down Expand Up @@ -492,7 +524,7 @@ struct HashMethodKeysFixed
}
}

ALWAYS_INLINE Key getKey(size_t row, Arena &) const
ALWAYS_INLINE NoopKeyPtr<Key> getKeyPtr(size_t row, Arena &) const
{
if constexpr (has_nullable_keys)
{
Expand All @@ -510,6 +542,52 @@ struct HashMethodKeysFixed
}
};

/**
* This is a key holder for a key that is already serialized to an arena. The
* key must be the last serialized key for this arena.
* In destructor, if the key was made persistent, we do nothing, and if not --
* roll back the last allocation using Arena::rollback().
*/
struct SerializedKeyPtr
{
StringRef key;
Arena * pool;

SerializedKeyPtr(StringRef key_, Arena & pool_) : key(key_), pool(&pool_) {}

StringRef & persist()
{
assert(pool != nullptr);
pool = nullptr;
return key;
}

StringRef & operator * ()
{
return key;
}

~SerializedKeyPtr()
{
if (pool != nullptr)
{
[[maybe_unused]] void * new_head = pool->rollback(key.size);
assert(new_head == key.data);
}
}

SerializedKeyPtr(SerializedKeyPtr && other)
{
key = other.key;
pool = other.pool;
other.pool = nullptr;
}

private:
SerializedKeyPtr(const SerializedKeyPtr &) = delete;
SerializedKeyPtr operator = (const SerializedKeyPtr &) = delete;
};

/** Hash by concatenating serialized key values.
* The serialized value differs in that it uniquely allows to deserialize it, having only the position with which it starts.
* That is, for example, for strings, it contains first the serialized length of the string, and then the bytes.
Expand All @@ -531,12 +609,12 @@ struct HashMethodSerialized
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, false>;

ALWAYS_INLINE StringRef getKey(size_t row, Arena & pool) const
ALWAYS_INLINE SerializedKeyPtr getKeyPtr(size_t row, Arena & pool) const
{
return serializeKeysToPoolContiguous(row, keys_size, key_columns, pool);
return SerializedKeyPtr(
serializeKeysToPoolContiguous(row, keys_size, key_columns, pool),
pool);
}

static ALWAYS_INLINE void onExistingKey(StringRef & key, Arena & pool) { pool.rollback(key.size); }
};

/// For the case when there is one string key.
Expand All @@ -553,7 +631,10 @@ struct HashMethodHashed
HashMethodHashed(ColumnRawPtrs key_columns, const Sizes &, const HashMethodContextPtr &)
: key_columns(std::move(key_columns)) {}

ALWAYS_INLINE Key getKey(size_t row, Arena &) const { return hash128(row, key_columns.size(), key_columns); }
ALWAYS_INLINE NoopKeyPtr<Key> getKeyPtr(size_t row, Arena &) const
{
return hash128(row, key_columns.size(), key_columns);
}

static ALWAYS_INLINE StringRef getValueRef(const Value & value)
{
Expand Down
36 changes: 10 additions & 26 deletions dbms/src/Common/ColumnsHashingImpl.h
Expand Up @@ -115,26 +115,22 @@ class HashMethodBase
template <typename Data>
ALWAYS_INLINE EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
return emplaceKeyImpl(key, data, pool);
auto key_ptr = static_cast<Derived &>(*this).getKeyPtr(row, pool);
return emplaceKeyPtrImpl(key_ptr, data);
}

template <typename Data>
ALWAYS_INLINE FindResult findKey(Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
auto res = findKeyImpl(key, data);
static_cast<Derived &>(*this).onExistingKey(key, pool);
return res;
auto key_ptr = static_cast<Derived &>(*this).getKeyPtr(row, pool);
return findKeyImpl(*key_ptr, data);
}

template <typename Data>
ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
auto res = data.hash(key);
static_cast<Derived &>(*this).onExistingKey(key, pool);
return res;
auto key_ptr = static_cast<Derived &>(*this).getKeyPtr(row, pool);
return data.hash(*key_ptr);
}

protected:
Expand All @@ -155,20 +151,13 @@ class HashMethodBase
}
}

template <typename Key>
static ALWAYS_INLINE void onNewKey(Key & /*key*/, Arena & /*pool*/) {}
template <typename Key>
static ALWAYS_INLINE void onExistingKey(Key & /*key*/, Arena & /*pool*/) {}

template <typename Data, typename Key>
ALWAYS_INLINE EmplaceResult emplaceKeyImpl(Key key, Data & data, Arena & pool)
template <typename Data, typename KeyPtr>
ALWAYS_INLINE EmplaceResult emplaceKeyPtrImpl(KeyPtr & key_ptr, Data & data)
{
if constexpr (Cache::consecutive_keys_optimization)
{
if (cache.found && cache.check(key))
if (cache.found && cache.check(*key_ptr))
{
static_cast<Derived &>(*this).onExistingKey(key, pool);

if constexpr (has_mapped)
return EmplaceResult(cache.value.second, cache.value.second, false);
else
Expand All @@ -178,7 +167,7 @@ class HashMethodBase

typename Data::iterator it;
bool inserted = false;
data.emplace(key, it, inserted);
data.emplacePtr(key_ptr, it, inserted);

[[maybe_unused]] Mapped * cached = nullptr;
if constexpr (has_mapped)
Expand All @@ -189,13 +178,8 @@ class HashMethodBase
if constexpr (has_mapped)
{
new(&it->getSecond()) Mapped();
static_cast<Derived &>(*this).onNewKey(it->getFirstMutable(), pool);
}
else
static_cast<Derived &>(*this).onNewKey(it->getValueMutable(), pool);
}
else
static_cast<Derived &>(*this).onExistingKey(key, pool);

if constexpr (consecutive_keys_optimization)
{
Expand Down
12 changes: 12 additions & 0 deletions dbms/src/Common/HashTable/FixedHashTable.h
Expand Up @@ -294,6 +294,18 @@ class FixedHashTable : private boost::noncopyable, protected Allocator, protecte
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted) { emplaceImpl(x, it, inserted); }
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t) { emplaceImpl(x, it, inserted); }

// FixedHashTable doesn't store references to the keys, so it doesn't care
// about key persistence.
template <typename KeyPtr>
void emplacePtr(KeyPtr key_ptr, iterator & it, bool & inserted) { emplace(*key_ptr, it, inserted); }

template <typename KeyPtr>
void emplacePtr(KeyPtr key_ptr, iterator & it, bool & inserted, size_t)
{
emplace(*key_ptr, it, inserted);
}


template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith x)
{
Expand Down

0 comments on commit 16d9136

Please sign in to comment.