提交 01209533 编写于 作者: N Nikolai Kochetov

Optimize reverse index for ColumnUnique.

上级 3f4348a8
#pragma once
#include <Columns/IColumnUnique.h>
#include <Common/HashTable/HashMap.h>
#include <ext/range.h>
#include <Common/typeid_cast.h>
#include <Columns/ReverseIndex.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/NumberTraits.h>
class NullMap;
template <typename ColumnType>
struct StringRefWrapper
{
const ColumnType * column = nullptr;
size_t row = 0;
StringRef ref;
StringRefWrapper(const ColumnType * column, size_t row) : column(column), row(row) {}
StringRefWrapper(StringRef ref) : ref(ref) {}
StringRefWrapper(const StringRefWrapper & other) = default;
StringRefWrapper & operator =(int) { column = nullptr; ref.data = nullptr; return *this; }
bool operator ==(int) const { return nullptr == column && nullptr == ref.data; }
StringRefWrapper() {}
operator StringRef() const { return column ? column->getDataAt(row) : ref; }
bool operator==(const StringRefWrapper<ColumnType> & other) const
{
return (column && column == other.column && row == other.row) || StringRef(*this) == other;
}
};
namespace ZeroTraits
{
template <typename ColumnType>
bool check(const StringRefWrapper<ColumnType> x) { return nullptr == x.column && nullptr == x.ref.data; }
template <typename ColumnType>
void set(StringRefWrapper<ColumnType> & x) { x.column = nullptr; x.ref.data = nullptr; }
};
#include <Common/typeid_cast.h>
#include <ext/range.h>
namespace DB
{
namespace
{
template <typename ColumnType>
struct ColumnUniqueContainer
{
using Type = HashMap<StringRefWrapper<ColumnType>, UInt64, StringRefHash>;
};
template <>
struct ColumnUniqueContainer<ColumnString>
{
using Type = HashMapWithSavedHash<StringRefWrapper<ColumnString>, UInt64, StringRefHash>;
};
template <>
struct ColumnUniqueContainer<ColumnFixedString>
{
using Type = HashMapWithSavedHash<StringRefWrapper<ColumnFixedString>, UInt64, StringRefHash>;
};
}
template <typename ColumnType>
class ColumnUnique final : public COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>>
......@@ -79,7 +25,7 @@ class ColumnUnique final : public COWPtrHelper<IColumnUnique, ColumnUnique<Colum
private:
explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable);
explicit ColumnUnique(const IDataType & type);
ColumnUnique(const ColumnUnique & other) : column_holder(other.column_holder), is_nullable(other.is_nullable) {}
ColumnUnique(const ColumnUnique & other);
public:
MutableColumnPtr cloneEmpty() const override;
......@@ -136,36 +82,30 @@ public:
size_t allocatedBytes() const override
{
return column_holder->allocatedBytes()
+ (index ? index->getBufferSizeInBytes() : 0)
+ index.allocatedBytes()
+ (cached_null_mask ? cached_null_mask->allocatedBytes() : 0);
}
void forEachSubcolumn(IColumn::ColumnCallback callback) override
{
callback(column_holder);
index = nullptr;
index.setColumn(getRawColumnPtr());
}
private:
using IndexMapType = typename ColumnUniqueContainer<ColumnType>::Type;
ColumnPtr column_holder;
bool is_nullable;
ReverseIndex<UInt64, ColumnType> index;
/// For DataTypeNullable, stores null map.
mutable ColumnPtr cached_null_mask;
mutable ColumnPtr cached_column_nullable;
/// Lazy initialized.
std::unique_ptr<IndexMapType> index;
static size_t numSpecialValues(bool is_nullable) { return is_nullable ? 2 : 1; }
size_t numSpecialValues() const { return numSpecialValues(is_nullable); }
bool is_nullable;
size_t numSpecialValues() const { return is_nullable ? 2 : 1; }
void buildIndex();
ColumnType * getRawColumnPtr() { return static_cast<ColumnType *>(column_holder->assumeMutable().get()); }
const ColumnType * getRawColumnPtr() const { return static_cast<const ColumnType *>(column_holder.get()); }
UInt64 insertIntoMap(const StringRefWrapper<ColumnType> & ref, UInt64 value);
template <typename IndexType>
MutableColumnPtr uniqueInsertRangeImpl(
......@@ -174,8 +114,7 @@ private:
size_t length,
size_t num_added_rows,
typename ColumnVector<IndexType>::MutablePtr && positions_column,
ColumnType * overflowed_keys,
IndexMapType * secondary_index,
ReverseIndex<UInt64, ColumnType> * secondary_index,
size_t max_dictionary_size);
};
......@@ -186,20 +125,36 @@ MutableColumnPtr ColumnUnique<ColumnType>::cloneEmpty() const
}
template <typename ColumnType>
ColumnUnique<ColumnType>::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable())
ColumnUnique<ColumnType>::ColumnUnique(const ColumnUnique & other)
: column_holder(other.column_holder)
, is_nullable(other.is_nullable)
, index(numSpecialValues(is_nullable), 0)
{
index.setColumn(getRawColumnPtr());
}
template <typename ColumnType>
ColumnUnique<ColumnType>::ColumnUnique(const IDataType & type)
: is_nullable(type.isNullable())
, index(numSpecialValues(is_nullable), 0)
{
const auto & holder_type = is_nullable ? *static_cast<const DataTypeNullable &>(type).getNestedType() : type;
column_holder = holder_type.createColumn()->cloneResized(numSpecialValues());
index.setColumn(getRawColumnPtr());
}
template <typename ColumnType>
ColumnUnique<ColumnType>::ColumnUnique(MutableColumnPtr && holder, bool is_nullable)
: column_holder(std::move(holder)), is_nullable(is_nullable)
: column_holder(std::move(holder))
, is_nullable(is_nullable)
, index(numSpecialValues(is_nullable), 0)
{
if (column_holder->size() < numSpecialValues())
throw Exception("Too small holder column for ColumnUnique.", ErrorCodes::ILLEGAL_COLUMN);
if (column_holder->isColumnNullable())
throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN);
index.setColumn(getRawColumnPtr());
}
template <typename ColumnType>
......@@ -238,38 +193,6 @@ size_t ColumnUnique<ColumnType>::getNullValueIndex() const
return 0;
}
template <typename ColumnType>
void ColumnUnique<ColumnType>::buildIndex()
{
if (index)
return;
auto column = getRawColumnPtr();
index = std::make_unique<IndexMapType>(column_holder->size());
for (auto row : ext::range(numSpecialValues(), column->size()))
{
(*index)[StringRefWrapper<ColumnType>(column, row)] = row;
}
}
template <typename ColumnType>
UInt64 ColumnUnique<ColumnType>::insertIntoMap(const StringRefWrapper<ColumnType> & ref, UInt64 value)
{
if (!index)
buildIndex();
using IteratorType = typename IndexMapType::iterator;
IteratorType it;
bool inserted;
index->emplace(ref, it, inserted);
if (inserted)
it->second = value;
return it->second;
}
template <typename ColumnType>
size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
{
......@@ -283,7 +206,7 @@ size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
return getDefaultValueIndex();
column->insert(x);
auto pos = insertIntoMap(StringRefWrapper<ColumnType>(column, prev_size), prev_size);
auto pos = index.insert(prev_size);
if (pos != prev_size)
column->popBack(1);
......@@ -306,24 +229,21 @@ size_t ColumnUnique<ColumnType>::uniqueInsertFrom(const IColumn & src, size_t n)
template <typename ColumnType>
size_t ColumnUnique<ColumnType>::uniqueInsertData(const char * pos, size_t length)
{
if (!index)
buildIndex();
auto column = getRawColumnPtr();
if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length))
return getDefaultValueIndex();
UInt64 size = column->size();
auto iter = index->find(StringRefWrapper<ColumnType>(StringRef(pos, length)));
UInt64 insertion_point = index.getInsertionPoint(StringRef(pos, length));
if (iter == index->end())
if (insertion_point == size)
{
column->insertData(pos, length);
return insertIntoMap(StringRefWrapper<ColumnType>(column, size), size);
index.insertFromLastRow();
}
return iter->second;
return insertion_point;
}
template <typename ColumnType>
......@@ -347,7 +267,7 @@ size_t ColumnUnique<ColumnType>::uniqueInsertDataWithTerminatingZero(const char
return getDefaultValueIndex();
}
auto position = insertIntoMap(StringRefWrapper<ColumnType>(column, prev_size), prev_size);
auto position = index.insert(prev_size);
if (position != prev_size)
column->popBack(1);
......@@ -367,7 +287,7 @@ size_t ColumnUnique<ColumnType>::uniqueDeserializeAndInsertFromArena(const char
return getDefaultValueIndex();
}
auto index_pos = insertIntoMap(StringRefWrapper<ColumnType>(column, prev_size), prev_size);
auto index_pos = index.insert(prev_size);
if (index_pos != prev_size)
column->popBack(1);
......@@ -397,18 +317,14 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
size_t length,
size_t num_added_rows,
typename ColumnVector<IndexType>::MutablePtr && positions_column,
ColumnType * overflowed_keys,
IndexMapType * secondary_index,
ReverseIndex<UInt64, ColumnType> * secondary_index,
size_t max_dictionary_size)
{
if (!index)
buildIndex();
const ColumnType * src_column;
const NullMap * null_map = nullptr;
auto & positions = positions_column->getData();
auto updatePosition = [&](UInt64 & next_position) -> MutableColumnPtr
auto update_position = [&](UInt64 & next_position) -> MutableColumnPtr
{
constexpr auto next_size = NumberTraits::nextSize(sizeof(IndexType));
using SuperiorIndexType = typename NumberTraits::Construct<false, false, next_size>::Type;
......@@ -432,7 +348,6 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
length,
num_added_rows,
std::move(expanded_column),
overflowed_keys,
secondary_index,
max_dictionary_size);
}
......@@ -458,6 +373,22 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
if (secondary_index)
next_position += secondary_index->size();
auto check_inserted_position = [&next_position](UInt64 inserted_position)
{
if (inserted_position != next_position)
throw Exception("Inserted position " + toString(inserted_position)
+ " is not equal with expected " + toString(next_position), ErrorCodes::LOGICAL_ERROR);
};
auto insert_key = [&](const StringRef & ref, ReverseIndex<UInt64, ColumnType> * cur_index)
{
positions[num_added_rows] = next_position;
cur_index->getColumn()->insertData(ref.data, ref.size);
auto inserted_pos = cur_index->insertFromLastRow();
check_inserted_position(inserted_pos);
return update_position(next_position);
};
for (; num_added_rows < length; ++num_added_rows)
{
auto row = start + num_added_rows;
......@@ -468,52 +399,40 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
positions[num_added_rows] = getDefaultValueIndex();
else
{
auto it = index->find(StringRefWrapper<ColumnType>(src_column, row));
if (it == index->end())
auto ref = src_column->getDataAt(row);
auto cur_index = &index;
bool inserted = false;
while (!inserted)
{
auto insertion_point = cur_index->getInsertionPoint(ref);
if (overflowed_keys && next_position >= max_dictionary_size)
if (insertion_point == cur_index->lastInsertionPoint())
{
auto jt = secondary_index->find(StringRefWrapper<ColumnType>(src_column, row));
if (jt == secondary_index->end())
if (secondary_index && cur_index != secondary_index && next_position >= max_dictionary_size)
{
positions[num_added_rows] = next_position;
auto ref = src_column->getDataAt(row);
overflowed_keys->insertData(ref.data, ref.size);
(*secondary_index)[StringRefWrapper<ColumnType>(src_column, row)] = next_position;
if (auto res = updatePosition(next_position))
return res;
cur_index = secondary_index;
continue;
}
else
positions[num_added_rows] = jt->second;
}
else
{
positions[num_added_rows] = next_position;
auto ref = src_column->getDataAt(row);
column->insertData(ref.data, ref.size);
(*index)[StringRefWrapper<ColumnType>(column, next_position)] = next_position;
if (auto res = updatePosition(next_position))
if (auto res = insert_key(ref, cur_index))
return res;
}
else
positions[num_added_rows] = insertion_point;
inserted = true;
}
else
positions[num_added_rows] = it->second;
}
}
/// TODO: remove
// checkIndexes(*positions_column, column->size() + (overflowed_keys ? overflowed_keys->size() : 0));
return std::move(positions_column);
}
template <typename ColumnType>
MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length)
{
auto callForType = [this, &src, start, length](auto x) -> MutableColumnPtr
{
size_t size = getRawColumnPtr()->size();
......@@ -522,8 +441,7 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeFrom(const IColumn &
if (size <= std::numeric_limits<IndexType>::max())
{
auto positions = ColumnVector<IndexType>::create(length);
return this->uniqueInsertRangeImpl<IndexType>(src, start, length, 0,
std::move(positions), nullptr, nullptr, 0);
return this->uniqueInsertRangeImpl<IndexType>(src, start, length, 0, std::move(positions), nullptr, 0);
}
return nullptr;
......@@ -551,7 +469,6 @@ IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWi
size_t length,
size_t max_dictionary_size)
{
auto overflowed_keys = column_holder->cloneEmpty();
auto overflowed_keys_ptr = typeid_cast<ColumnType *>(overflowed_keys.get());
if (!overflowed_keys_ptr)
......@@ -565,9 +482,10 @@ IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWi
if (size <= std::numeric_limits<IndexType>::max())
{
auto positions = ColumnVector<IndexType>::create(length);
IndexMapType secondary_index;
ReverseIndex<UInt64, ColumnType> secondary_index(0, max_dictionary_size);
secondary_index.setColumn(overflowed_keys_ptr);
return this->uniqueInsertRangeImpl<IndexType>(src, start, length, 0, std::move(positions),
overflowed_keys_ptr, &secondary_index, max_dictionary_size);
&secondary_index, max_dictionary_size);
}
return nullptr;
......
#pragma once
#include <Common/HashTable/Hash.h>
#include <Common/HashTable/HashTable.h>
#include <Common/HashTable/HashTableAllocator.h>
#include <Columns/ColumnString.h>
#include <ext/range.h>
namespace DB
{
namespace
{
template <typename ColumnType, bool with_saved_hash, bool has_base_index>
struct ReverseIndexHashTableState;
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ false>
{
constexpr static bool with_saved_hash = false;
constexpr static bool has_base_index = false;
ColumnType * index_column;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash */ false, /* has_base_index */ true>
{
constexpr static bool with_saved_hash = false;
constexpr static bool has_base_index = true;
ColumnType * index_column;
size_t base_index;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ false>
{
constexpr static bool with_saved_hash = true;
constexpr static bool has_base_index = false;
ColumnType * index_column;
typename ColumnVector<UInt64>::Container * saved_hash_column;
};
template <typename ColumnType>
struct ReverseIndexHashTableState<ColumnType, /* with_saved_hash = */ true, /* has_base_index */ true>
{
constexpr static bool with_saved_hash = true;
constexpr static bool has_base_index = true;
ColumnType * index_column;
typename ColumnVector<UInt64>::Container * saved_hash_column;
size_t base_index;
};
template <typename Hash>
struct ReverseIndexHash : public Hash
{
template <typename T>
size_t operator()(T) const
{
throw Exception("operator()(key) is not implemented for ReverseIndexHash.", ErrorCodes::LOGICAL_ERROR);
}
template <typename State, typename T>
size_t operator()(const State & state, T key) const
{
auto index = key;
if constexpr (State::has_base_index)
index -= state.base_index;
return Hash::operator()(state.index_column->getElement(index));
}
};
using ReverseIndexStringHash = ReverseIndexHash<StringRefHash>;
template <typename IndexType>
using ReverseIndexNumberHash = ReverseIndexHash<DefaultHash<IndexType>>;
template <typename IndexType, typename Hash, typename HashTable, typename ColumnType, bool string_hash, bool has_base_index>
struct ReverseIndexHashTableCell
: public HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>
{
using Base = HashTableCell<IndexType, Hash, ReverseIndexHashTableState<ColumnType, string_hash, has_base_index>>;
using State = typename Base::State;
using Base::Base;
using Base::key;
using Base::keyEquals;
using Base::isZero;
template <typename T>
static bool isZero(const T &, const State & /*state*/)
{
static_assert(!std::is_same_v<typename std::decay<T>::type, typename std::decay<IndexType>::type>);
return false;
}
/// Special case when we want to compare with something not in index_column.
/// When we compare something inside column default keyEquals checks only that row numbers are equal.
bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const
{
auto index = key;
if constexpr (has_base_index)
index -= state.base_index;
if constexpr (string_hash)
return hash_ == (*state.saved_hash_column)[index] && object == state.index_column->getDataAt(index);
else
return object == state.index_column->getDataAt(index);
}
size_t getHash(const Hash & hash) const
{
auto index = key;
/// Hack. HashTable is Hash itself.
const auto & state = static_cast<const State &>(static_cast<const HashTable &>(hash));
if constexpr (has_base_index)
index -= state.base_index;
if constexpr (string_hash)
return (*state.saved_hash_column)[index];
else
return hash(state, key);
}
};
template <typename Key, typename Cell, typename Hash>
class HashTableWithPublicState : public HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>
{
using State = typename Cell::State;
using Base = HashTable<Key, Cell, Hash, HashTableGrower<>, HashTableAllocator>;
public:
using Base::Base;
State & getState() { return *this; }
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexStringHashTable : public HashTableWithPublicState<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexStringHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>,
ReverseIndexStringHash>
{
using Base = HashTableWithPublicState<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexStringHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>,
ReverseIndexStringHash>;
public:
using Base::Base;
friend class ReverseIndexHashTableCell<
IndexType,
ReverseIndexStringHash,
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
true,
has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index>
class ReverseIndexNumberHashTable : public HashTableWithPublicState<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexNumberHash<typename ColumnType::value_type>,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>,
ReverseIndexNumberHash<typename ColumnType::value_type>>
{
using Base = HashTableWithPublicState<
IndexType,
ReverseIndexHashTableCell<
IndexType,
ReverseIndexNumberHash<typename ColumnType::value_type>,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>,
ReverseIndexNumberHash<typename ColumnType::value_type>>;
public:
using Base::Base;
friend class ReverseIndexHashTableCell<
IndexType,
ReverseIndexNumberHash<typename ColumnType::value_type>,
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
ColumnType,
false,
has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index, bool is_numeric_column>
struct SelectReverseIndexHashTable;
template <typename IndexType, typename ColumnType, bool has_base_index>
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, true>
{
using Type = ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>;
};
template <typename IndexType, typename ColumnType, bool has_base_index>
struct SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index, false>
{
using Type = ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>;
};
template <typename T>
constexpr bool isNumericColumn(const T *) { return false; }
template <typename T>
constexpr bool isNumericColumn(const ColumnVector<T> *) { return true; }
static_assert(isNumericColumn(static_cast<ColumnVector<UInt8> *>(nullptr)));
static_assert(!isNumericColumn(static_cast<ColumnString *>(nullptr)));
template <typename IndexType, typename ColumnType, bool has_base_index>
using ReverseIndexHashTable = typename SelectReverseIndexHashTable<IndexType, ColumnType, has_base_index,
isNumericColumn(static_cast<ColumnType *>(nullptr))>::Type;
}
template <typename IndexType, typename ColumnType>
class ReverseIndex
{
public:
explicit ReverseIndex(UInt64 num_prefix_rows_to_skip, UInt64 base_index)
: num_prefix_rows_to_skip(num_prefix_rows_to_skip), base_index(base_index) {}
void setColumn(ColumnType * column_);
static constexpr bool is_numeric_column = isNumericColumn(static_cast<ColumnType *>(nullptr));
static constexpr bool use_saved_hash = !is_numeric_column;
UInt64 insert(UInt64 from_position); /// Insert into index column[from_position];
UInt64 insertFromLastRow();
UInt64 getInsertionPoint(const StringRef & data);
UInt64 lastInsertionPoint() const { return size() + base_index; }
ColumnType * getColumn() const { return column; }
size_t size() const;
size_t allocatedBytes() const { return index ? index->getBufferSizeInBytes() : 0; }
private:
ColumnType * column = nullptr;
UInt64 num_prefix_rows_to_skip; /// The number prefix tows in column which won't be sored at index.
UInt64 base_index; /// This values will be added to row number which is inserted into index.
using IndexMapType = ReverseIndexHashTable<IndexType, ColumnType, true>;
/// Lazy initialized.
std::unique_ptr<IndexMapType> index;
ColumnUInt64::MutablePtr saved_hash;
void buildIndex();
UInt64 getHash(const StringRef & ref) const
{
if constexpr (is_numeric_column)
{
using ValueType = typename ColumnType::value_type;
ValueType value = *reinterpret_cast<const ValueType *>(ref.data);
return DefaultHash<ValueType>()(value);
}
else
return StringRefHash()(ref);
}
};
template <typename IndexType, typename ColumnType>
void ReverseIndex<IndexType, ColumnType>:: setColumn(ColumnType * column_)
{
if (column != column_)
index = nullptr;
column = column_;
}
template <typename IndexType, typename ColumnType>
size_t ReverseIndex<IndexType, ColumnType>::size() const
{
if (!column)
throw Exception("ReverseIndex has not size because index column wasn't set.", ErrorCodes::LOGICAL_ERROR);
return column->size();
}
template <typename IndexType, typename ColumnType>
void ReverseIndex<IndexType, ColumnType>::buildIndex()
{
if (index)
return;
if (!column)
throw Exception("ReverseIndex can't build index because index column wasn't set.", ErrorCodes::LOGICAL_ERROR);
auto size = column->size();
index = std::make_unique<IndexMapType>(size);
if constexpr (use_saved_hash)
saved_hash = ColumnUInt64::create(size);
auto & state = index->getState();
state.index_column = column;
state.base_index = base_index;
if constexpr (use_saved_hash)
state.saved_hash_column = &saved_hash->getData();
using IteratorType = typename IndexMapType::iterator;
IteratorType iterator;
bool inserted;
for (auto row : ext::range(num_prefix_rows_to_skip, size))
{
auto hash = getHash(column->getDataAt(row));
if constexpr (use_saved_hash)
saved_hash->getElement(row) = hash;
index->emplace(row + base_index, iterator, inserted, hash);
if (!inserted)
throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR);
}
}
template <typename IndexType, typename ColumnType>
UInt64 ReverseIndex<IndexType, ColumnType>::insert(UInt64 from_position)
{
if (!index)
buildIndex();
using IteratorType = typename IndexMapType::iterator;
IteratorType iterator;
bool inserted;
auto hash = getHash(column->getDataAt(from_position));
if constexpr (use_saved_hash)
{
auto & data = saved_hash->getData();
if (data.size() <= from_position)
data.resize(from_position + 1);
data[from_position] = hash;
}
index->emplace(from_position + base_index, iterator, inserted, hash);
return *iterator;
}
template <typename IndexType, typename ColumnType>
UInt64 ReverseIndex<IndexType, ColumnType>::insertFromLastRow()
{
if (!column)
throw Exception("ReverseIndex can't insert row from column because index column wasn't set.",
ErrorCodes::LOGICAL_ERROR);
UInt64 num_rows = size();
if (num_rows == 0)
throw Exception("ReverseIndex can't insert row from column because it is empty.", ErrorCodes::LOGICAL_ERROR);
UInt64 position = num_rows - 1;
UInt64 inserted_pos = insert(position);
if (position + base_index != inserted_pos)
throw Exception("Can't insert into reverse index from last row (" + toString(position + base_index)
+ ") because the same row is in position " + toString(inserted_pos), ErrorCodes::LOGICAL_ERROR);
return inserted_pos;
}
template <typename IndexType, typename ColumnType>
UInt64 ReverseIndex<IndexType, ColumnType>::getInsertionPoint(const StringRef & data)
{
if (!index)
buildIndex();
using IteratorType = typename IndexMapType::iterator;
IteratorType iterator;
auto hash = getHash(data);
iterator = index->find(data, hash);
return iterator == index->end() ? size() + base_index : *iterator;
}
}
......@@ -53,6 +53,7 @@ struct HashMapCell
bool keyEquals(const Key & key_) const { return value.first == key_; }
bool keyEquals(const Key & key_, size_t /*hash_*/) const { return value.first == key_; }
bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return value.first == key_; }
void setHash(size_t /*hash_value*/) {}
size_t getHash(const Hash & hash) const { return hash(value.first); }
......@@ -112,6 +113,7 @@ struct HashMapCellWithSavedHash : public HashMapCell<Key, TMapped, Hash, TState>
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_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); }
void setHash(size_t hash_value) { saved_hash = hash_value; }
size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
......
......@@ -75,6 +75,7 @@ struct HashSetCellWithSavedHash : public HashTableCell<Key, Hash, TState>
bool keyEquals(const Key & key_) const { return this->key == key_; }
bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && 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; }
size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; }
......
......@@ -108,6 +108,7 @@ struct HashTableCell
/// Are the keys at the cells equal?
bool keyEquals(const Key & key_) const { return key == key_; }
bool keyEquals(const Key & key_, size_t /*hash_*/) const { return key == key_; }
bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return key == key_; }
/// If the cell can remember the value of the hash function, then remember it.
void setHash(size_t /*hash_value*/) {}
......@@ -280,9 +281,10 @@ protected:
#endif
/// Find a cell with the same key or an empty cell, starting from the specified position and further along the collision resolution chain.
size_t ALWAYS_INLINE findCell(const Key & x, size_t hash_value, size_t place_value) const
template <typename ObjectToCompareWith>
size_t ALWAYS_INLINE findCell(const ObjectToCompareWith & x, size_t hash_value, size_t place_value) const
{
while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value))
while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value, *this))
{
place_value = grower.next(place_value);
#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS
......@@ -734,7 +736,8 @@ public:
}
iterator ALWAYS_INLINE find(Key x)
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith x)
{
if (Cell::isZero(x, *this))
return this->hasZero() ? iteratorToZero() : end();
......@@ -745,7 +748,8 @@ public:
}
const_iterator ALWAYS_INLINE find(Key x) const
template <typename ObjectToCompareWith>
const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const
{
if (Cell::isZero(x, *this))
return this->hasZero() ? iteratorToZero() : end();
......@@ -756,7 +760,8 @@ public:
}
iterator ALWAYS_INLINE find(Key x, size_t hash_value)
template <typename ObjectToCompareWith>
iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value)
{
if (Cell::isZero(x, *this))
return this->hasZero() ? iteratorToZero() : end();
......@@ -766,7 +771,8 @@ public:
}
const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const
template <typename ObjectToCompareWith>
const_iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) const
{
if (Cell::isZero(x, *this))
return this->hasZero() ? iteratorToZero() : end();
......
......@@ -2,6 +2,7 @@
#include <Columns/ColumnUnique.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnsCommon.h>
#include <Common/HashTable/HashMap.h>
#include <Common/typeid_cast.h>
#include <Core/TypeListNumber.h>
#include <DataTypes/DataTypeFactory.h>
......
......@@ -268,7 +268,7 @@ struct Settings
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \
\
M(SettingUInt64, max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for WithDictionary type.") \
M(SettingBool, use_single_dictionary_for_part, true, "WithDictionary type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \
M(SettingBool, use_single_dictionary_for_part, false, "WithDictionary type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册