A Proper lookup table that uses HashTable's API

This is the first step of allowing heterogeneous cells in hash tables.

performance test results are

```

1. HashMap<UInt16, UInt8, TrivialHash, HashTableFixedGrower<16>>;
2. NewLookupMap<UInt16, UInt8>

ResolutionWidth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................223550276.46
ResolutionWidth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................248772721.24
Best: 2 - 24877272124

ResolutionWidth 100000 1 ..........................................................................................................................................................................................................................................................238498413.99
ResolutionWidth 100000 2 ..........................................................................................................................................................................................................................................................261808889.98
Best: 2 - 26180888998

ResolutionWidth 300000 1 ...................................................................................239307348.81
ResolutionWidth 300000 2 ...................................................................................257592761.30
Best: 2 - 25759276130

ResolutionWidth 1000000 1 .........................240144759.26
ResolutionWidth 1000000 2 .........................257093531.91
Best: 2 - 25709353191

ResolutionWidth 5000000 1 .....241573260.35
ResolutionWidth 5000000 2 .....259314162.79
Best: 2 - 25931416279

ResolutionDepth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................217108119.84
ResolutionDepth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................249459504.41
Best: 2 - 24945950441

ResolutionDepth 100000 1 ..........................................................................................................................................................................................................................................................229065162.17
ResolutionDepth 100000 2 ..........................................................................................................................................................................................................................................................253769105.64
Best: 2 - 25376910564

ResolutionDepth 300000 1 ...................................................................................233079225.18
ResolutionDepth 300000 2 ...................................................................................256316273.78
Best: 2 - 25631627378

ResolutionDepth 1000000 1 .........................234184633.51
ResolutionDepth 1000000 2 .........................261100491.57
Best: 2 - 26110049157

ResolutionDepth 5000000 1 .....233118795.66
ResolutionDepth 5000000 2 .....252436160.41
Best: 2 - 25243616041

```
This commit is contained in:
Amos Bird 2019-02-28 17:35:38 +08:00
parent ca752504fc
commit 26ab5dd7a7
56 changed files with 1020 additions and 245 deletions

View File

@ -577,7 +577,7 @@ public:
{
for (auto & elem : table)
{
Histogram & histogram = elem.second;
Histogram & histogram = elem.getSecond();
if (histogram.buckets.size() < params.num_buckets_cutoff)
{
@ -591,7 +591,7 @@ public:
{
for (auto & elem : table)
{
Histogram & histogram = elem.second;
Histogram & histogram = elem.getSecond();
if (!histogram.total)
continue;
@ -623,7 +623,7 @@ public:
{
for (auto & elem : table)
{
Histogram & histogram = elem.second;
Histogram & histogram = elem.getSecond();
if (!histogram.total)
continue;
@ -639,7 +639,7 @@ public:
{
for (auto & elem : table)
{
Histogram & histogram = elem.second;
Histogram & histogram = elem.getSecond();
if (!histogram.total)
continue;
@ -674,7 +674,7 @@ public:
while (true)
{
it = table.find(hashContext(code_points.data() + code_points.size() - context_size, code_points.data() + code_points.size()));
if (table.end() != it && it->second.total + it->second.count_end != 0)
if (table.end() != it && it->getSecond().total + it->getSecond().count_end != 0)
break;
if (context_size == 0)
@ -708,7 +708,7 @@ public:
if (num_bytes_after_desired_size > 0)
end_probability_multiplier = std::pow(1.25, num_bytes_after_desired_size);
CodePoint code = it->second.sample(determinator, end_probability_multiplier);
CodePoint code = it->getSecond().sample(determinator, end_probability_multiplier);
if (code == END)
break;

View File

@ -54,7 +54,7 @@ struct EntropyData
void merge(const EntropyData & rhs)
{
for (const auto & pair : rhs.map)
map[pair.first] += pair.second;
map[pair.getFirst()] += pair.getSecond();
}
void serialize(WriteBuffer & buf) const
@ -68,7 +68,7 @@ struct EntropyData
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
map[pair.getFirst()] = pair.getSecond();
}
}
@ -76,12 +76,12 @@ struct EntropyData
{
UInt64 total_value = 0;
for (const auto & pair : map)
total_value += pair.second;
total_value += pair.getSecond();
Float64 shannon_entropy = 0;
for (const auto & pair : map)
{
Float64 frequency = Float64(pair.second) / total_value;
Float64 frequency = Float64(pair.getSecond()) / total_value;
shannon_entropy -= frequency * log2(frequency);
}

View File

@ -94,7 +94,7 @@ public:
size_t i = 0;
for (auto it = set.begin(); it != set.end(); ++it, ++i)
data_to[old_size + i] = *it;
data_to[old_size + i] = it->getValue();
}
const char * getHeaderFilePath() const override { return __FILE__; }
@ -150,7 +150,7 @@ public:
for (const auto & elem : set)
{
writeStringBinary(elem, buf);
writeStringBinary(elem.getValue(), buf);
}
}
@ -185,7 +185,7 @@ public:
else
{
if (inserted)
it->data = arena->insert(str_serialized.data, str_serialized.size);
it->getValueMutable().data = arena->insert(str_serialized.data, str_serialized.size);
}
}
@ -198,9 +198,9 @@ public:
State::Set::iterator it;
for (auto & rhs_elem : rhs_set)
{
cur_set.emplace(rhs_elem, it, inserted);
if (inserted && it->size)
it->data = arena->insert(it->data, it->size);
cur_set.emplace(rhs_elem.getValue(), it, inserted);
if (inserted && it->getValue().size)
it->getValueMutable().data = arena->insert(it->getValue().data, it->getValue().size);
}
}
@ -215,7 +215,7 @@ public:
for (auto & elem : set)
{
deserializeAndInsert(elem, data_to);
deserializeAndInsert(elem.getValue(), data_to);
}
}

View File

@ -48,7 +48,7 @@ struct QuantileExactWeighted
void merge(const QuantileExactWeighted & rhs)
{
for (const auto & pair : rhs.map)
map[pair.first] += pair.second;
map[pair.getFirst()] += pair.getSecond();
}
void serialize(WriteBuffer & buf) const
@ -62,7 +62,7 @@ struct QuantileExactWeighted
while (reader.next())
{
const auto & pair = reader.get();
map[pair.first] = pair.second;
map[pair.getFirst()] = pair.getSecond();
}
}
@ -83,12 +83,12 @@ struct QuantileExactWeighted
UInt64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.second;
array[i] = pair;
sum_weight += pair.getSecond();
array[i] = pair.getValue();
++i;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.getFirst() < b.getFirst(); });
UInt64 threshold = std::ceil(sum_weight * level);
UInt64 accumulated = 0;
@ -97,7 +97,7 @@ struct QuantileExactWeighted
const Pair * end = array + size;
while (it < end)
{
accumulated += it->second;
accumulated += it->getSecond();
if (accumulated >= threshold)
break;
@ -108,7 +108,7 @@ struct QuantileExactWeighted
if (it == end)
--it;
return it->first;
return it->getFirst();
}
/// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address.
@ -133,12 +133,12 @@ struct QuantileExactWeighted
UInt64 sum_weight = 0;
for (const auto & pair : map)
{
sum_weight += pair.second;
array[i] = pair;
sum_weight += pair.getSecond();
array[i] = pair.getValue();
++i;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.getFirst() < b.getFirst(); });
UInt64 accumulated = 0;
@ -150,11 +150,11 @@ struct QuantileExactWeighted
while (it < end)
{
accumulated += it->second;
accumulated += it->getSecond();
while (accumulated >= threshold)
{
result[indices[level_index]] = it->first;
result[indices[level_index]] = it->getFirst();
++level_index;
if (level_index == num_levels)
@ -168,7 +168,7 @@ struct QuantileExactWeighted
while (level_index < num_levels)
{
result[indices[level_index]] = array[size - 1].first;
result[indices[level_index]] = array[size - 1].getFirst();
++level_index;
}
}

View File

@ -33,7 +33,7 @@ namespace
data.resize(hash_map.size());
for (auto val : hash_map)
data[val.second] = val.first;
data[val.getSecond()] = val.getFirst();
for (auto & ind : index)
ind = hash_map[ind];

View File

@ -414,7 +414,7 @@ UInt64 ReverseIndex<IndexType, ColumnType>::insert(const StringRef & data)
column->popBack(1);
}
return *iterator;
return iterator->getValue();
}
template <typename IndexType, typename ColumnType>
@ -429,7 +429,7 @@ UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPoint(const StringRef &
auto hash = getHash(data);
iterator = index->find(data, hash);
return iterator == index->end() ? size() + base_index : *iterator;
return iterator == index->end() ? size() + base_index : iterator->getValue();
}
}

View File

@ -56,7 +56,7 @@ struct HashMethodOneNumber
/// Get StringRef from value which can be inserted into column.
static StringRef getValueRef(const Value & value)
{
return StringRef(reinterpret_cast<const char *>(&value.first), sizeof(value.first));
return StringRef(reinterpret_cast<const char *>(&value.getFirst()), sizeof(value.getFirst()));
}
};
@ -85,7 +85,7 @@ struct HashMethodString
return StringRef(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1);
}
static StringRef getValueRef(const Value & value) { return StringRef(value.first.data, value.first.size); }
static StringRef getValueRef(const Value & value) { return StringRef(value.getFirst().data, value.getFirst().size); }
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
@ -122,7 +122,7 @@ struct HashMethodFixedString
StringRef getKey(size_t row, Arena &) const { return StringRef(&(*chars)[row * n], n); }
static StringRef getValueRef(const Value & value) { return StringRef(value.first.data, value.first.size); }
static StringRef getValueRef(const Value & value) { return StringRef(value.getFirst().data, value.getFirst().size); }
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
@ -356,8 +356,8 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
{
if constexpr (has_mapped)
{
new(&it->second) Mapped();
Base::onNewKey(it->first, pool);
new(&it->getSecond()) Mapped();
Base::onNewKey(it->getFirstMutable(), pool);
}
else
Base::onNewKey(*it, pool);
@ -365,8 +365,8 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
if constexpr (has_mapped)
{
mapped_cache[row] = it->second;
return EmplaceResult(it->second, mapped_cache[row], inserted);
mapped_cache[row] = it->getSecond();
return EmplaceResult(it->getSecond(), mapped_cache[row], inserted);
}
else
return EmplaceResult(inserted);

View File

@ -39,7 +39,7 @@ struct LastElementCache
bool check(const Value & value_) { return !empty && value == value_; }
template <typename Key>
bool check(const Key & key) { return !empty && value.first == key; }
bool check(const Key & key) { return !empty && value.getFirst() == key; }
};
template <typename Data>
@ -147,9 +147,8 @@ protected:
if constexpr (has_mapped)
{
/// Init PairNoInit elements.
cache.value.second = Mapped();
using Key = decltype(cache.value.first);
cache.value.first = Key();
cache.value.getSecond() = Mapped();
cache.value.getFirstMutable() = {};
}
else
cache.value = Value();
@ -171,7 +170,7 @@ protected:
static_cast<Derived &>(*this).onExistingKey(key, pool);
if constexpr (has_mapped)
return EmplaceResult(cache.value.second, cache.value.second, false);
return EmplaceResult(cache.value.getSecond(), cache.value.getSecond(), false);
else
return EmplaceResult(false);
}
@ -183,33 +182,33 @@ protected:
[[maybe_unused]] Mapped * cached = nullptr;
if constexpr (has_mapped)
cached = &it->second;
cached = &it->getSecond();
if (inserted)
{
if constexpr (has_mapped)
{
new(&it->second) Mapped();
static_cast<Derived &>(*this).onNewKey(it->first, pool);
new(&it->getSecond()) Mapped();
static_cast<Derived &>(*this).onNewKey(it->getFirstMutable(), pool);
}
else
static_cast<Derived &>(*this).onNewKey(*it, pool);
static_cast<Derived &>(*this).onNewKey(it->getValueMutable(), pool);
}
else
static_cast<Derived &>(*this).onExistingKey(key, pool);
if constexpr (consecutive_keys_optimization)
{
cache.value = *it;
cache.value = it->getValue();
cache.found = true;
cache.empty = false;
if constexpr (has_mapped)
cached = &cache.value.second;
cached = &cache.value.getSecond();
}
if constexpr (has_mapped)
return EmplaceResult(it->second, *cached, inserted);
return EmplaceResult(it->getSecond(), *cached, inserted);
else
return EmplaceResult(inserted);
}
@ -222,7 +221,7 @@ protected:
if (cache.check(key))
{
if constexpr (has_mapped)
return FindResult(&cache.value.second, cache.found);
return FindResult(&cache.value.getSecond(), cache.found);
else
return FindResult(cache.found);
}
@ -237,18 +236,18 @@ protected:
cache.empty = false;
if (found)
cache.value = *it;
cache.value = it->getValue();
else
{
if constexpr (has_mapped)
cache.value.first = key;
cache.value.getFirstMutable() = key;
else
cache.value = key;
}
}
if constexpr (has_mapped)
return FindResult(found ? &it->second : nullptr, found);
return FindResult(found ? &it->getSecond() : nullptr, found);
else
return FindResult(found);
}

View File

@ -137,12 +137,12 @@ public:
if (rhs.getContainerType() == details::ContainerType::SMALL)
{
for (const auto & x : rhs.small)
insert(x);
insert(x.getValue());
}
else if (rhs.getContainerType() == details::ContainerType::MEDIUM)
{
for (const auto & x : rhs.getContainer<Medium>())
insert(x);
insert(x.getValue());
}
else if (rhs.getContainerType() == details::ContainerType::LARGE)
getContainer<Large>().merge(rhs.getContainer<Large>());
@ -234,7 +234,7 @@ private:
auto tmp_medium = std::make_unique<Medium>();
for (const auto & x : small)
tmp_medium->insert(x);
tmp_medium->insert(x.getValue());
medium = tmp_medium.release();
setContainerType(details::ContainerType::MEDIUM);
@ -253,12 +253,12 @@ private:
if (container_type == details::ContainerType::SMALL)
{
for (const auto & x : small)
tmp_large->insert(x);
tmp_large->insert(x.getValue());
}
else if (container_type == details::ContainerType::MEDIUM)
{
for (const auto & x : getContainer<Medium>())
tmp_large->insert(x);
tmp_large->insert(x.getValue());
destroy();
}

View File

@ -37,9 +37,9 @@ public:
this->emplace(x, it, inserted);
if (inserted)
new(&it->second) mapped_type();
new(&it->getSecond()) mapped_type();
return it->second;
return it->getSecond();
}
void clear()

View File

@ -0,0 +1,73 @@
#pragma once
#include <Common/HashTable/ClearableHashMap.h>
#include <Common/HashTable/FixedHashMap.h>
template <typename Key, typename TMapped>
struct FixedClearableHashMapCell
{
using Mapped = TMapped;
using State = ClearableHashSetState;
using value_type = PairNoInit<Key, Mapped>;
UInt32 version;
Mapped mapped;
FixedClearableHashMapCell() {}
FixedClearableHashMapCell(const Key &, const State & state) : version(state.version) {}
FixedClearableHashMapCell(const value_type & value_, const State & state) : version(state.version), mapped(value_.second) {}
Mapped & getSecond() { return mapped; }
const Mapped & getSecond() const { return mapped; }
bool isZero(const State & state) const { return version != state.version; }
void setZero() { version = 0; }
static constexpr bool need_zero_value_storage = false;
void setMapped(const value_type & value) { mapped = value.getSecond(); }
struct CellExt
{
CellExt() {}
CellExt(Key && key_, FixedClearableHashMapCell * ptr_) : key(key_), ptr(ptr_) {}
void update(Key && key_, FixedClearableHashMapCell * ptr_)
{
key = key_;
ptr = ptr_;
}
Key key;
FixedClearableHashMapCell * ptr;
Key & getFirstMutable() { return key; }
const Key & getFirst() const { return key; }
Mapped & getSecond() { return ptr->mapped; }
const Mapped & getSecond() const { return *ptr->mapped; }
const value_type getValue() const { return {key, *ptr->mapped}; }
};
};
template <typename Key, typename Mapped, typename Allocator = HashTableAllocator>
class FixedClearableHashMap : public FixedHashMap<Key, FixedClearableHashMapCell<Key, Mapped>, Allocator>
{
public:
using key_type = Key;
using mapped_type = Mapped;
using value_type = typename FixedClearableHashMap::cell_type::value_type;
mapped_type & operator[](Key x)
{
typename FixedClearableHashMap::iterator it;
bool inserted;
this->emplace(x, it, inserted);
if (inserted)
new (&it->second) mapped_type();
return it->second;
}
void clear()
{
++this->version;
this->m_size = 0;
}
};

View File

@ -0,0 +1,45 @@
#pragma once
#include <Common/HashTable/ClearableHashSet.h>
#include <Common/HashTable/FixedHashTable.h>
template <typename Key>
struct FixedClearableHashTableCell
{
using State = ClearableHashSetState;
using value_type = Key;
UInt32 version;
FixedClearableHashTableCell() {}
FixedClearableHashTableCell(const Key &, const State & state) : version(state.version) {}
bool isZero(const State & state) const { return version != state.version; }
void setZero() { version = 0; }
static constexpr bool need_zero_value_storage = false;
void setMapped(const value_type & /*value*/) {}
struct CellExt
{
Key key;
value_type & getValueMutable() { return key; }
const value_type & getValue() const { return key; }
void update(Key && key_, FixedClearableHashTableCell *) { key = key_; }
};
};
template <typename Key, typename Allocator = HashTableAllocator>
class FixedClearableHashSet : public FixedHashTable<Key, FixedClearableHashTableCell<Key>, Allocator>
{
public:
using key_type = Key;
using value_type = typename FixedClearableHashSet::cell_type::value_type;
void clear()
{
++this->version;
this->m_size = 0;
}
};

View File

@ -0,0 +1,72 @@
#pragma once
#include <Common/HashTable/FixedHashTable.h>
#include <Common/HashTable/HashMap.h>
template <typename Key, typename TMapped, typename TState = HashTableNoState>
struct FixedHashMapCell
{
using Mapped = TMapped;
using State = TState;
using value_type = PairNoInit<Key, Mapped>;
bool full;
Mapped mapped;
FixedHashMapCell() {}
FixedHashMapCell(const Key &, const State &) : full(true) {}
FixedHashMapCell(const value_type & value_, const State &) : full(true), mapped(value_.second) {}
Mapped & getSecond() { return mapped; }
const Mapped & getSecond() const { return mapped; }
bool isZero(const State &) const { return !full; }
void setZero() { full = false; }
static constexpr bool need_zero_value_storage = false;
void setMapped(const value_type & value) { mapped = value.getSecond(); }
/// Similar to FixedHashSetCell except that we need to contain a pointer to the Mapped field.
/// Note that we have to assemble a continuous layout for the value_type on each call of getValue().
struct CellExt
{
CellExt() {}
CellExt(Key && key_, const FixedHashMapCell * ptr_) : key(key_), ptr(const_cast<FixedHashMapCell *>(ptr_)) {}
void update(Key && key_, const FixedHashMapCell * ptr_)
{
key = key_;
ptr = const_cast<FixedHashMapCell *>(ptr_);
}
Key key;
FixedHashMapCell * ptr;
Key & getFirstMutable() { return key; }
const Key & getFirst() const { return key; }
Mapped & getSecond() { return ptr->mapped; }
const Mapped & getSecond() const { return ptr->mapped; }
const value_type getValue() const { return {key, ptr->mapped}; }
};
};
template <typename Key, typename Mapped, typename Allocator = HashTableAllocator>
class FixedHashMap : public FixedHashTable<Key, FixedHashMapCell<Key, Mapped>, Allocator>
{
public:
using Base = FixedHashTable<Key, FixedHashMapCell<Key, Mapped>, Allocator>;
using key_type = Key;
using mapped_type = Mapped;
using value_type = typename Base::cell_type::value_type;
using Base::Base;
mapped_type & ALWAYS_INLINE operator[](Key x)
{
typename Base::iterator it;
bool inserted;
this->emplace(x, it, inserted);
if (inserted)
new (&it->getSecond()) mapped_type();
return it->getSecond();
}
};

View File

@ -0,0 +1,25 @@
#pragma once
#include <Common/HashTable/FixedHashTable.h>
template <typename Key, typename Allocator = HashTableAllocator>
class FixedHashSet : public FixedHashTable<Key, FixedHashTableCell<Key>, Allocator>
{
public:
using Base = FixedHashTable<Key, FixedHashTableCell<Key>, Allocator>;
using Self = FixedHashSet;
void merge(const Self & rhs)
{
for (size_t i = 0; i < Base::BUFFER_SIZE; ++i)
if (Base::buf[i].isZero(*this) && !rhs.buf[i].isZero(*this))
Base::buf[i] = rhs.buf[i];
}
/// NOTE: Currently this method isn't used. When it does, the ReadBuffer should
/// contain the Key explicitly.
// void readAndMerge(DB::ReadBuffer & rb)
// {
// }
};

View File

@ -0,0 +1,420 @@
#pragma once
#include <Common/HashTable/HashTable.h>
template <typename Key, typename TState = HashTableNoState>
struct FixedHashTableCell
{
using State = TState;
using value_type = Key;
bool full;
FixedHashTableCell() {}
FixedHashTableCell(const Key &, const State &) : full(true) {}
bool isZero(const State &) const { return !full; }
void setZero() { full = false; }
static constexpr bool need_zero_value_storage = false;
void setMapped(const value_type & /*value*/) {}
/// This Cell is only stored inside an iterator. It's used to accomodate the fact
/// that the iterator based API always provide a reference to a continuous memory
/// containing the Key. As a result, we have to instantiate a real Key field.
/// All methods that return a mutable reference to the Key field are named with
/// -Mutable suffix, indicating this is uncommon usage. As this is only for lookup
/// tables, it's totally fine to discard the Key mutations.
struct CellExt
{
Key key;
value_type & getValueMutable() { return key; }
const value_type & getValue() const { return key; }
void update(Key && key_, FixedHashTableCell *) { key = key_; }
};
};
/** 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
* HashTable: a) It doesn't have a conflict chain; b) There is no key
* comparision; c) The number of cycles for checking cell empty is halved; d)
* Memory layout is tighter, especially the Clearable variants.
*
* NOTE: For Set variants this should always be better. For Map variants
* however, as we need to assemble the real cell inside each iterator, there
* might be some cases we fall short.
*
* TODO: Deprecate the cell API so that end users don't rely on the structure
* of cell. Instead iterator should be used for operations such as cell
* 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
{
static constexpr size_t BUFFER_SIZE = 1ULL << (sizeof(Key) * 8);
protected:
friend class const_iterator;
friend class iterator;
friend class Reader;
using Self = FixedHashTable;
using cell_type = Cell;
size_t m_size = 0; /// Amount of elements
Cell * buf; /// A piece of memory for all elements except the element with zero key.
void alloc() { buf = reinterpret_cast<Cell *>(Allocator::alloc(BUFFER_SIZE * sizeof(Cell))); }
void free()
{
if (buf)
{
Allocator::free(buf, getBufferSizeInBytes());
buf = nullptr;
}
}
void destroyElements()
{
if (!std::is_trivially_destructible_v<Cell>)
for (iterator it = begin(), it_end = end(); it != it_end; ++it)
it.ptr->~Cell();
}
template <typename Derived, bool is_const>
class iterator_base
{
using Container = std::conditional_t<is_const, const Self, Self>;
using cell_type = std::conditional_t<is_const, const Cell, Cell>;
Container * container;
cell_type * ptr;
friend class FixedHashTable;
public:
iterator_base() {}
iterator_base(Container * container_, cell_type * ptr_) : container(container_), ptr(ptr_)
{
cell.update(ptr - container->buf, ptr);
}
bool operator==(const iterator_base & rhs) const { return ptr == rhs.ptr; }
bool operator!=(const iterator_base & rhs) const { return ptr != rhs.ptr; }
Derived & operator++()
{
++ptr;
/// Skip empty cells in the main buffer.
auto buf_end = container->buf + container->BUFFER_SIZE;
while (ptr < buf_end && ptr->isZero(*container))
++ptr;
return static_cast<Derived &>(*this);
}
auto & operator*()
{
if (cell.key != ptr - container->buf)
cell.update(ptr - container->buf, ptr);
return cell;
}
auto * operator-> ()
{
if (cell.key != ptr - container->buf)
cell.update(ptr - container->buf, ptr);
return &cell;
}
auto getPtr() const { return ptr; }
size_t getHash() const { return ptr - container->buf; }
size_t getCollisionChainLength() const { return 0; }
typename cell_type::CellExt cell;
};
public:
using key_type = Key;
using value_type = typename Cell::value_type;
size_t hash(const Key & x) const { return x; }
FixedHashTable() { alloc(); }
FixedHashTable(FixedHashTable && rhs) : buf(nullptr) { *this = std::move(rhs); }
~FixedHashTable()
{
destroyElements();
free();
}
FixedHashTable & operator=(FixedHashTable && rhs)
{
destroyElements();
free();
std::swap(buf, rhs.buf);
std::swap(m_size, rhs.m_size);
Allocator::operator=(std::move(rhs));
Cell::State::operator=(std::move(rhs));
return *this;
}
class Reader final : private Cell::State
{
public:
Reader(DB::ReadBuffer & in_) : in(in_) {}
Reader(const Reader &) = delete;
Reader & operator=(const Reader &) = delete;
bool next()
{
if (!is_initialized)
{
Cell::State::read(in);
DB::readVarUInt(size, in);
is_initialized = true;
}
if (read_count == size)
{
is_eof = true;
return false;
}
cell.read(in);
++read_count;
return true;
}
inline const value_type & get() const
{
if (!is_initialized || is_eof)
throw DB::Exception("No available data", DB::ErrorCodes::NO_AVAILABLE_DATA);
return cell.getValue();
}
private:
DB::ReadBuffer & in;
Cell cell;
size_t read_count = 0;
size_t size;
bool is_eof = false;
bool is_initialized = false;
};
class iterator : public iterator_base<iterator, false>
{
public:
using iterator_base<iterator, false>::iterator_base;
};
class const_iterator : public iterator_base<const_iterator, true>
{
public:
using iterator_base<const_iterator, true>::iterator_base;
};
const_iterator begin() const
{
if (!buf)
return end();
const Cell * ptr = buf;
auto buf_end = buf + BUFFER_SIZE;
while (ptr < buf_end && ptr->isZero(*this))
++ptr;
return const_iterator(this, ptr);
}
const_iterator cbegin() const { return begin(); }
iterator begin()
{
if (!buf)
return end();
Cell * ptr = buf;
auto buf_end = buf + BUFFER_SIZE;
while (ptr < buf_end && ptr->isZero(*this))
++ptr;
return iterator(this, ptr);
}
const_iterator end() const { return const_iterator(this, buf + BUFFER_SIZE); }
const_iterator cend() const { return end(); }
iterator end() { return iterator(this, buf + BUFFER_SIZE); }
protected:
void ALWAYS_INLINE emplaceImpl(Key x, iterator & it, bool & inserted)
{
it = iterator(this, &buf[x]);
if (!buf[x].isZero(*this))
{
inserted = false;
return;
}
new (&buf[x]) Cell(x, *this);
inserted = true;
++m_size;
}
public:
std::pair<iterator, bool> ALWAYS_INLINE insert(const value_type & x)
{
std::pair<iterator, bool> res;
emplaceImpl(Cell::getKey(x), res.first, res.second);
if (res.second)
res.first.ptr->setMapped(x);
return res;
}
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); }
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith x)
{
return !buf[x].isZero(*this) ? iterator(this, &buf[x]) : end();
}
template <typename ObjectToCompareWith>
const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const
{
return !buf[x].isZero(*this) ? const_iterator(this, &buf[x]) : end();
}
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith, size_t hash_value)
{
return !buf[hash_value].isZero(*this) ? iterator(this, &buf[hash_value]) : end();
}
template <typename ObjectToCompareWith>
const_iterator ALWAYS_INLINE find(ObjectToCompareWith, size_t hash_value) const
{
return !buf[hash_value].isZero(*this) ? const_iterator(this, &buf[hash_value]) : end();
}
bool ALWAYS_INLINE has(Key x) const { return !buf[x].isZero(*this); }
bool ALWAYS_INLINE has(Key, size_t hash_value) const { return !buf[hash_value].isZero(*this); }
void write(DB::WriteBuffer & wb) const
{
Cell::State::write(wb);
DB::writeVarUInt(m_size, wb);
for (auto ptr = buf, buf_end = buf + BUFFER_SIZE; ptr < buf_end; ++ptr)
if (!ptr->isZero(*this))
{
DB::writeVarUInt(ptr - buf);
ptr->write(wb);
}
}
void writeText(DB::WriteBuffer & wb) const
{
Cell::State::writeText(wb);
DB::writeText(m_size, wb);
for (auto ptr = buf, buf_end = buf + BUFFER_SIZE; ptr < buf_end; ++ptr)
{
if (!ptr->isZero(*this))
{
DB::writeChar(',', wb);
DB::writeText(ptr - buf, wb);
DB::writeChar(',', wb);
ptr->writeText(wb);
}
}
}
void read(DB::ReadBuffer & rb)
{
Cell::State::read(rb);
destroyElements();
DB::readVarUInt(m_size, rb);
free();
alloc();
for (size_t i = 0; i < m_size; ++i)
{
size_t place_value = 0;
DB::readVarUInt(place_value, rb);
Cell x;
x.read(rb);
new (&buf[place_value]) Cell(x, *this);
}
}
void readText(DB::ReadBuffer & rb)
{
Cell::State::readText(rb);
destroyElements();
DB::readText(m_size, rb);
free();
alloc();
for (size_t i = 0; i < m_size; ++i)
{
size_t place_value = 0;
DB::assertChar(',', rb);
DB::readText(place_value, rb);
Cell x;
DB::assertChar(',', rb);
x.readText(rb);
new (&buf[place_value]) Cell(x, *this);
}
}
size_t size() const { return m_size; }
bool empty() const { return 0 == m_size; }
void clear()
{
destroyElements();
m_size = 0;
memset(static_cast<void *>(buf), 0, BUFFER_SIZE * sizeof(*buf));
}
/// After executing this function, the table can only be destroyed,
/// and also you can use the methods `size`, `empty`, `begin`, `end`.
void clearAndShrink()
{
destroyElements();
m_size = 0;
free();
}
size_t getBufferSizeInBytes() const { return BUFFER_SIZE * sizeof(Cell); }
size_t getBufferSizeInCells() const { return BUFFER_SIZE; }
#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
size_t getCollisions() const { return 0; }
#endif
};

View File

@ -15,11 +15,14 @@ struct NoInitTag {};
/// A pair that does not initialize the elements, if not needed.
template <typename First, typename Second>
struct PairNoInit
class PairNoInit
{
First first;
Second second;
template <typename, typename, typename, typename>
friend class HashMapCell;
public:
PairNoInit() {}
template <typename First_>
@ -29,6 +32,11 @@ struct PairNoInit
template <typename First_, typename Second_>
PairNoInit(First_ && first_, Second_ && second_)
: first(std::forward<First_>(first_)), second(std::forward<Second_>(second_)) {}
First & getFirstMutable() { return first; }
const First & getFirst() const { return first; }
Second & getSecond() { return second; }
const Second & getSecond() const { return second; }
};
@ -45,10 +53,14 @@ struct HashMapCell
HashMapCell(const Key & key_, const State &) : value(key_, NoInitTag()) {}
HashMapCell(const value_type & value_, const State &) : value(value_) {}
value_type & getValue() { return value; }
Key & getFirstMutable() { return value.first; }
const Key & getFirst() const { return value.first; }
Mapped & getSecond() { return value.second; }
const Mapped & getSecond() const { return value.second; }
value_type & getValueMutable() { return value; }
const value_type & getValue() const { return value; }
static Key & getKey(value_type & value) { return value.first; }
static const Key & getKey(const value_type & value) { return value.first; }
bool keyEquals(const Key & key_) const { return value.first == key_; }
@ -111,8 +123,8 @@ struct HashMapCellWithSavedHash : public HashMapCell<Key, TMapped, Hash, TState>
using Base::Base;
bool keyEquals(const Key & key_) const { return this->value.first == key_; }
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->value.first == key_; }
bool keyEquals(const Key & key_) const { return this->value.getFirst() == key_; }
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->value.getFirst() == 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; }
@ -158,9 +170,9 @@ public:
* the compiler can not guess about this, and generates the `load`, `increment`, `store` code.
*/
if (inserted)
new(&it->second) mapped_type();
new(&it->getSecond()) mapped_type();
return it->second;
return it->getSecond();
}
};

View File

@ -98,11 +98,10 @@ struct HashTableCell
/// HashTableCell(const value_type & value_, const State & state) : key(value_) {}
/// Get what the value_type of the container will be.
value_type & getValue() { return key; }
value_type & getValueMutable() { return key; }
const value_type & getValue() const { return key; }
/// Get the key.
static Key & getKey(value_type & value) { return value; }
static const Key & getKey(const value_type & value) { return value; }
/// Are the keys at the cells equal?
@ -459,8 +458,8 @@ protected:
return static_cast<Derived &>(*this);
}
auto & operator* () const { return ptr->getValue(); }
auto * operator->() const { return &ptr->getValue(); }
auto & operator* () const { return *ptr; }
auto * operator->() const { return ptr; }
auto getPtr() const { return ptr; }
size_t getHash() const { return ptr->getHash(*container); }

View File

@ -148,8 +148,8 @@ public:
return *this;
}
value_type & operator* () const { return ptr->getValue(); }
value_type * operator->() const { return &ptr->getValue(); }
Cell & operator* () const { return *ptr; }
Cell * operator->() const { return ptr; }
Cell * getPtr() const { return ptr; }
};
@ -176,8 +176,8 @@ public:
return *this;
}
const value_type & operator* () const { return ptr->getValue(); }
const value_type * operator->() const { return &ptr->getValue(); }
const Cell & operator* () const { return *ptr; }
const Cell * operator->() const { return ptr; }
const Cell * getPtr() const { return ptr; }
};
@ -399,8 +399,8 @@ public:
typename SmallMapTable::iterator it;
bool inserted;
this->emplace(x, it, inserted);
new(&it->second) mapped_type();
return it->second;
new(&it->getSecond()) mapped_type();
return it->getSecond();
}
};

View File

@ -29,9 +29,9 @@ public:
this->emplace(x, it, inserted);
if (inserted)
new(&it->second) mapped_type();
new(&it->getSecond()) mapped_type();
return it->second;
return it->getSecond();
}
};

View File

@ -98,7 +98,7 @@ public:
/// It is assumed that the zero key (stored separately) is first in iteration order.
if (it != src.end() && it.getPtr()->isZero(src))
{
insert(*it);
insert(it->getValue());
++it;
}
@ -141,8 +141,8 @@ public:
return *this;
}
value_type & operator* () const { return *current_it; }
value_type * operator->() const { return &*current_it; }
Cell & operator* () const { return *current_it; }
Cell * operator->() const { return current_it.getPtr(); }
Cell * getPtr() const { return current_it.getPtr(); }
size_t getHash() const { return current_it.getHash(); }
@ -179,8 +179,8 @@ public:
return *this;
}
const value_type & operator* () const { return *current_it; }
const value_type * operator->() const { return &*current_it; }
const Cell & operator* () const { return *current_it; }
const Cell * operator->() const { return current_it->getPtr(); }
const Cell * getPtr() const { return current_it.getPtr(); }
size_t getHash() const { return current_it.getHash(); }

View File

@ -45,7 +45,7 @@ private:
Large * tmp_large = new Large;
for (const auto & x : small)
tmp_large->insert(x);
tmp_large->insert(x.getValue());
large = tmp_large;
}
@ -99,7 +99,7 @@ public:
else
{
for (const auto & x : rhs.small)
insert(x);
insert(x.getValue());
}
}

View File

@ -152,7 +152,7 @@ public:
auto it = counter_map.find(key, hash);
if (it != counter_map.end())
{
auto c = it->second;
auto c = it->getSecond();
c->count += increment;
c->error += error;
percolate(c);
@ -189,8 +189,8 @@ public:
min->error = alpha + error;
percolate(min);
it->second = min;
it->first = min->key;
it->getSecond() = min;
it->getFirstMutable() = min->key;
counter_map.reinsert(it, hash);
}
}

View File

@ -155,10 +155,10 @@ int main(int argc, char ** argv)
map.emplace(rand(), it, inserted);
if (inserted)
{
new(&it->second) Arr(n);
new(&it->getSecond()) Arr(n);
for (size_t j = 0; j < n; ++j)
it->second[j] = field;
it->getSecond()[j] = field;
}
}

View File

@ -21,13 +21,13 @@ int main(int, char **)
bool inserted;
cont.emplace(3, it, inserted);
std::cerr << inserted << ", " << *it << std::endl;
std::cerr << inserted << ", " << it->getValue() << std::endl;
cont.emplace(3, it, inserted);
std::cerr << inserted << ", " << *it << std::endl;
std::cerr << inserted << ", " << it->getValue() << std::endl;
for (auto x : cont)
std::cerr << x << std::endl;
std::cerr << x.getValue() << std::endl;
DB::WriteBufferFromOwnString wb;
cont.writeText(wb);

View File

@ -82,14 +82,14 @@ void aggregate12(Map & map, Source::const_iterator begin, Source::const_iterator
{
if (*it == *prev_it)
{
++found->second;
++found->getSecond();
continue;
}
prev_it = it;
bool inserted;
map.emplace(*it, found, inserted);
++found->second;
++found->getSecond();
}
}
@ -107,14 +107,14 @@ void aggregate22(MapTwoLevel & map, Source::const_iterator begin, Source::const_
{
if (*it == *prev_it)
{
++found->second;
++found->getSecond();
continue;
}
prev_it = it;
bool inserted;
map.emplace(*it, found, inserted);
++found->second;
++found->getSecond();
}
}
@ -126,7 +126,7 @@ void merge2(MapTwoLevel * maps, size_t num_threads, size_t bucket)
{
for (size_t i = 1; i < num_threads; ++i)
for (auto it = maps[i].impls[bucket].begin(); it != maps[i].impls[bucket].end(); ++it)
maps[0].impls[bucket][it->first] += it->second;
maps[0].impls[bucket][it->getFirst()] += it->getSecond();
}
void aggregate3(Map & local_map, Map & global_map, Mutex & mutex, Source::const_iterator begin, Source::const_iterator end)
@ -138,7 +138,7 @@ void aggregate3(Map & local_map, Map & global_map, Mutex & mutex, Source::const_
Map::iterator found = local_map.find(*it);
if (found != local_map.end())
++found->second;
++found->getSecond();
else if (local_map.size() < threshold)
++local_map[*it]; /// TODO You could do one lookup, not two.
else
@ -163,13 +163,13 @@ void aggregate33(Map & local_map, Map & global_map, Mutex & mutex, Source::const
Map::iterator found;
bool inserted;
local_map.emplace(*it, found, inserted);
++found->second;
++found->getSecond();
if (inserted && local_map.size() == threshold)
{
std::lock_guard<Mutex> lock(mutex);
for (auto & value_type : local_map)
global_map[value_type.first] += value_type.second;
global_map[value_type.getFirst()] += value_type.getSecond();
local_map.clear();
}
@ -198,7 +198,7 @@ void aggregate4(Map & local_map, MapTwoLevel & global_map, Mutex * mutexes, Sour
Map::iterator found = local_map.find(*it);
if (found != local_map.end())
++found->second;
++found->getSecond();
else
{
size_t hash_value = global_map.hash(*it);
@ -311,7 +311,7 @@ int main(int argc, char ** argv)
for (size_t i = 1; i < num_threads; ++i)
for (auto it = maps[i].begin(); it != maps[i].end(); ++it)
maps[0][it->first] += it->second;
maps[0][it->getFirst()] += it->getSecond();
watch.stop();
double time_merged = watch.elapsedSeconds();
@ -365,7 +365,7 @@ int main(int argc, char ** argv)
for (size_t i = 1; i < num_threads; ++i)
for (auto it = maps[i].begin(); it != maps[i].end(); ++it)
maps[0][it->first] += it->second;
maps[0][it->getFirst()] += it->getSecond();
watch.stop();
@ -435,7 +435,7 @@ int main(int argc, char ** argv)
continue;
finish = false;
maps[0][iterators[i]->first] += iterators[i]->second;
maps[0][iterators[i]->getFirst()] += iterators[i]->getSecond();
++iterators[i];
}
@ -623,7 +623,7 @@ int main(int argc, char ** argv)
for (size_t i = 0; i < num_threads; ++i)
for (auto it = local_maps[i].begin(); it != local_maps[i].end(); ++it)
global_map[it->first] += it->second;
global_map[it->getFirst()] += it->getSecond();
pool.wait();
@ -689,7 +689,7 @@ int main(int argc, char ** argv)
for (size_t i = 0; i < num_threads; ++i)
for (auto it = local_maps[i].begin(); it != local_maps[i].end(); ++it)
global_map[it->first] += it->second;
global_map[it->getFirst()] += it->getSecond();
pool.wait();
@ -760,7 +760,7 @@ int main(int argc, char ** argv)
for (size_t i = 0; i < num_threads; ++i)
for (auto it = local_maps[i].begin(); it != local_maps[i].end(); ++it)
global_map[it->first] += it->second;
global_map[it->getFirst()] += it->getSecond();
pool.wait();

View File

@ -51,9 +51,9 @@ struct AggregateIndependent
map.emplace(*it, place, inserted);
if (inserted)
creator(place->second);
creator(place->getSecond());
else
updater(place->second);
updater(place->getSecond());
}
});
}
@ -93,7 +93,7 @@ struct AggregateIndependentWithSequentialKeysOptimization
{
if (it != begin && *it == prev_key)
{
updater(place->second);
updater(place->getSecond());
continue;
}
prev_key = *it;
@ -102,9 +102,9 @@ struct AggregateIndependentWithSequentialKeysOptimization
map.emplace(*it, place, inserted);
if (inserted)
creator(place->second);
creator(place->getSecond());
else
updater(place->second);
updater(place->getSecond());
}
});
}
@ -131,7 +131,7 @@ struct MergeSequential
auto begin = source_maps[i]->begin();
auto end = source_maps[i]->end();
for (auto it = begin; it != end; ++it)
merger((*source_maps[0])[it->first], it->second);
merger((*source_maps[0])[it->getFirst()], it->getSecond());
}
result_map = source_maps[0];
@ -161,7 +161,7 @@ struct MergeSequentialTransposed /// In practice not better than usual.
continue;
finish = false;
merger((*result_map)[iterators[i]->first], iterators[i]->second);
merger((*result_map)[iterators[i]->getFirst()], iterators[i]->getSecond());
++iterators[i];
}

View File

@ -20,13 +20,13 @@ int main(int, char **)
bool inserted;
cont.emplace(3, it, inserted);
std::cerr << inserted << ", " << *it << std::endl;
std::cerr << inserted << ", " << it->getValue() << std::endl;
cont.emplace(3, it, inserted);
std::cerr << inserted << ", " << *it << std::endl;
std::cerr << inserted << ", " << it->getValue() << std::endl;
for (auto x : cont)
std::cerr << x << std::endl;
std::cerr << x.getValue() << std::endl;
DB::WriteBufferFromOwnString wb;
cont.writeText(wb);
@ -42,7 +42,7 @@ int main(int, char **)
cont[1] = "Goodbye.";
for (auto x : cont)
std::cerr << x.first << " -> " << x.second << std::endl;
std::cerr << x.getFirst() << " -> " << x.getSecond() << std::endl;
DB::WriteBufferFromOwnString wb;
cont.writeText(wb);

View File

@ -222,7 +222,7 @@ int main(int argc, char ** argv)
size_t i = 0;
for (auto it = set.begin(); i < elems_show && it != set.end(); ++it, ++i)
{
devnull.write(it->first.data, it->first.size);
devnull.write(it->getFirst().data, it->getFirst().size);
devnull << std::endl;
}
@ -249,7 +249,7 @@ int main(int argc, char ** argv)
size_t i = 0;
for (auto it = set.begin(); i < elems_show && it != set.end(); ++it, ++i)
{
devnull.write(it->first.data, it->first.size);
devnull.write(it->getFirst().data, it->getFirst().size);
devnull << std::endl;
}
}

View File

@ -74,7 +74,7 @@ void DataTypeEnum<Type>::fillMaps()
if (!name_to_value_pair.second)
throw Exception{"Duplicate names in enum: '" + name_and_value.first + "' = " + toString(name_and_value.second)
+ " and '" + name_to_value_pair.first->first.toString() + "' = " + toString(name_to_value_pair.first->second),
+ " and '" + name_to_value_pair.first->getFirst().toString() + "' = " + toString(name_to_value_pair.first->getSecond()),
ErrorCodes::SYNTAX_ERROR};
const auto value_to_name_pair = value_to_name_map.insert(

View File

@ -81,7 +81,7 @@ public:
if (it == std::end(name_to_value_map))
throw Exception{"Unknown element '" + field_name.toString() + "' for type " + getName(), ErrorCodes::LOGICAL_ERROR};
return it->second;
return it->getSecond();
}
Field castToName(const Field & value_or_name) const override;

View File

@ -222,7 +222,7 @@ void ComplexKeyCacheDictionary::has(const Columns & key_columns, const DataTypes
std::vector<size_t> required_rows(outdated_keys.size());
std::transform(
std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.second.front(); });
std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.getSecond().front(); });
/// request new values
update(

View File

@ -342,7 +342,7 @@ private:
std::vector<size_t> required_rows(outdated_keys.size());
std::transform(
std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.second.front(); });
std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.getSecond().front(); });
/// request new values
update(
@ -468,7 +468,7 @@ private:
std::vector<size_t> required_rows(outdated_keys.size());
std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair)
{
return pair.second.front();
return pair.getSecond().front();
});
update(
@ -500,7 +500,7 @@ private:
{
const StringRef key = keys_array[row];
const auto it = map.find(key);
const auto string_ref = it != std::end(map) ? it->second : get_default(row);
const auto string_ref = it != std::end(map) ? it->getSecond() : get_default(row);
out->insertData(string_ref.data, string_ref.size);
}
}
@ -607,7 +607,7 @@ private:
/// Check which ids have not been found and require setting null_value
for (const auto & key_found_pair : remaining_keys)
{
if (key_found_pair.second)
if (key_found_pair.getSecond())
{
++found_num;
continue;
@ -615,7 +615,7 @@ private:
++not_found_num;
auto key = key_found_pair.first;
auto key = key_found_pair.getFirst();
const auto hash = StringRefHash{}(key);
const auto find_result = findCellIdx(key, now, hash);
const auto & cell_idx = find_result.cell_idx;

View File

@ -611,7 +611,7 @@ void ComplexKeyHashedDictionary::getItemsImpl(
const auto key = placeKeysInPool(i, key_columns, keys, temporary_keys_pool);
const auto it = attr.find(key);
set_value(i, it != attr.end() ? static_cast<OutputType>(it->second) : get_default(i));
set_value(i, it != attr.end() ? static_cast<OutputType>(it->getSecond()) : get_default(i));
/// free memory allocated for the key
temporary_keys_pool.rollback(key.size);
@ -779,7 +779,7 @@ std::vector<StringRef> ComplexKeyHashedDictionary::getKeys(const Attribute & att
std::vector<StringRef> keys;
keys.reserve(attr.size());
for (const auto & key : attr)
keys.push_back(key.first);
keys.push_back(key.getFirst());
return keys;
}

View File

@ -83,7 +83,7 @@ void HashedDictionary::isInImpl(const ChildType & child_ids, const AncestorType
{
auto it = attr.find(id);
if (it != std::end(attr))
id = it->second;
id = it->getSecond();
else
break;
}
@ -605,7 +605,7 @@ void HashedDictionary::getItemsImpl(
for (const auto i : ext::range(0, rows))
{
const auto it = attr.find(ids[i]);
set_value(i, it != attr.end() ? static_cast<OutputType>(it->second) : get_default(i));
set_value(i, it != attr.end() ? static_cast<OutputType>(it->getSecond()) : get_default(i));
}
query_count.fetch_add(rows, std::memory_order_relaxed);
@ -707,7 +707,7 @@ PaddedPODArray<HashedDictionary::Key> HashedDictionary::getIds(const Attribute &
PaddedPODArray<Key> ids;
ids.reserve(attr.size());
for (const auto & value : attr)
ids.push_back(value.first);
ids.push_back(value.getFirst());
return ids;
}

View File

@ -137,7 +137,7 @@ void RangeHashedDictionary::getString(
if (it != std::end(attr))
{
const auto date = dates[i];
const auto & ranges_and_values = it->second;
const auto & ranges_and_values = it->getSecond();
const auto val_it
= std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values), [date](const Value<StringRef> & v)
{
@ -408,7 +408,7 @@ void RangeHashedDictionary::getItemsImpl(
if (it != std::end(attr))
{
const auto date = dates[i];
const auto & ranges_and_values = it->second;
const auto & ranges_and_values = it->getSecond();
const auto val_it
= std::find_if(std::begin(ranges_and_values), std::end(ranges_and_values), [date](const Value<AttributeType> & v)
{
@ -435,7 +435,7 @@ void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, const K
if (it != map.end())
{
auto & values = it->second;
auto & values = it->getSecond();
const auto insert_it
= std::lower_bound(std::begin(values), std::end(values), range, [](const Value<T> & lhs, const Range & rhs_range)
@ -508,7 +508,7 @@ void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Key i
if (it != map.end())
{
auto & values = it->second;
auto & values = it->getSecond();
const auto insert_it = std::lower_bound(
std::begin(values), std::end(values), range, [](const Value<StringRef> & lhs, const Range & rhs_range)
@ -620,9 +620,9 @@ void RangeHashedDictionary::getIdsAndDates(
for (const auto & key : attr)
{
for (const auto & value : key.second)
for (const auto & value : key.getSecond())
{
ids.push_back(key.first);
ids.push_back(key.getFirst());
start_dates.push_back(value.range.left);
end_dates.push_back(value.range.right);

View File

@ -64,9 +64,9 @@ inline size_t JSONEachRowRowInputStream::columnIndex(const StringRef & name, siz
if (prev_positions.size() > key_index
&& prev_positions[key_index] != name_map.end()
&& name == prev_positions[key_index]->first)
&& name == prev_positions[key_index]->getFirst())
{
return prev_positions[key_index]->second;
return prev_positions[key_index]->getSecond();
}
else
{
@ -77,7 +77,7 @@ inline size_t JSONEachRowRowInputStream::columnIndex(const StringRef & name, siz
if (key_index < prev_positions.size())
prev_positions[key_index] = it;
return it->second;
return it->getSecond();
}
else
return UNKNOWN_FIELD;

View File

@ -128,7 +128,7 @@ bool TSKVRowInputStream::read(MutableColumns & columns, RowReadExtension & ext)
}
else
{
index = it->second;
index = it->getSecond();
if (read_columns[index])
throw Exception("Duplicate field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);

View File

@ -425,15 +425,15 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable
for (const auto & pair : map)
{
if (pair.second == args)
if (pair.getSecond() == args)
{
++result_offset;
if constexpr (is_numeric_column)
result_data.insertValue(pair.first);
result_data.insertValue(pair.getFirst());
else if constexpr (std::is_same<ColumnType, ColumnString>::value || std::is_same<ColumnType, ColumnFixedString>::value)
result_data.insertData(pair.first.data, pair.first.size);
result_data.insertData(pair.getFirst().data, pair.getFirst().size);
else
result_data.deserializeAndInsertFromArena(pair.first.data);
result_data.deserializeAndInsertFromArena(pair.getFirst().data);
if (all_nullable)
null_map.push_back(0);

View File

@ -508,7 +508,7 @@ private:
{
auto it = table.find(src[i]);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
memcpy(&dst[i], &it->getSecond(), sizeof(dst[i])); /// little endian.
else
dst[i] = dst_default;
}
@ -524,7 +524,7 @@ private:
{
auto it = table.find(src[i]);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i])); /// little endian.
memcpy(&dst[i], &it->getSecond(), sizeof(dst[i])); /// little endian.
else
dst[i] = dst_default[i];
}
@ -540,7 +540,7 @@ private:
{
auto it = table.find(src[i]);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
memcpy(&dst[i], &it->getSecond(), sizeof(dst[i]));
else
dst[i] = src[i];
}
@ -557,7 +557,7 @@ private:
for (size_t i = 0; i < size; ++i)
{
auto it = table.find(src[i]);
StringRef ref = it != table.end() ? it->second : dst_default;
StringRef ref = it != table.end() ? it->getSecond() : dst_default;
dst_data.resize(current_dst_offset + ref.size);
memcpy(&dst_data[current_dst_offset], ref.data, ref.size);
current_dst_offset += ref.size;
@ -581,7 +581,7 @@ private:
StringRef ref;
if (it != table.end())
ref = it->second;
ref = it->getSecond();
else
{
ref.data = reinterpret_cast<const char *>(&dst_default_data[current_dst_default_offset]);
@ -611,7 +611,7 @@ private:
current_src_offset = src_offsets[i];
auto it = table.find(ref);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
memcpy(&dst[i], &it->getSecond(), sizeof(dst[i]));
else
dst[i] = dst_default;
}
@ -632,7 +632,7 @@ private:
current_src_offset = src_offsets[i];
auto it = table.find(ref);
if (it != table.end())
memcpy(&dst[i], &it->second, sizeof(dst[i]));
memcpy(&dst[i], &it->getSecond(), sizeof(dst[i]));
else
dst[i] = dst_default[i];
}
@ -655,7 +655,7 @@ private:
auto it = table.find(src_ref);
StringRef dst_ref = it != table.end() ? it->second : (with_default ? dst_default : src_ref);
StringRef dst_ref = it != table.end() ? it->getSecond() : (with_default ? dst_default : src_ref);
dst_data.resize(current_dst_offset + dst_ref.size);
memcpy(&dst_data[current_dst_offset], dst_ref.data, dst_ref.size);
current_dst_offset += dst_ref.size;
@ -697,7 +697,7 @@ private:
StringRef dst_ref;
if (it != table.end())
dst_ref = it->second;
dst_ref = it->getSecond();
else
{
dst_ref.data = reinterpret_cast<const char *>(&dst_default_data[current_dst_default_offset]);

View File

@ -1120,11 +1120,11 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
for (const auto & value : data)
{
method.insertKeyIntoColumns(value, key_columns, key_sizes);
method.insertKeyIntoColumns(value.getValue(), key_columns, key_sizes);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->insertResultInto(
value.second + offsets_of_aggregate_states[i],
value.getSecond() + offsets_of_aggregate_states[i],
*final_aggregate_columns[i]);
}
@ -1151,13 +1151,13 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal(
for (auto & value : data)
{
method.insertKeyIntoColumns(value, key_columns, key_sizes);
method.insertKeyIntoColumns(value.getValue(), key_columns, key_sizes);
/// reserved, so push_back does not throw exceptions
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_columns[i]->push_back(value.second + offsets_of_aggregate_states[i]);
aggregate_columns[i]->push_back(value.getSecond() + offsets_of_aggregate_states[i]);
value.second = nullptr;
value.getSecond() = nullptr;
}
}
@ -1495,26 +1495,26 @@ void NO_INLINE Aggregator::mergeDataImpl(
{
typename Table::iterator res_it;
bool inserted;
table_dst.emplace(it->first, res_it, inserted, it.getHash());
table_dst.emplace(it->getFirst(), res_it, inserted, it.getHash());
if (!inserted)
{
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(
res_it->second + offsets_of_aggregate_states[i],
it->second + offsets_of_aggregate_states[i],
res_it->getSecond() + offsets_of_aggregate_states[i],
it->getSecond() + offsets_of_aggregate_states[i],
arena);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(
it->second + offsets_of_aggregate_states[i]);
it->getSecond() + offsets_of_aggregate_states[i]);
}
else
{
res_it->second = it->second;
res_it->getSecond() = it->getSecond();
}
it->second = nullptr;
it->getSecond() = nullptr;
}
table_src.clearAndShrink();
@ -1534,22 +1534,22 @@ void NO_INLINE Aggregator::mergeDataNoMoreKeysImpl(
for (auto it = table_src.begin(), end = table_src.end(); it != end; ++it)
{
typename Table::iterator res_it = table_dst.find(it->first, it.getHash());
typename Table::iterator res_it = table_dst.find(it->getFirst(), it.getHash());
AggregateDataPtr res_data = table_dst.end() == res_it
? overflows
: res_it->second;
: res_it->getSecond();
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(
res_data + offsets_of_aggregate_states[i],
it->second + offsets_of_aggregate_states[i],
it->getSecond() + offsets_of_aggregate_states[i],
arena);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(it->second + offsets_of_aggregate_states[i]);
aggregate_functions[i]->destroy(it->getSecond() + offsets_of_aggregate_states[i]);
it->second = nullptr;
it->getSecond() = nullptr;
}
table_src.clearAndShrink();
@ -1567,23 +1567,23 @@ void NO_INLINE Aggregator::mergeDataOnlyExistingKeysImpl(
for (auto it = table_src.begin(); it != table_src.end(); ++it)
{
decltype(it) res_it = table_dst.find(it->first, it.getHash());
decltype(it) res_it = table_dst.find(it->getFirst(), it.getHash());
if (table_dst.end() == res_it)
continue;
AggregateDataPtr res_data = res_it->second;
AggregateDataPtr res_data = res_it->getSecond();
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->merge(
res_data + offsets_of_aggregate_states[i],
it->second + offsets_of_aggregate_states[i],
it->getSecond() + offsets_of_aggregate_states[i],
arena);
for (size_t i = 0; i < params.aggregates_size; ++i)
aggregate_functions[i]->destroy(it->second + offsets_of_aggregate_states[i]);
aggregate_functions[i]->destroy(it->getSecond() + offsets_of_aggregate_states[i]);
it->second = nullptr;
it->getSecond() = nullptr;
}
table_src.clearAndShrink();
@ -2428,7 +2428,7 @@ void NO_INLINE Aggregator::destroyImpl(Table & table) const
{
for (auto elem : table)
{
AggregateDataPtr & data = elem.second;
AggregateDataPtr & data = elem.getSecond();
/** If an exception (usually a lack of memory, the MemoryTracker throws) arose
* after inserting the key into a hash table, but before creating all states of aggregate functions,

View File

@ -10,6 +10,7 @@
#include <common/StringRef.h>
#include <Common/Arena.h>
#include <Common/HashTable/FixedHashMap.h>
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/TwoLevelHashMap.h>
#include <Common/ThreadPool.h>
@ -64,8 +65,8 @@ class IBlockOutputStream;
using AggregatedDataWithoutKey = AggregateDataPtr;
using AggregatedDataWithUInt8Key = HashMap<UInt64, AggregateDataPtr, TrivialHash, HashTableFixedGrower<8>>;
using AggregatedDataWithUInt16Key = HashMap<UInt64, AggregateDataPtr, TrivialHash, HashTableFixedGrower<16>>;
using AggregatedDataWithUInt8Key = FixedHashMap<UInt8, AggregateDataPtr>;
using AggregatedDataWithUInt16Key = FixedHashMap<UInt16, AggregateDataPtr>;
using AggregatedDataWithUInt64Key = HashMap<UInt64, AggregateDataPtr, HashCRC32<UInt64>>;
using AggregatedDataWithStringKey = HashMapWithSavedHash<StringRef, AggregateDataPtr>;
@ -178,7 +179,7 @@ struct AggregationMethodOneNumber
// Insert the key from the hash table into columns.
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes & /*key_sizes*/)
{
static_cast<ColumnVectorHelper *>(key_columns[0].get())->insertRawData<sizeof(FieldType)>(reinterpret_cast<const char *>(&value.first));
static_cast<ColumnVectorHelper *>(key_columns[0].get())->insertRawData<sizeof(FieldType)>(reinterpret_cast<const char *>(&value.getFirst()));
}
};
@ -206,7 +207,7 @@ struct AggregationMethodString
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &)
{
key_columns[0]->insertData(value.first.data, value.first.size);
key_columns[0]->insertData(value.getFirst().data, value.getFirst().size);
}
};
@ -234,7 +235,7 @@ struct AggregationMethodFixedString
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &)
{
key_columns[0]->insertData(value.first.data, value.first.size);
key_columns[0]->insertData(value.getFirst().data, value.getFirst().size);
}
};
@ -326,7 +327,7 @@ struct AggregationMethodKeysFixed
/// corresponding key is nullable. Update the null map accordingly.
size_t bucket = i / 8;
size_t offset = i % 8;
UInt8 val = (reinterpret_cast<const UInt8 *>(&value.first)[bucket] >> offset) & 1;
UInt8 val = (reinterpret_cast<const UInt8 *>(&value.getFirst())[bucket] >> offset) & 1;
null_map->insertValue(val);
is_null = val == 1;
}
@ -338,7 +339,7 @@ struct AggregationMethodKeysFixed
else
{
size_t size = key_sizes[i];
observed_column->insertData(reinterpret_cast<const char *>(&value.first) + pos, size);
observed_column->insertData(reinterpret_cast<const char *>(&value.getFirst()) + pos, size);
pos += size;
}
}
@ -373,7 +374,7 @@ struct AggregationMethodSerialized
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, const Sizes &)
{
auto pos = value.first.data;
auto pos = value.getFirst().data;
for (auto & column : key_columns)
pos = column->deserializeAndInsertFromArena(pos);
}

View File

@ -1317,10 +1317,10 @@ private:
for (; it != end; ++it)
{
if (it->second.getUsed())
if (it->getSecond().getUsed())
continue;
AdderNonJoined<STRICTNESS, typename Map::mapped_type>::add(it->second, rows_added, columns_left, columns_keys_and_right);
AdderNonJoined<STRICTNESS, typename Map::mapped_type>::add(it->getSecond(), rows_added, columns_left, columns_keys_and_right);
if (rows_added >= max_block_size)
{

View File

@ -10,6 +10,7 @@
#include <Common/Arena.h>
#include <Common/ColumnsHashing.h>
#include <Common/HashTable/HashMap.h>
#include <Common/HashTable/FixedHashMap.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
@ -218,8 +219,8 @@ public:
template <typename Mapped>
struct MapsTemplate
{
std::unique_ptr<HashMap<UInt8, Mapped, TrivialHash, HashTableFixedGrower<8>>> key8;
std::unique_ptr<HashMap<UInt16, Mapped, TrivialHash, HashTableFixedGrower<16>>> key16;
std::unique_ptr<FixedHashMap<UInt8, Mapped>> key8;
std::unique_ptr<FixedHashMap<UInt16, Mapped>> key16;
std::unique_ptr<HashMap<UInt32, Mapped, HashCRC32<UInt32>>> key32;
std::unique_ptr<HashMap<UInt64, Mapped, HashCRC32<UInt64>>> key64;
std::unique_ptr<HashMapWithSavedHash<StringRef, Mapped>> key_string;

View File

@ -8,6 +8,8 @@
#include <Common/Arena.h>
#include <Common/HashTable/HashSet.h>
#include <Common/HashTable/ClearableHashSet.h>
#include <Common/HashTable/FixedClearableHashSet.h>
#include <Common/HashTable/FixedHashSet.h>
#include <Common/UInt128.h>
@ -182,9 +184,8 @@ struct SetMethodHashed
*/
struct NonClearableSet
{
/// TODO Use either bit- or byte-set for these two options.
std::unique_ptr<SetMethodOneNumber<UInt8, HashSet<UInt8, TrivialHash, HashTableFixedGrower<8>>>> key8;
std::unique_ptr<SetMethodOneNumber<UInt16, HashSet<UInt16, TrivialHash, HashTableFixedGrower<16>>>> key16;
std::unique_ptr<SetMethodOneNumber<UInt8, FixedHashSet<UInt8>>> key8;
std::unique_ptr<SetMethodOneNumber<UInt16, FixedHashSet<UInt16>>> key16;
/** Also for the experiment was tested the ability to use SmallSet,
* as long as the number of elements in the set is small (and, if necessary, converted to a full-fledged HashSet).
@ -209,9 +210,8 @@ struct NonClearableSet
struct ClearableSet
{
/// TODO Use either bit- or byte-set for these two options.
std::unique_ptr<SetMethodOneNumber<UInt8, ClearableHashSet<UInt8, TrivialHash, HashTableFixedGrower<8>>>> key8;
std::unique_ptr<SetMethodOneNumber<UInt16, ClearableHashSet<UInt16, TrivialHash, HashTableFixedGrower<16>>>> key16;
std::unique_ptr<SetMethodOneNumber<UInt8, FixedClearableHashSet<UInt8>>> key8;
std::unique_ptr<SetMethodOneNumber<UInt16, FixedClearableHashSet<UInt16>>> key16;
std::unique_ptr<SetMethodOneNumber<UInt32, ClearableHashSet<UInt32, HashCRC32<UInt32>>>> key32;
std::unique_ptr<SetMethodOneNumber<UInt64, ClearableHashSet<UInt64, HashCRC32<UInt64>>>> key64;

View File

@ -14,6 +14,10 @@ add_executable (hash_map hash_map.cpp)
target_include_directories (hash_map SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
target_link_libraries (hash_map PRIVATE dbms clickhouse_compression)
add_executable (hash_map_lookup hash_map_lookup.cpp)
target_include_directories (hash_map_lookup SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
target_link_libraries (hash_map_lookup PRIVATE dbms clickhouse_compression)
add_executable (hash_map3 hash_map3.cpp)
target_include_directories(hash_map3 SYSTEM BEFORE PRIVATE ${METROHASH_INCLUDE_DIR})
target_link_libraries (hash_map3 PRIVATE dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES})

View File

@ -162,8 +162,8 @@ int main(int argc, char ** argv)
map.emplace(data[i], it, inserted);
if (inserted)
{
new(&it->second) Value;
std::swap(it->second, value);
new(&it->getSecond()) Value;
std::swap(it->getSecond(), value);
INIT
}
}
@ -193,8 +193,8 @@ int main(int argc, char ** argv)
map.emplace(data[i], it, inserted);
if (inserted)
{
new(&it->second) Value;
std::swap(it->second, value);
new(&it->getSecond()) Value;
std::swap(it->getSecond(), value);
INIT
}
}
@ -225,8 +225,8 @@ int main(int argc, char ** argv)
map.emplace(data[i], it, inserted);
if (inserted)
{
new(&it->second) Value;
std::swap(it->second, value);
new(&it->getSecond()) Value;
std::swap(it->getSecond(), value);
INIT
}
}

View File

@ -38,7 +38,7 @@ public:
if (this->buf[i].isZero(*this))
std::cerr << "[ ]";
else
std::cerr << '[' << this->buf[i].getValue().first.data << ", " << this->buf[i].getValue().second << ']';
std::cerr << '[' << this->buf[i].getValue().getFirst().data << ", " << this->buf[i].getValue().getSecond() << ']';
}
std::cerr << std::endl;
}
@ -85,7 +85,7 @@ int main(int, char **)
std::cerr << "Collisions: " << map.getCollisions() << std::endl;
for (auto x : map)
std::cerr << x.first.toString() << " -> " << x.second << std::endl;
std::cerr << x.getFirst().toString() << " -> " << x.getSecond() << std::endl;
return 0;
}

View File

@ -0,0 +1,124 @@
#include <iomanip>
#include <iostream>
#include <vector>
#include <Common/Stopwatch.h>
#define DBMS_HASH_MAP_COUNT_COLLISIONS
#define DBMS_HASH_MAP_DEBUG_RESIZES
#include <Compression/CompressedReadBuffer.h>
#include <Core/Types.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/AggregationCommon.h>
#include <Common/HashTable/FixedHashMap.h>
#include <Common/HashTable/HashMap.h>
/** Do this:
for file in ResolutionWidth ResolutionDepth; do
for size in 30000 100000 300000 1000000 5000000; do
echo
BEST_METHOD=0
BEST_RESULT=0
for method in {1..2}; do
echo -ne $file $size $method '';
TOTAL_ELEMS=0
for i in {0..1000}; do
TOTAL_ELEMS=$(( $TOTAL_ELEMS + $size ))
if [[ $TOTAL_ELEMS -gt 25000000 ]]; then break; fi
./hash_map_lookup $size $method < ${file}.bin 2>&1 |
grep HashMap | grep -oE '[0-9\.]+ elem';
done | awk -W interactive '{ if ($1 > x) { x = $1 }; printf(".") } END { print x }' | tee /tmp/hash_map_lookup_res;
CUR_RESULT=$(cat /tmp/hash_map_lookup_res | tr -d '.')
if [[ $CUR_RESULT -gt $BEST_RESULT ]]; then
BEST_METHOD=$method
BEST_RESULT=$CUR_RESULT
fi;
done;
echo Best: $BEST_METHOD - $BEST_RESULT
done;
done
*/
template <typename Map>
void NO_INLINE bench(const std::vector<UInt16> & data, const char * name)
{
Map map;
typename Map::iterator it;
bool inserted;
Stopwatch watch;
for (size_t i = 0, size = data.size(); i < size; ++i)
{
map.emplace(data[i], it, inserted);
if (inserted)
it->getSecond() = 1;
else
++it->getSecond();
}
for (size_t i = 0, size = data.size(); i < size; ++i)
{
it = map.find(data[i]);
++it->getSecond();
}
watch.stop();
std::cerr << std::fixed << std::setprecision(2) << "HashMap (" << name << "). Size: " << map.size()
<< ", elapsed: " << watch.elapsedSeconds() << " (" << data.size() / watch.elapsedSeconds() << " elem/sec.)"
#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
<< ", collisions: " << map.getCollisions()
#endif
<< std::endl;
}
template <typename Map>
void insert(Map & map, StringRef & k)
{
bool inserted;
typename Map::iterator it;
map.emplace(k, it, inserted, nullptr);
if (inserted)
*it = 1;
else
++*it;
std::cout << *map.find(k) << std::endl;
}
int main(int argc, char ** argv)
{
if (argc < 3)
{
std::cerr << "Usage: program n m\n";
return 1;
}
size_t n = atoi(argv[1]);
size_t m = atoi(argv[2]);
std::vector<UInt16> data(n);
{
Stopwatch watch;
DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
DB::CompressedReadBuffer in2(in1);
for (size_t i = 0; i < n && !in2.eof(); ++i)
{
DB::readBinary(data[i], in2);
}
watch.stop();
std::cerr << std::fixed << std::setprecision(2) << "Vector. Size: " << n << ", elapsed: " << watch.elapsedSeconds() << " ("
<< n / watch.elapsedSeconds() << " elem/sec.)" << std::endl;
}
using OldLookup = HashMap<UInt16, UInt8, TrivialHash, HashTableFixedGrower<16>>;
using NewLookup = FixedHashMap<UInt16, UInt8>;
if (!m || m == 1)
bench<OldLookup>(data, "Old Lookup");
if (!m || m == 2)
bench<NewLookup>(data, "New Lookup");
return 0;
}

View File

@ -337,8 +337,8 @@ int main(int argc, char ** argv)
{
map.emplace(data[i], it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();
@ -366,8 +366,8 @@ int main(int argc, char ** argv)
{
map.emplace(data[i], it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();
@ -396,8 +396,8 @@ int main(int argc, char ** argv)
{
map.emplace(data[i], it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();
@ -426,8 +426,8 @@ int main(int argc, char ** argv)
{
map.emplace(data[i], it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();

View File

@ -595,8 +595,8 @@ void NO_INLINE bench(const std::vector<StringRef> & data, const char * name)
{
map.emplace(static_cast<const Key &>(data[i]), it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();

View File

@ -442,8 +442,8 @@ void NO_INLINE bench(const std::vector<StringRef> & data, const char * name)
{
map.emplace(static_cast<const Key &>(data[i]), it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();

View File

@ -144,8 +144,8 @@ int main(int argc, char ** argv)
{
map.emplace(data[i], it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();
@ -173,8 +173,8 @@ int main(int argc, char ** argv)
{
map.emplace(SmallStringRef(data[i].data, data[i].size), it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();

View File

@ -67,8 +67,8 @@ int main(int argc, char ** argv)
{
map.emplace(data[i], it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();
@ -82,7 +82,7 @@ int main(int argc, char ** argv)
size_t elems = 0;
for (const auto & kv : map)
{
sum_counts += kv.second;
sum_counts += kv.getSecond();
++elems;
}
@ -103,8 +103,8 @@ int main(int argc, char ** argv)
{
map.emplace(i, it, inserted);
if (inserted)
it->second = 0;
++it->second;
it->getSecond() = 0;
++it->getSecond();
}
watch.stop();
@ -118,11 +118,11 @@ int main(int argc, char ** argv)
size_t elems = 0;
for (const auto & kv : map)
{
sum_counts += kv.second;
sum_counts += kv.getSecond();
++elems;
if (kv.first > n)
std::cerr << kv.first << std::endl;
if (kv.getFirst() > n)
std::cerr << kv.getFirst() << std::endl;
}
std::cerr << "sum_counts: " << sum_counts << ", elems: " << elems << std::endl;

View File

@ -48,7 +48,7 @@ void buildScatterSelector(
if (inserted)
{
partition_num_to_first_row.push_back(i);
it->second = partitions_count;
it->getSecond() = partitions_count;
++partitions_count;
@ -61,7 +61,7 @@ void buildScatterSelector(
}
if (partitions_count > 1)
selector[i] = it->second;
selector[i] = it->getSecond();
}
}

View File

@ -327,18 +327,18 @@ private:
{
for (size_t j = 0; j < columns.size(); ++j)
if (j == key_pos)
columns[j]->insertData(rawData(it->first), rawSize(it->first));
columns[j]->insertData(rawData(it->getFirst()), rawSize(it->getFirst()));
else
columns[j]->insertFrom(*it->second.block->getByPosition(column_indices[j]).column.get(), it->second.row_num);
columns[j]->insertFrom(*it->getSecond().block->getByPosition(column_indices[j]).column.get(), it->getSecond().row_num);
++rows_added;
}
else
for (auto current = &static_cast<const typename Map::mapped_type::Base_t &>(it->second); current != nullptr;
for (auto current = &static_cast<const typename Map::mapped_type::Base_t &>(it->getSecond()); current != nullptr;
current = current->next)
{
for (size_t j = 0; j < columns.size(); ++j)
if (j == key_pos)
columns[j]->insertData(rawData(it->first), rawSize(it->first));
columns[j]->insertData(rawData(it->getFirst()), rawSize(it->getFirst()));
else
columns[j]->insertFrom(*current->block->getByPosition(column_indices[j]).column.get(), current->row_num);
++rows_added;

View File

@ -105,7 +105,7 @@ public:
if (table.end() == it)
return pos - data;
*pos = it->second.sample(random());
*pos = it->getSecond().sample(random());
/// Zero byte marks end of string.
if (0 == *pos)
@ -125,12 +125,12 @@ public:
for (auto & elem : table)
{
UInt32 new_total = 0;
for (auto & frequency : elem.second.data)
for (auto & frequency : elem.getSecond().data)
{
frequency.count = transform(frequency.count);
new_total += frequency.count;
}
elem.second.total = new_total;
elem.getSecond().total = new_total;
}
}
@ -142,10 +142,10 @@ public:
for (const auto & elem : table)
{
writeBinary(elem.first, out);
writeBinary(UInt8(elem.second.data.size()), out);
writeBinary(elem.getFirst(), out);
writeBinary(UInt8(elem.getSecond().data.size()), out);
for (const auto & frequency : elem.second.data)
for (const auto & frequency : elem.getSecond().data)
{
writeBinary(frequency.byte, out);
writeVarUInt(frequency.count, out);