mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Update ReverseIndex. Added tests.
This commit is contained in:
parent
7af621f6f7
commit
ec59f65a1e
@ -0,0 +1,3 @@
|
||||
if (ENABLE_TESTS)
|
||||
add_subdirectory (tests)
|
||||
endif ()
|
@ -212,13 +212,6 @@ void ColumnLowCardinality::insertData(const char * pos, size_t length)
|
||||
idx.check(getDictionary().size());
|
||||
}
|
||||
|
||||
void ColumnLowCardinality::insertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
{
|
||||
compactIfSharedDictionary();
|
||||
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertDataWithTerminatingZero(pos, length));
|
||||
idx.check(getDictionary().size());
|
||||
}
|
||||
|
||||
StringRef ColumnLowCardinality::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
||||
{
|
||||
return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin);
|
||||
|
@ -73,8 +73,6 @@ public:
|
||||
void insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions);
|
||||
|
||||
void insertData(const char * pos, size_t length) override;
|
||||
void insertDataWithTerminatingZero(const char * pos, size_t length) override;
|
||||
|
||||
|
||||
void popBack(size_t n) override { idx.popBack(n); }
|
||||
|
||||
|
@ -158,7 +158,8 @@ public:
|
||||
offsets.push_back(new_size);
|
||||
}
|
||||
|
||||
void insertDataWithTerminatingZero(const char * pos, size_t length) override
|
||||
/// Like getData, but inserting data should be zero-ending (i.e. length is 1 byte greater than real string size).
|
||||
void insertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
{
|
||||
const size_t old_size = chars.size();
|
||||
const size_t new_size = old_size + length;
|
||||
|
@ -44,7 +44,6 @@ public:
|
||||
IColumnUnique::IndexesWithOverflow uniqueInsertRangeWithOverflow(const IColumn & src, size_t start, size_t length,
|
||||
size_t max_dictionary_size) override;
|
||||
size_t uniqueInsertData(const char * pos, size_t length) override;
|
||||
size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) override;
|
||||
size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) override;
|
||||
|
||||
size_t getDefaultValueIndex() const override { return 0; }
|
||||
@ -100,6 +99,7 @@ private:
|
||||
|
||||
ColumnPtr column_holder;
|
||||
bool is_nullable;
|
||||
size_t size_of_value_if_fixed = 0;
|
||||
ReverseIndex<UInt64, ColumnType> index;
|
||||
|
||||
/// For DataTypeNullable, stores null map.
|
||||
@ -151,6 +151,7 @@ template <typename ColumnType>
|
||||
ColumnUnique<ColumnType>::ColumnUnique(const ColumnUnique & other)
|
||||
: column_holder(other.column_holder)
|
||||
, is_nullable(other.is_nullable)
|
||||
, size_of_value_if_fixed (other.size_of_value_if_fixed)
|
||||
, index(numSpecialValues(is_nullable), 0)
|
||||
{
|
||||
index.setColumn(getRawColumnPtr());
|
||||
@ -166,6 +167,9 @@ ColumnUnique<ColumnType>::ColumnUnique(const IDataType & type)
|
||||
column_holder = holder_type.createColumn()->cloneResized(numSpecialValues());
|
||||
index.setColumn(getRawColumnPtr());
|
||||
createNullMask();
|
||||
|
||||
if (column_holder->valuesHaveFixedSize())
|
||||
size_of_value_if_fixed = column_holder->sizeOfValueIfFixed();
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
@ -181,6 +185,9 @@ ColumnUnique<ColumnType>::ColumnUnique(MutableColumnPtr && holder, bool is_nulla
|
||||
|
||||
index.setColumn(getRawColumnPtr());
|
||||
createNullMask();
|
||||
|
||||
if (column_holder->valuesHaveFixedSize())
|
||||
size_of_value_if_fixed = column_holder->sizeOfValueIfFixed();
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
@ -243,20 +250,11 @@ size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
||||
if (x.getType() == Field::Types::Null)
|
||||
return getNullValueIndex();
|
||||
|
||||
auto column = getRawColumnPtr();
|
||||
auto prev_size = static_cast<UInt64>(column->size());
|
||||
if (size_of_value_if_fixed)
|
||||
return uniqueInsertData(&x.get<char>(), size_of_value_if_fixed);
|
||||
|
||||
if ((*column)[getNestedTypeDefaultValueIndex()] == x)
|
||||
return getNestedTypeDefaultValueIndex();
|
||||
|
||||
column->insert(x);
|
||||
auto pos = index.insert(prev_size);
|
||||
if (pos != prev_size)
|
||||
column->popBack(1);
|
||||
|
||||
updateNullMask();
|
||||
|
||||
return pos;
|
||||
auto & val = x.get<String>();
|
||||
return uniqueInsertData(val.data(), val.size());
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
@ -280,50 +278,13 @@ size_t ColumnUnique<ColumnType>::uniqueInsertData(const char * pos, size_t lengt
|
||||
if (column->getDataAt(getNestedTypeDefaultValueIndex()) == StringRef(pos, length))
|
||||
return getNestedTypeDefaultValueIndex();
|
||||
|
||||
UInt64 size = column->size();
|
||||
UInt64 insertion_point = index.getInsertionPoint(StringRef(pos, length));
|
||||
|
||||
if (insertion_point == size)
|
||||
{
|
||||
column->insertData(pos, length);
|
||||
index.insertFromLastRow();
|
||||
}
|
||||
auto insertion_point = index.insert(StringRef(pos, length));
|
||||
|
||||
updateNullMask();
|
||||
|
||||
return insertion_point;
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
{
|
||||
if (std::is_same<ColumnType, ColumnString>::value)
|
||||
return uniqueInsertData(pos, length - 1);
|
||||
|
||||
if (column_holder->valuesHaveFixedSize())
|
||||
return uniqueInsertData(pos, length);
|
||||
|
||||
/// Don't know if data actually has terminating zero. So, insert it firstly.
|
||||
|
||||
auto column = getRawColumnPtr();
|
||||
size_t prev_size = column->size();
|
||||
column->insertDataWithTerminatingZero(pos, length);
|
||||
|
||||
if (column->compareAt(getNestedTypeDefaultValueIndex(), prev_size, *column, 1) == 0)
|
||||
{
|
||||
column->popBack(1);
|
||||
return getNestedTypeDefaultValueIndex();
|
||||
}
|
||||
|
||||
auto position = index.insert(prev_size);
|
||||
if (position != prev_size)
|
||||
column->popBack(1);
|
||||
|
||||
updateNullMask();
|
||||
|
||||
return static_cast<size_t>(position);
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
StringRef ColumnUnique<ColumnType>::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
|
||||
{
|
||||
@ -362,23 +323,17 @@ size_t ColumnUnique<ColumnType>::uniqueDeserializeAndInsertFromArena(const char
|
||||
}
|
||||
}
|
||||
|
||||
auto column = getRawColumnPtr();
|
||||
size_t prev_size = column->size();
|
||||
new_pos = column->deserializeAndInsertFromArena(pos);
|
||||
|
||||
if (column->compareAt(getNestedTypeDefaultValueIndex(), prev_size, *column, 1) == 0)
|
||||
if (size_of_value_if_fixed)
|
||||
{
|
||||
column->popBack(1);
|
||||
return getNestedTypeDefaultValueIndex();
|
||||
new_pos = pos + size_of_value_if_fixed;
|
||||
return uniqueInsertData(pos, size_of_value_if_fixed);
|
||||
}
|
||||
|
||||
auto index_pos = index.insert(prev_size);
|
||||
if (index_pos != prev_size)
|
||||
column->popBack(1);
|
||||
const size_t string_size = *reinterpret_cast<const size_t *>(pos);
|
||||
pos += sizeof(string_size);
|
||||
new_pos = pos + string_size;
|
||||
|
||||
updateNullMask();
|
||||
|
||||
return static_cast<size_t>(index_pos);
|
||||
return uniqueInsertData(pos, string_size);
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
@ -482,20 +437,14 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||
if (secondary_index)
|
||||
next_position += secondary_index->size();
|
||||
|
||||
auto check_inserted_position = [&next_position](UInt64 inserted_position)
|
||||
auto insert_key = [&](const StringRef & ref, ReverseIndex<UInt64, ColumnType> & cur_index) -> MutableColumnPtr
|
||||
{
|
||||
if (inserted_position != next_position)
|
||||
throw Exception("Inserted position " + toString(inserted_position)
|
||||
+ " is not equal with expected " + toString(next_position), ErrorCodes::LOGICAL_ERROR);
|
||||
};
|
||||
auto inserted_pos = cur_index.insert(ref);
|
||||
positions[num_added_rows] = inserted_pos;
|
||||
if (inserted_pos == next_position)
|
||||
return update_position(next_position);
|
||||
|
||||
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);
|
||||
return nullptr;
|
||||
};
|
||||
|
||||
for (; num_added_rows < length; ++num_added_rows)
|
||||
@ -509,29 +458,21 @@ MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||
else
|
||||
{
|
||||
auto ref = src_column->getDataAt(row);
|
||||
auto cur_index = &index;
|
||||
bool inserted = false;
|
||||
MutableColumnPtr res = nullptr;
|
||||
|
||||
while (!inserted)
|
||||
if (secondary_index && next_position >= max_dictionary_size)
|
||||
{
|
||||
auto insertion_point = cur_index->getInsertionPoint(ref);
|
||||
|
||||
if (insertion_point == cur_index->lastInsertionPoint())
|
||||
{
|
||||
if (secondary_index && cur_index != secondary_index && next_position >= max_dictionary_size)
|
||||
{
|
||||
cur_index = secondary_index;
|
||||
continue;
|
||||
}
|
||||
|
||||
if (auto res = insert_key(ref, cur_index))
|
||||
return res;
|
||||
}
|
||||
auto insertion_point = index.getInsertionPoint(ref);
|
||||
if (insertion_point == index.lastInsertionPoint())
|
||||
res = insert_key(ref, *secondary_index);
|
||||
else
|
||||
positions[num_added_rows] = insertion_point;
|
||||
|
||||
inserted = true;
|
||||
positions[num_added_rows] = insertion_point;
|
||||
}
|
||||
else
|
||||
res = insert_key(ref, index);
|
||||
|
||||
if (res)
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -143,13 +143,6 @@ public:
|
||||
/// Parameter length could be ignored if column values have fixed size.
|
||||
virtual void insertData(const char * pos, size_t length) = 0;
|
||||
|
||||
/// Like getData, but has special behavior for columns that contain variable-length strings.
|
||||
/// In this special case inserting data should be zero-ending (i.e. length is 1 byte greater than real string size).
|
||||
virtual void insertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
{
|
||||
insertData(pos, length);
|
||||
}
|
||||
|
||||
/// Appends "default value".
|
||||
/// Is used when there are need to increase column size, but inserting value doesn't make sense.
|
||||
/// For example, ColumnNullable(Nested) absolutely ignores values of nested column if it is marked as NULL.
|
||||
|
@ -51,7 +51,6 @@ public:
|
||||
/// Is used to optimize some computations (in aggregation, for example).
|
||||
/// Parameter length could be ignored if column values have fixed size.
|
||||
virtual size_t uniqueInsertData(const char * pos, size_t length) = 0;
|
||||
virtual size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) = 0;
|
||||
|
||||
virtual size_t getDefaultValueIndex() const = 0; /// Nullable ? getNullValueIndex : getNestedTypeDefaultValueIndex
|
||||
virtual size_t getNullValueIndex() const = 0; /// Throws if not nullable.
|
||||
|
@ -56,32 +56,15 @@ namespace
|
||||
};
|
||||
|
||||
|
||||
template <typename Hash>
|
||||
struct ReverseIndexHash : public Hash
|
||||
struct ReverseIndexHash
|
||||
{
|
||||
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>>
|
||||
@ -99,6 +82,7 @@ namespace
|
||||
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
|
||||
@ -126,7 +110,11 @@ namespace
|
||||
if constexpr (string_hash)
|
||||
return (*state.saved_hash_column)[index];
|
||||
else
|
||||
return hash(state, key);
|
||||
{
|
||||
using ValueType = typename ColumnType::value_type;
|
||||
ValueType value = *reinterpret_cast<const ValueType *>(state.index_column->getDataAt(key).data);
|
||||
return DefaultHash<ValueType>()(value);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
@ -147,28 +135,28 @@ namespace
|
||||
IndexType,
|
||||
ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexStringHash,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
true,
|
||||
has_base_index>,
|
||||
ReverseIndexStringHash>
|
||||
ReverseIndexHash>
|
||||
{
|
||||
using Base = HashTableWithPublicState<
|
||||
IndexType,
|
||||
ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexStringHash,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
true,
|
||||
has_base_index>,
|
||||
ReverseIndexStringHash>;
|
||||
ReverseIndexHash>;
|
||||
public:
|
||||
using Base::Base;
|
||||
friend struct ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexStringHash,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexStringHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
true,
|
||||
@ -180,28 +168,28 @@ namespace
|
||||
IndexType,
|
||||
ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
false,
|
||||
has_base_index>,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>>
|
||||
ReverseIndexHash>
|
||||
{
|
||||
using Base = HashTableWithPublicState<
|
||||
IndexType,
|
||||
ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
false,
|
||||
has_base_index>,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>>;
|
||||
ReverseIndexHash>;
|
||||
public:
|
||||
using Base::Base;
|
||||
friend struct ReverseIndexHashTableCell<
|
||||
IndexType,
|
||||
ReverseIndexNumberHash<typename ColumnType::value_type>,
|
||||
ReverseIndexHash,
|
||||
ReverseIndexNumberHashTable<IndexType, ColumnType, has_base_index>,
|
||||
ColumnType,
|
||||
false,
|
||||
@ -253,8 +241,7 @@ public:
|
||||
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 insert(const StringRef & data);
|
||||
UInt64 getInsertionPoint(const StringRef & data);
|
||||
UInt64 lastInsertionPoint() const { return size() + base_index; }
|
||||
|
||||
@ -367,7 +354,7 @@ void ReverseIndex<IndexType, ColumnType>::buildIndex()
|
||||
else
|
||||
hash = getHash(column->getDataAt(row));
|
||||
|
||||
index->emplace(row + base_index, iterator, inserted, hash);
|
||||
index->emplace(row + base_index, iterator, inserted, hash, column->getDataAt(row));
|
||||
|
||||
if (!inserted)
|
||||
throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -390,7 +377,7 @@ ColumnUInt64::MutablePtr ReverseIndex<IndexType, ColumnType>::calcHashes() const
|
||||
}
|
||||
|
||||
template <typename IndexType, typename ColumnType>
|
||||
UInt64 ReverseIndex<IndexType, ColumnType>::insert(UInt64 from_position)
|
||||
UInt64 ReverseIndex<IndexType, ColumnType>::insert(const StringRef & data)
|
||||
{
|
||||
if (!index)
|
||||
buildIndex();
|
||||
@ -399,42 +386,35 @@ UInt64 ReverseIndex<IndexType, ColumnType>::insert(UInt64 from_position)
|
||||
IteratorType iterator;
|
||||
bool inserted;
|
||||
|
||||
auto hash = getHash(column->getDataAt(from_position));
|
||||
auto hash = getHash(data);
|
||||
UInt64 num_rows = size();
|
||||
|
||||
if constexpr (use_saved_hash)
|
||||
{
|
||||
auto & data = saved_hash->getData();
|
||||
if (data.size() <= from_position)
|
||||
data.resize(from_position + 1);
|
||||
data[from_position] = hash;
|
||||
if (data.size() <= num_rows)
|
||||
data.resize(num_rows + 1);
|
||||
data[num_rows] = hash;
|
||||
}
|
||||
else
|
||||
column->insertData(data.data, data.size);
|
||||
|
||||
index->emplace(num_rows + base_index, iterator, inserted, hash, data);
|
||||
|
||||
if constexpr (use_saved_hash)
|
||||
{
|
||||
if (inserted)
|
||||
column->insertData(data.data, data.size);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!inserted)
|
||||
column->popBack(1);
|
||||
}
|
||||
|
||||
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)
|
||||
{
|
||||
|
4
dbms/src/Columns/tests/CMakeLists.txt
Normal file
4
dbms/src/Columns/tests/CMakeLists.txt
Normal file
@ -0,0 +1,4 @@
|
||||
set(SRCS)
|
||||
|
||||
add_executable (column_unique column_unique.cpp ${SRCS})
|
||||
target_link_libraries (column_unique PRIVATE dbms gtest_main)
|
103
dbms/src/Columns/tests/column_unique.cpp
Normal file
103
dbms/src/Columns/tests/column_unique.cpp
Normal file
@ -0,0 +1,103 @@
|
||||
#include <Columns/ColumnUnique.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
#pragma GCC diagnostic ignored "-Wsign-compare"
|
||||
#ifdef __clang__
|
||||
#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
|
||||
#endif
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
#include <unordered_map>
|
||||
#include <vector>
|
||||
using namespace DB;
|
||||
|
||||
TEST(column_unique, column_unique_unique_insert_range_Test)
|
||||
{
|
||||
std::unordered_map<String, size_t> ref_map;
|
||||
auto data_type = std::make_shared<DataTypeString>();
|
||||
auto column_unique = ColumnUnique<ColumnString>::create(*data_type);
|
||||
auto column_string = ColumnString::create();
|
||||
|
||||
size_t num_values = 1000000;
|
||||
size_t mod_to = 1000;
|
||||
|
||||
std::vector<size_t> indexes(num_values);
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
String str = toString(i % mod_to);
|
||||
column_string->insertData(str.data(), str.size());
|
||||
|
||||
if (ref_map.count(str) == 0)
|
||||
ref_map[str] = ref_map.size();
|
||||
|
||||
indexes[i]= ref_map[str];
|
||||
}
|
||||
|
||||
auto idx = column_unique->uniqueInsertRangeFrom(*column_string, 0, num_values);
|
||||
ASSERT_EQ(idx->size(), num_values);
|
||||
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
ASSERT_EQ(indexes[i] + 1, idx->getUInt(i)) << "Different indexes at position " << i;
|
||||
}
|
||||
|
||||
auto & nested = column_unique->getNestedColumn();
|
||||
ASSERT_EQ(nested->size(), mod_to + 1);
|
||||
|
||||
for (size_t i = 0; i < mod_to; ++i)
|
||||
{
|
||||
ASSERT_EQ(std::to_string(i), nested->getDataAt(i + 1).toString());
|
||||
}
|
||||
}
|
||||
|
||||
TEST(column_unique, column_unique_unique_insert_range_with_overflow_Test)
|
||||
{
|
||||
std::unordered_map<String, size_t> ref_map;
|
||||
auto data_type = std::make_shared<DataTypeString>();
|
||||
auto column_unique = ColumnUnique<ColumnString>::create(*data_type);
|
||||
auto column_string = ColumnString::create();
|
||||
|
||||
size_t num_values = 1000000;
|
||||
size_t mod_to = 1000;
|
||||
|
||||
std::vector<size_t> indexes(num_values);
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
String str = toString(i % mod_to);
|
||||
column_string->insertData(str.data(), str.size());
|
||||
|
||||
if (ref_map.count(str) == 0)
|
||||
ref_map[str] = ref_map.size();
|
||||
|
||||
indexes[i]= ref_map[str];
|
||||
}
|
||||
|
||||
size_t max_val = mod_to / 2;
|
||||
size_t max_dict_size = max_val + 1;
|
||||
auto idx_with_overflow = column_unique->uniqueInsertRangeWithOverflow(*column_string, 0, num_values, max_dict_size);
|
||||
auto & idx = idx_with_overflow.indexes;
|
||||
auto & add_keys = idx_with_overflow.overflowed_keys;
|
||||
|
||||
ASSERT_EQ(idx->size(), num_values);
|
||||
|
||||
for (size_t i = 0; i < num_values; ++i)
|
||||
{
|
||||
ASSERT_EQ(indexes[i] + 1, idx->getUInt(i)) << "Different indexes at position " << i;
|
||||
}
|
||||
|
||||
auto & nested = column_unique->getNestedColumn();
|
||||
ASSERT_EQ(nested->size(), max_dict_size);
|
||||
ASSERT_EQ(add_keys->size(), mod_to - max_val);
|
||||
|
||||
for (size_t i = 0; i < max_val; ++i)
|
||||
{
|
||||
ASSERT_EQ(std::to_string(i), nested->getDataAt(i + 1).toString());
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < mod_to - max_val; ++i)
|
||||
{
|
||||
ASSERT_EQ(std::to_string(max_val + i), add_keys->getDataAt(i).toString());
|
||||
}
|
||||
}
|
@ -658,12 +658,8 @@ protected:
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
/// Only for non-zero keys. Find the right place, insert the key there, if it does not already exist. Set iterator to the cell in output parameter.
|
||||
void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value)
|
||||
void ALWAYS_INLINE emplaceNonZeroImpl(size_t place_value, Key x, iterator & it, bool & inserted, size_t hash_value)
|
||||
{
|
||||
size_t place_value = findCell(x, hash_value, grower.place(hash_value));
|
||||
|
||||
it = iterator(this, &buf[place_value]);
|
||||
|
||||
if (!buf[place_value].isZero(*this))
|
||||
@ -698,6 +694,21 @@ protected:
|
||||
}
|
||||
}
|
||||
|
||||
/// Only for non-zero keys. Find the right place, insert the key there, if it does not already exist. Set iterator to the cell in output parameter.
|
||||
void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value)
|
||||
{
|
||||
size_t place_value = findCell(x, hash_value, grower.place(hash_value));
|
||||
emplaceNonZeroImpl(place_value, x, it, inserted, hash_value);
|
||||
}
|
||||
|
||||
/// Same but find place using object. Hack for ReverseIndex.
|
||||
template <typename ObjectToCompareWith>
|
||||
void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
|
||||
{
|
||||
size_t place_value = findCell(object, hash_value, grower.place(hash_value));
|
||||
emplaceNonZeroImpl(place_value, x, it, inserted, hash_value);
|
||||
}
|
||||
|
||||
|
||||
public:
|
||||
/// Insert a value. In the case of any more complex values, it is better to use the `emplace` function.
|
||||
@ -753,6 +764,13 @@ public:
|
||||
emplaceNonZero(x, it, inserted, hash_value);
|
||||
}
|
||||
|
||||
/// Same, but search position by object. Hack for ReverseIndex.
|
||||
template <typename ObjectToCompareWith>
|
||||
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t hash_value, const ObjectToCompareWith & object)
|
||||
{
|
||||
if (!emplaceIfZero(x, it, inserted, hash_value))
|
||||
emplaceNonZero(x, it, inserted, hash_value, object);
|
||||
}
|
||||
|
||||
/// Copy the cell from another hash table. It is assumed that the cell is not zero, and also that there was no such key in the table yet.
|
||||
void ALWAYS_INLINE insertUniqueNonZero(const Cell * cell, size_t hash_value)
|
||||
|
Loading…
Reference in New Issue
Block a user