mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 00:22:29 +00:00
Expanding indexes for ColumnUnique.
This commit is contained in:
parent
bf3327da8b
commit
c542cb6314
@ -7,6 +7,7 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/NumberTraits.h>
|
||||
|
||||
class NullMap;
|
||||
|
||||
@ -48,10 +49,10 @@ namespace ZeroTraits
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
class ColumnUnique final : public COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType, IndexType>>
|
||||
template <typename ColumnType>
|
||||
class ColumnUnique final : public COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>>
|
||||
{
|
||||
friend class COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType, IndexType>>;
|
||||
friend class COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>;
|
||||
|
||||
private:
|
||||
explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable);
|
||||
@ -122,7 +123,7 @@ public:
|
||||
|
||||
private:
|
||||
|
||||
using IndexMapType = HashMap<StringRefWrapper<ColumnType>, IndexType, StringRefHash>;
|
||||
using IndexMapType = HashMap<StringRefWrapper<ColumnType>, UInt64, StringRefHash>;
|
||||
|
||||
ColumnPtr column_holder;
|
||||
|
||||
@ -139,26 +140,27 @@ private:
|
||||
void buildIndex();
|
||||
ColumnType * getRawColumnPtr() { return static_cast<ColumnType *>(column_holder->assumeMutable().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,
|
||||
size_t start,
|
||||
size_t length,
|
||||
typename ColumnVector<IndexType>::Container & positions,
|
||||
typename ColumnVector<IndexType>::MutablePtr && positions_column,
|
||||
ColumnType * overflowed_keys,
|
||||
size_t max_dictionary_size);
|
||||
};
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
ColumnUnique<ColumnType, IndexType>::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable())
|
||||
template <typename ColumnType>
|
||||
ColumnUnique<ColumnType>::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable())
|
||||
{
|
||||
const auto & holder_type = is_nullable ? *static_cast<const DataTypeNullable &>(type).getNestedType() : type;
|
||||
column_holder = holder_type.createColumn()->cloneResized(numSpecialValues());
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
ColumnUnique<ColumnType, IndexType>::ColumnUnique(MutableColumnPtr && holder, bool is_nullable)
|
||||
template <typename ColumnType>
|
||||
ColumnUnique<ColumnType>::ColumnUnique(MutableColumnPtr && holder, bool is_nullable)
|
||||
: column_holder(std::move(holder)), is_nullable(is_nullable)
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
ColumnPtr ColumnUnique<ColumnType, IndexType>::getNestedColumn() const
|
||||
template <typename ColumnType>
|
||||
ColumnPtr ColumnUnique<ColumnType>::getNestedColumn() const
|
||||
{
|
||||
if (is_nullable)
|
||||
{
|
||||
@ -192,8 +194,8 @@ ColumnPtr ColumnUnique<ColumnType, IndexType>::getNestedColumn() const
|
||||
return column_holder;
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
size_t ColumnUnique<ColumnType, IndexType>::getNullValueIndex() const
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::getNullValueIndex() const
|
||||
{
|
||||
if (!is_nullable)
|
||||
throw Exception("ColumnUnique can't contain null values.", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -201,8 +203,8 @@ size_t ColumnUnique<ColumnType, IndexType>::getNullValueIndex() const
|
||||
return 0;
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
void ColumnUnique<ColumnType, IndexType>::buildIndex()
|
||||
template <typename ColumnType>
|
||||
void ColumnUnique<ColumnType>::buildIndex()
|
||||
{
|
||||
if (index)
|
||||
return;
|
||||
@ -216,8 +218,8 @@ void ColumnUnique<ColumnType, IndexType>::buildIndex()
|
||||
}
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
IndexType ColumnUnique<ColumnType, IndexType>::insertIntoMap(const StringRefWrapper<ColumnType> & ref, IndexType value)
|
||||
template <typename ColumnType>
|
||||
IndexType ColumnUnique<ColumnType>::insertIntoMap(const StringRefWrapper<ColumnType> & ref, IndexType value)
|
||||
{
|
||||
if (!index)
|
||||
buildIndex();
|
||||
@ -233,8 +235,8 @@ IndexType ColumnUnique<ColumnType, IndexType>::insertIntoMap(const StringRefWrap
|
||||
return it->second;
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
size_t ColumnUnique<ColumnType, IndexType>::uniqueInsert(const Field & x)
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
||||
{
|
||||
if (x.getType() == Field::Types::Null)
|
||||
return getNullValueIndex();
|
||||
@ -253,8 +255,8 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsert(const Field & x)
|
||||
return pos;
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertFrom(const IColumn & src, size_t n)
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::uniqueInsertFrom(const IColumn & src, size_t n)
|
||||
{
|
||||
if (is_nullable && src.isNullAt(n))
|
||||
return getNullValueIndex();
|
||||
@ -263,8 +265,8 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertFrom(const IColumn & src
|
||||
return uniqueInsertData(ref.data, ref.size);
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertData(const char * pos, size_t length)
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::uniqueInsertData(const char * pos, size_t length)
|
||||
{
|
||||
if (!index)
|
||||
buildIndex();
|
||||
@ -274,7 +276,7 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertData(const char * pos, s
|
||||
if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length))
|
||||
return getDefaultValueIndex();
|
||||
|
||||
auto size = static_cast<IndexType>(column->size());
|
||||
UInt64 size = column->size();
|
||||
auto iter = index->find(StringRefWrapper<ColumnType>(StringRef(pos, length)));
|
||||
|
||||
if (iter == index->end())
|
||||
@ -286,8 +288,8 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertData(const char * pos, s
|
||||
return iter->second;
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length)
|
||||
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);
|
||||
@ -314,8 +316,8 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueInsertDataWithTerminatingZero(
|
||||
return static_cast<size_t>(position);
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
size_t ColumnUnique<ColumnType, IndexType>::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos)
|
||||
template <typename ColumnType>
|
||||
size_t ColumnUnique<ColumnType>::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos)
|
||||
{
|
||||
auto column = getRawColumnPtr();
|
||||
size_t prev_size = column->size();
|
||||
@ -334,12 +336,13 @@ size_t ColumnUnique<ColumnType, IndexType>::uniqueDeserializeAndInsertFromArena(
|
||||
return static_cast<size_t>(index_pos);
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
|
||||
template <typename ColumnType>
|
||||
template <typename IndexType>
|
||||
MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
||||
const IColumn & src,
|
||||
size_t start,
|
||||
size_t length,
|
||||
typename ColumnVector<IndexType>::Container & positions,
|
||||
typename ColumnVector<IndexType>::MutablePtr && positions_column,
|
||||
ColumnType * overflowed_keys,
|
||||
size_t max_dictionary_size)
|
||||
{
|
||||
@ -348,6 +351,35 @@ void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
|
||||
|
||||
const ColumnType * src_column;
|
||||
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())
|
||||
{
|
||||
@ -364,7 +396,7 @@ void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
|
||||
|
||||
auto column = getRawColumnPtr();
|
||||
|
||||
size_t next_position = column->size();
|
||||
UInt64 next_position = column->size();
|
||||
for (auto i : ext::range(0, length))
|
||||
{
|
||||
auto row = start + i;
|
||||
@ -388,7 +420,9 @@ void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
|
||||
auto ref = src_column->getDataAt(row);
|
||||
overflowed_keys->insertData(ref.data, ref.size);
|
||||
(*secondary_index)[StringRefWrapper<ColumnType>(src_column, row)] = next_position;
|
||||
++next_position;
|
||||
|
||||
if (auto res = updatePosition(next_position, i))
|
||||
return res;
|
||||
}
|
||||
else
|
||||
positions[i] = jt->second;
|
||||
@ -399,43 +433,92 @@ void ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeImpl(
|
||||
auto ref = src_column->getDataAt(row);
|
||||
column->insertData(ref.data, ref.size);
|
||||
(*index)[StringRefWrapper<ColumnType>(column, next_position)] = next_position;
|
||||
++next_position;
|
||||
|
||||
if (auto res = updatePosition(next_position, i))
|
||||
return res;
|
||||
}
|
||||
}
|
||||
else
|
||||
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;
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType, IndexType>::uniqueInsertRangeWithOverflow(
|
||||
template <typename ColumnType>
|
||||
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,
|
||||
size_t start,
|
||||
size_t length,
|
||||
size_t max_dictionary_size)
|
||||
{
|
||||
|
||||
auto positions_column = ColumnVector<IndexType>::create(length);
|
||||
size_t size = getRawColumnPtr()->size();
|
||||
auto overflowed_keys = column_holder->cloneEmpty();
|
||||
auto & positions = positions_column->getData();
|
||||
|
||||
auto overflowed_keys_ptr = typeid_cast<ColumnType *>(overflowed_keys.get());
|
||||
if (!overflowed_keys_ptr)
|
||||
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;
|
||||
indexes_with_overflow.indexes = std::move(positions_column);
|
||||
@ -443,8 +526,8 @@ IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType, IndexType>::uniqueIn
|
||||
return indexes_with_overflow;
|
||||
}
|
||||
|
||||
template <typename ColumnType, typename IndexType>
|
||||
IColumnUnique::SerializableState ColumnUnique<ColumnType, IndexType>::getSerializableState() const
|
||||
template <typename ColumnType>
|
||||
IColumnUnique::SerializableState ColumnUnique<ColumnType>::getSerializableState() const
|
||||
{
|
||||
IColumnUnique::SerializableState state;
|
||||
state.column = column_holder;
|
||||
|
@ -9,6 +9,8 @@ ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, M
|
||||
{
|
||||
if (!dynamic_cast<const IColumnUnique *>(column_unique.get()))
|
||||
throw Exception("ColumnUnique expected as argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
getSizeOfCurrentIndexType();
|
||||
}
|
||||
|
||||
ColumnWithDictionary::ColumnWithDictionary(const ColumnWithDictionary & other)
|
||||
@ -21,4 +23,90 @@ void ColumnWithDictionary::gather(ColumnGathererStream & gatherer)
|
||||
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)
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -37,34 +37,22 @@ public:
|
||||
std::string getName() const override { return "ColumnWithDictionary"; }
|
||||
const char * getFamilyName() const override { return "ColumnWithDictionary"; }
|
||||
|
||||
ColumnPtr convertToFullColumn() const
|
||||
{
|
||||
return getUnique()->getNestedColumn()->index(*indexes, 0);
|
||||
}
|
||||
|
||||
ColumnPtr convertToFullColumn() const { return getUnique()->getNestedColumn()->index(*indexes, 0); }
|
||||
ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); }
|
||||
|
||||
MutableColumnPtr cloneResized(size_t size) const override
|
||||
{
|
||||
auto unique_ptr = column_unique;
|
||||
return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), indexes->cloneResized(size));
|
||||
}
|
||||
|
||||
MutableColumnPtr cloneResized(size_t size) const override;
|
||||
size_t size() const override { return indexes->size(); }
|
||||
|
||||
|
||||
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); }
|
||||
|
||||
StringRef getDataAt(size_t n) const override { return column_unique->getDataAt(indexes->getUInt(n)); }
|
||||
|
||||
StringRef getDataAtWithTerminatingZero(size_t n) const override
|
||||
{
|
||||
return column_unique->getDataAtWithTerminatingZero(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)); }
|
||||
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)); }
|
||||
@ -252,6 +240,13 @@ private:
|
||||
ColumnPtr column_unique;
|
||||
ColumnPtr indexes;
|
||||
|
||||
size_t getSizeOfCurrentIndexType() const;
|
||||
|
||||
template <typename IndexType>
|
||||
void convertIndexes();
|
||||
void insertIndex(size_t value);
|
||||
void insertIndexesRange(const ColumnPtr & column);
|
||||
|
||||
};
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user