Expanding indexes for ColumnUnique.

This commit is contained in:
Nikolai Kochetov 2018-06-28 21:12:29 +03:00
parent bf3327da8b
commit c542cb6314
3 changed files with 233 additions and 67 deletions

View File

@ -7,6 +7,7 @@
#include <Columns/ColumnNullable.h> #include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <DataTypes/NumberTraits.h>
class NullMap; class NullMap;
@ -48,10 +49,10 @@ namespace ZeroTraits
namespace DB namespace DB
{ {
template <typename ColumnType, typename IndexType> template <typename ColumnType>
class ColumnUnique final : public COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType, IndexType>> class ColumnUnique final : public COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>>
{ {
friend class COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType, IndexType>>; friend class COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>;
private: private:
explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable); explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable);
@ -122,7 +123,7 @@ public:
private: private:
using IndexMapType = HashMap<StringRefWrapper<ColumnType>, IndexType, StringRefHash>; using IndexMapType = HashMap<StringRefWrapper<ColumnType>, UInt64, StringRefHash>;
ColumnPtr column_holder; ColumnPtr column_holder;
@ -139,26 +140,27 @@ private:
void buildIndex(); void buildIndex();
ColumnType * getRawColumnPtr() { return static_cast<ColumnType *>(column_holder->assumeMutable().get()); } ColumnType * getRawColumnPtr() { return static_cast<ColumnType *>(column_holder->assumeMutable().get()); }
const ColumnType * getRawColumnPtr() const { return static_cast<const ColumnType *>(column_holder.get()); } const ColumnType * getRawColumnPtr() const { return static_cast<const ColumnType *>(column_holder.get()); }
IndexType insertIntoMap(const StringRefWrapper<ColumnType> & ref, IndexType value); UInt64 insertIntoMap(const StringRefWrapper<ColumnType> & ref, UInt64 value);
void uniqueInsertRangeImpl( template <typename IndexType>
MutableColumnPtr uniqueInsertRangeImpl(
const IColumn & src, const IColumn & src,
size_t start, size_t start,
size_t length, size_t length,
typename ColumnVector<IndexType>::Container & positions, typename ColumnVector<IndexType>::MutablePtr && positions_column,
ColumnType * overflowed_keys, ColumnType * overflowed_keys,
size_t max_dictionary_size); size_t max_dictionary_size);
}; };
template <typename ColumnType, typename IndexType> template <typename ColumnType>
ColumnUnique<ColumnType, IndexType>::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable()) ColumnUnique<ColumnType>::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable())
{ {
const auto & holder_type = is_nullable ? *static_cast<const DataTypeNullable &>(type).getNestedType() : type; const auto & holder_type = is_nullable ? *static_cast<const DataTypeNullable &>(type).getNestedType() : type;
column_holder = holder_type.createColumn()->cloneResized(numSpecialValues()); column_holder = holder_type.createColumn()->cloneResized(numSpecialValues());
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
ColumnUnique<ColumnType, IndexType>::ColumnUnique(MutableColumnPtr && holder, bool is_nullable) 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)
{ {
if (column_holder->size() < numSpecialValues()) if (column_holder->size() < numSpecialValues())
@ -167,8 +169,8 @@ ColumnUnique<ColumnType, IndexType>::ColumnUnique(MutableColumnPtr && holder, bo
throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN); throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN);
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
ColumnPtr ColumnUnique<ColumnType, IndexType>::getNestedColumn() const ColumnPtr ColumnUnique<ColumnType>::getNestedColumn() const
{ {
if (is_nullable) if (is_nullable)
{ {
@ -192,8 +194,8 @@ ColumnPtr ColumnUnique<ColumnType, IndexType>::getNestedColumn() const
return column_holder; return column_holder;
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
size_t ColumnUnique<ColumnType, IndexType>::getNullValueIndex() const size_t ColumnUnique<ColumnType>::getNullValueIndex() const
{ {
if (!is_nullable) if (!is_nullable)
throw Exception("ColumnUnique can't contain null values.", ErrorCodes::LOGICAL_ERROR); throw Exception("ColumnUnique can't contain null values.", ErrorCodes::LOGICAL_ERROR);
@ -201,8 +203,8 @@ size_t ColumnUnique<ColumnType, IndexType>::getNullValueIndex() const
return 0; return 0;
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
void ColumnUnique<ColumnType, IndexType>::buildIndex() void ColumnUnique<ColumnType>::buildIndex()
{ {
if (index) if (index)
return; return;
@ -216,8 +218,8 @@ void ColumnUnique<ColumnType, IndexType>::buildIndex()
} }
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
IndexType ColumnUnique<ColumnType, IndexType>::insertIntoMap(const StringRefWrapper<ColumnType> & ref, IndexType value) IndexType ColumnUnique<ColumnType>::insertIntoMap(const StringRefWrapper<ColumnType> & ref, IndexType value)
{ {
if (!index) if (!index)
buildIndex(); buildIndex();
@ -233,8 +235,8 @@ IndexType ColumnUnique<ColumnType, IndexType>::insertIntoMap(const StringRefWrap
return it->second; return it->second;
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
size_t ColumnUnique<ColumnType, IndexType>::uniqueInsert(const Field & x) size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
{ {
if (x.getType() == Field::Types::Null) if (x.getType() == Field::Types::Null)
return getNullValueIndex(); return getNullValueIndex();
@ -253,8 +255,8 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsert(const Field & x)
return pos; return pos;
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertFrom(const IColumn & src, size_t n) size_t ColumnUnique<ColumnType>::uniqueInsertFrom(const IColumn & src, size_t n)
{ {
if (is_nullable && src.isNullAt(n)) if (is_nullable && src.isNullAt(n))
return getNullValueIndex(); return getNullValueIndex();
@ -263,8 +265,8 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertFrom(const IColumn & src
return uniqueInsertData(ref.data, ref.size); return uniqueInsertData(ref.data, ref.size);
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertData(const char * pos, size_t length) size_t ColumnUnique<ColumnType>::uniqueInsertData(const char * pos, size_t length)
{ {
if (!index) if (!index)
buildIndex(); buildIndex();
@ -274,7 +276,7 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertData(const char * pos, s
if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length)) if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length))
return getDefaultValueIndex(); return getDefaultValueIndex();
auto size = static_cast<IndexType>(column->size()); UInt64 size = column->size();
auto iter = index->find(StringRefWrapper<ColumnType>(StringRef(pos, length))); auto iter = index->find(StringRefWrapper<ColumnType>(StringRef(pos, length)));
if (iter == index->end()) if (iter == index->end())
@ -286,8 +288,8 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertData(const char * pos, s
return iter->second; return iter->second;
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) size_t ColumnUnique<ColumnType>::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length)
{ {
if (std::is_same<ColumnType, ColumnString>::value) if (std::is_same<ColumnType, ColumnString>::value)
return uniqueInsertData(pos, length - 1); return uniqueInsertData(pos, length - 1);
@ -314,8 +316,8 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertDataWithTerminatingZero(
return static_cast<size_t>(position); return static_cast<size_t>(position);
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
size_t ColumnUnique<ColumnType, IndexType>::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) size_t ColumnUnique<ColumnType>::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos)
{ {
auto column = getRawColumnPtr(); auto column = getRawColumnPtr();
size_t prev_size = column->size(); size_t prev_size = column->size();
@ -334,12 +336,13 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueDeserializeAndInsertFromArena(
return static_cast<size_t>(index_pos); return static_cast<size_t>(index_pos);
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl( template <typename IndexType>
MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
const IColumn & src, const IColumn & src,
size_t start, size_t start,
size_t length, size_t length,
typename ColumnVector<IndexType>::Container & positions, typename ColumnVector<IndexType>::MutablePtr && positions_column,
ColumnType * overflowed_keys, ColumnType * overflowed_keys,
size_t max_dictionary_size) size_t max_dictionary_size)
{ {
@ -348,6 +351,35 @@ void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
const ColumnType * src_column; const ColumnType * src_column;
const NullMap * null_map = nullptr; const NullMap * null_map = nullptr;
auto & positions = positions_column->getData();
using SuperiorIndexType = NumberTraits::Construct<false, false, NumberTraits::nextSize(sizeof(IndexType))>::Type;
auto updatePosition = [&](UInt64 & next_position, UInt64 num_added_rows) -> MutableColumnPtr
{
++next_position;
if (next_position > std::numeric_limits<IndexType>::max())
{
if (sizeof(SuperiorIndexType) == sizeof(IndexType))
throw Exception("Can't find superior index type for type " + demangle(typeid(IndexType).name()),
ErrorCodes::LOGICAL_ERROR);
auto expanded_column = ColumnVector<IndexType>::create(length);
auto & expanded_data = expanded_column->getData();
for (size_t i = 0; i < num_added_rows; ++i)
expanded_data[i] = positions[i];
return uniqueInsertRangeImpl<SuperiorIndexType>(
src,
start + num_added_rows,
length - num_added_rows,
std::move(expanded_column),
overflowed_keys,
max_dictionary_size);
}
return nullptr;
};
if (src.isColumnNullable()) if (src.isColumnNullable())
{ {
@ -364,7 +396,7 @@ void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
auto column = getRawColumnPtr(); auto column = getRawColumnPtr();
size_t next_position = column->size(); UInt64 next_position = column->size();
for (auto i : ext::range(0, length)) for (auto i : ext::range(0, length))
{ {
auto row = start + i; auto row = start + i;
@ -388,7 +420,9 @@ void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
auto ref = src_column->getDataAt(row); auto ref = src_column->getDataAt(row);
overflowed_keys->insertData(ref.data, ref.size); overflowed_keys->insertData(ref.data, ref.size);
(*secondary_index)[StringRefWrapper<ColumnType>(src_column, row)] = next_position; (*secondary_index)[StringRefWrapper<ColumnType>(src_column, row)] = next_position;
++next_position;
if (auto res = updatePosition(next_position, i))
return res;
} }
else else
positions[i] = jt->second; positions[i] = jt->second;
@ -399,43 +433,92 @@ void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
auto ref = src_column->getDataAt(row); auto ref = src_column->getDataAt(row);
column->insertData(ref.data, ref.size); column->insertData(ref.data, ref.size);
(*index)[StringRefWrapper<ColumnType>(column, next_position)] = next_position; (*index)[StringRefWrapper<ColumnType>(column, next_position)] = next_position;
++next_position;
if (auto res = updatePosition(next_position, i))
return res;
} }
} }
else else
positions[i] = it->second; positions[i] = it->second;
} }
} }
}
template <typename ColumnType, typename IndexType>
MutableColumnPtr ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length)
{
auto positions_column = ColumnVector<IndexType>::create(length);
auto & positions = positions_column->getData();
uniqueInsertRangeImpl(src, start, length, positions, nullptr, 0);
return positions_column; return positions_column;
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeWithOverflow( MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length)
{
size_t size = getRawColumnPtr()->size();
auto callForType = [&](auto x)
{
using IndexType = decltype(x);
if (size <= std::numeric_limits<IndexType>::max())
{
auto positions_column = ColumnVector<IndexType>::create(length);
auto & positions = positions_column->getData();
return uniqueInsertRangeImpl(src, start, length, positions, nullptr, 0);
}
return nullptr;
};
MutableColumnPtr positions_column;
if (!positions_column)
positions_column = callForType(UInt8());
if (!positions_column)
positions_column = callForType(UInt16());
if (!positions_column)
positions_column = callForType(UInt32());
if (!positions_column)
positions_column = callForType(UInt64());
if (!positions_column)
throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR);
return positions_column;
}
template <typename ColumnType>
IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWithOverflow(
const IColumn & src, const IColumn & src,
size_t start, size_t start,
size_t length, size_t length,
size_t max_dictionary_size) size_t max_dictionary_size)
{ {
auto positions_column = ColumnVector<IndexType>::create(length); size_t size = getRawColumnPtr()->size();
auto overflowed_keys = column_holder->cloneEmpty(); auto overflowed_keys = column_holder->cloneEmpty();
auto & positions = positions_column->getData();
auto overflowed_keys_ptr = typeid_cast<ColumnType *>(overflowed_keys.get()); auto overflowed_keys_ptr = typeid_cast<ColumnType *>(overflowed_keys.get());
if (!overflowed_keys_ptr) if (!overflowed_keys_ptr)
throw Exception("Invalid keys type for ColumnUnique.", ErrorCodes::LOGICAL_ERROR); throw Exception("Invalid keys type for ColumnUnique.", ErrorCodes::LOGICAL_ERROR);
uniqueInsertRangeImpl(src, start, length, positions, overflowed_keys_ptr, max_dictionary_size); auto callForType = [&](auto x)
{
using IndexType = decltype(x);
if (size <= std::numeric_limits<IndexType>::max())
{
auto positions_column = ColumnVector<IndexType>::create(length);
auto & positions = positions_column->getData();
return uniqueInsertRangeImpl(src, start, length, positions, overflowed_keys_ptr, max_dictionary_size);
}
return nullptr;
};
MutableColumnPtr positions_column;
if (!positions_column)
positions_column = callForType(UInt8());
if (!positions_column)
positions_column = callForType(UInt16());
if (!positions_column)
positions_column = callForType(UInt32());
if (!positions_column)
positions_column = callForType(UInt64());
if (!positions_column)
throw Exception("Can't find index type for ColumnUnique", ErrorCodes::LOGICAL_ERROR);
IColumnUnique::IndexesWithOverflow indexes_with_overflow; IColumnUnique::IndexesWithOverflow indexes_with_overflow;
indexes_with_overflow.indexes = std::move(positions_column); indexes_with_overflow.indexes = std::move(positions_column);
@ -443,8 +526,8 @@ IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType, IndexType>::uniqueIn
return indexes_with_overflow; return indexes_with_overflow;
} }
template <typename ColumnType, typename IndexType> template <typename ColumnType>
IColumnUnique::SerializableState ColumnUnique<ColumnType, IndexType>::getSerializableState() const IColumnUnique::SerializableState ColumnUnique<ColumnType>::getSerializableState() const
{ {
IColumnUnique::SerializableState state; IColumnUnique::SerializableState state;
state.column = column_holder; state.column = column_holder;

View File

@ -9,6 +9,8 @@ ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, M
{ {
if (!dynamic_cast<const IColumnUnique *>(column_unique.get())) if (!dynamic_cast<const IColumnUnique *>(column_unique.get()))
throw Exception("ColumnUnique expected as argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); throw Exception("ColumnUnique expected as argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN);
getSizeOfCurrentIndexType();
} }
ColumnWithDictionary::ColumnWithDictionary(const ColumnWithDictionary & other) ColumnWithDictionary::ColumnWithDictionary(const ColumnWithDictionary & other)
@ -21,4 +23,90 @@ void ColumnWithDictionary::gather(ColumnGathererStream & gatherer)
gatherer.gather(*this); gatherer.gather(*this);
} }
MutableColumnPtr ColumnWithDictionary::cloneResized(size_t size) const
{
auto unique_ptr = column_unique;
return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), indexes->cloneResized(size));
}
size_t ColumnWithDictionary::getSizeOfCurrentIndexType() const
{
if (typeid_cast<const ColumnUInt8 *>(indexes.get()))
return sizeof(UInt8);
if (typeid_cast<const ColumnUInt16 *>(indexes.get()))
return sizeof(UInt16);
if (typeid_cast<const ColumnUInt32 *>(indexes.get()))
return sizeof(UInt32);
if (typeid_cast<const ColumnUInt64 *>(indexes.get()))
return sizeof(UInt64);
throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected ColumnUInt, got " + indexes->getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
template <typename IndexType>
void ColumnWithDictionary::convertIndexes()
{
auto convert = [&](auto x)
{
using CurIndexType = typeof(x);
if (auto * index_col = typeid_cast<const ColumnVector<CurIndexType> *>(indexes.get()))
{
if (sizeof(CurIndexType) != sizeof(IndexType))
{
size_t size = index_col->size();
auto new_index_col = ColumnVector<IndexType>::create(size);
auto & data = index_col->getData();
auto & new_data = new_index_col->getData();
for (size_t i = 0; i < size; ++i)
new_data[i] = data[i];
indexes = std::move(new_index_col);
}
return true;
}
return false;
};
if (!convert(UInt8()) &&
!convert(UInt16()) &&
!convert(UInt32()) &&
!convert(UInt64()))
throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected ColumnUInt, got "
+ indexes->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
void ColumnWithDictionary::insertIndex(size_t value)
{
auto current_index_type = getSizeOfCurrentIndexType();
auto insertForType = [&](auto x)
{
using IndexType = typeof(x);
if (value <= std::numeric_limits<IndexType>::max())
{
if (sizeof(IndexType) > current_index_type)
convertIndexes<IndexType>();
getIndexes()->insert(UInt64(value));
return true;
}
return false;
};
if (!insertForType(UInt8()) &&
!insertForType(UInt16()) &&
!insertForType(UInt32()) &&
!insertForType(UInt64()))
throw Exception("Unexpected indexes type for ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN);
}
void ColumnWithDictionary::insertIndexesRange(const ColumnPtr & column)
{
}
} }

View File

@ -37,34 +37,22 @@ public:
std::string getName() const override { return "ColumnWithDictionary"; } std::string getName() const override { return "ColumnWithDictionary"; }
const char * getFamilyName() const override { return "ColumnWithDictionary"; } const char * getFamilyName() const override { return "ColumnWithDictionary"; }
ColumnPtr convertToFullColumn() const ColumnPtr convertToFullColumn() const { return getUnique()->getNestedColumn()->index(*indexes, 0); }
{
return getUnique()->getNestedColumn()->index(*indexes, 0);
}
ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); } ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); }
MutableColumnPtr cloneResized(size_t size) const override MutableColumnPtr cloneResized(size_t size) const override;
{
auto unique_ptr = column_unique;
return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), indexes->cloneResized(size));
}
size_t size() const override { return indexes->size(); } size_t size() const override { return indexes->size(); }
Field operator[](size_t n) const override { return (*column_unique)[indexes->getUInt(n)]; } Field operator[](size_t n) const override { return (*column_unique)[indexes->getUInt(n)]; }
void get(size_t n, Field & res) const override { column_unique->get(indexes->getUInt(n), res); } void get(size_t n, Field & res) const override { column_unique->get(indexes->getUInt(n), res); }
StringRef getDataAt(size_t n) const override { return column_unique->getDataAt(indexes->getUInt(n)); } StringRef getDataAt(size_t n) const override { return column_unique->getDataAt(indexes->getUInt(n)); }
StringRef getDataAtWithTerminatingZero(size_t n) const override StringRef getDataAtWithTerminatingZero(size_t n) const override
{ {
return column_unique->getDataAtWithTerminatingZero(indexes->getUInt(n)); return column_unique->getDataAtWithTerminatingZero(indexes->getUInt(n));
} }
UInt64 get64(size_t n) const override { return column_unique->get64(indexes->getUInt(n)); } UInt64 get64(size_t n) const override { return column_unique->get64(indexes->getUInt(n)); }
UInt64 getUInt(size_t n) const override { return column_unique->getUInt(indexes->getUInt(n)); } UInt64 getUInt(size_t n) const override { return column_unique->getUInt(indexes->getUInt(n)); }
Int64 getInt(size_t n) const override { return column_unique->getInt(indexes->getUInt(n)); } Int64 getInt(size_t n) const override { return column_unique->getInt(indexes->getUInt(n)); }
bool isNullAt(size_t n) const override { return column_unique->isNullAt(indexes->getUInt(n)); } bool isNullAt(size_t n) const override { return column_unique->isNullAt(indexes->getUInt(n)); }
@ -252,6 +240,13 @@ private:
ColumnPtr column_unique; ColumnPtr column_unique;
ColumnPtr indexes; ColumnPtr indexes;
size_t getSizeOfCurrentIndexType() const;
template <typename IndexType>
void convertIndexes();
void insertIndex(size_t value);
void insertIndexesRange(const ColumnPtr & column);
}; };