2018-03-29 14:04:24 +00:00
|
|
|
#pragma once
|
2018-03-14 10:47:13 +00:00
|
|
|
#include <Columns/IColumnUnique.h>
|
|
|
|
#include <Common/HashTable/HashMap.h>
|
2018-03-19 11:45:17 +00:00
|
|
|
#include <ext/range.h>
|
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
#include <Columns/ColumnVector.h>
|
|
|
|
#include <Columns/ColumnNullable.h>
|
2018-03-29 14:04:24 +00:00
|
|
|
#include <Columns/ColumnString.h>
|
2018-04-17 11:28:35 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2018-06-28 18:12:29 +00:00
|
|
|
#include <DataTypes/NumberTraits.h>
|
2018-03-14 10:47:13 +00:00
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
class NullMap;
|
2018-03-29 14:04:24 +00:00
|
|
|
|
|
|
|
|
|
|
|
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;
|
2018-05-21 16:21:15 +00:00
|
|
|
StringRefWrapper & operator =(int) { column = nullptr; ref.data = nullptr; return *this; }
|
|
|
|
bool operator ==(int) const { return nullptr == column && nullptr == ref.data; }
|
2018-03-29 14:04:24 +00:00
|
|
|
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>
|
2018-07-09 18:19:03 +00:00
|
|
|
bool check(const StringRefWrapper<ColumnType> x) { return nullptr == x.column && nullptr == x.ref.data; }
|
2018-03-29 14:04:24 +00:00
|
|
|
|
|
|
|
template <typename ColumnType>
|
2018-07-09 18:19:03 +00:00
|
|
|
void set(StringRefWrapper<ColumnType> & x) { x.column = nullptr; x.ref.data = nullptr; }
|
2018-03-29 14:04:24 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2018-03-14 10:47:13 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
class ColumnUnique final : public COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>>
|
2018-03-14 10:47:13 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
friend class COWPtrHelper<IColumnUnique, ColumnUnique<ColumnType>>;
|
2018-03-19 11:45:17 +00:00
|
|
|
|
|
|
|
private:
|
2018-06-26 17:07:08 +00:00
|
|
|
explicit ColumnUnique(MutableColumnPtr && holder, bool is_nullable);
|
2018-06-07 18:14:37 +00:00
|
|
|
explicit ColumnUnique(const IDataType & type);
|
2018-06-26 17:07:08 +00:00
|
|
|
ColumnUnique(const ColumnUnique & other) : column_holder(other.column_holder), is_nullable(other.is_nullable) {}
|
2018-03-19 11:45:17 +00:00
|
|
|
|
2018-03-14 10:47:13 +00:00
|
|
|
public:
|
2018-07-09 18:19:03 +00:00
|
|
|
MutableColumnPtr cloneEmpty() const override;
|
|
|
|
|
|
|
|
const ColumnPtr & getNestedColumn() const override;
|
2018-06-26 17:07:08 +00:00
|
|
|
const ColumnPtr & getNestedNotNullableColumn() const override { return column_holder; }
|
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
size_t uniqueInsert(const Field & x) override;
|
|
|
|
size_t uniqueInsertFrom(const IColumn & src, size_t n) override;
|
2018-06-07 18:14:37 +00:00
|
|
|
MutableColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) override;
|
|
|
|
IColumnUnique::IndexesWithOverflow uniqueInsertRangeWithOverflow(const IColumn & src, size_t start, size_t length,
|
|
|
|
size_t max_dictionary_size) override;
|
2018-03-19 11:45:17 +00:00
|
|
|
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;
|
2018-03-14 10:47:13 +00:00
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
size_t getDefaultValueIndex() const override { return is_nullable ? 1 : 0; }
|
|
|
|
size_t getNullValueIndex() const override;
|
|
|
|
bool canContainNulls() const override { return is_nullable; }
|
|
|
|
|
2018-06-06 11:20:27 +00:00
|
|
|
Field operator[](size_t n) const override { return (*getNestedColumn())[n]; }
|
|
|
|
void get(size_t n, Field & res) const override { getNestedColumn()->get(n, res); }
|
|
|
|
StringRef getDataAt(size_t n) const override { return getNestedColumn()->getDataAt(n); }
|
2018-04-17 10:45:05 +00:00
|
|
|
StringRef getDataAtWithTerminatingZero(size_t n) const override
|
|
|
|
{
|
2018-06-06 11:20:27 +00:00
|
|
|
return getNestedColumn()->getDataAtWithTerminatingZero(n);
|
2018-04-17 10:45:05 +00:00
|
|
|
}
|
2018-06-06 11:20:27 +00:00
|
|
|
UInt64 get64(size_t n) const override { return getNestedColumn()->get64(n); }
|
|
|
|
UInt64 getUInt(size_t n) const override { return getNestedColumn()->getUInt(n); }
|
|
|
|
Int64 getInt(size_t n) const override { return getNestedColumn()->getInt(n); }
|
2018-06-06 13:43:16 +00:00
|
|
|
bool isNullAt(size_t n) const override { return is_nullable && n == getNullValueIndex(); }
|
2018-04-17 10:45:05 +00:00
|
|
|
StringRef serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const override
|
|
|
|
{
|
|
|
|
return column_holder->serializeValueIntoArena(n, arena, begin);
|
|
|
|
}
|
|
|
|
void updateHashWithValue(size_t n, SipHash & hash) const override
|
|
|
|
{
|
2018-06-06 11:20:27 +00:00
|
|
|
return getNestedColumn()->updateHashWithValue(n, hash);
|
2018-04-17 10:45:05 +00:00
|
|
|
}
|
2018-06-06 11:20:27 +00:00
|
|
|
|
2018-04-17 10:45:05 +00:00
|
|
|
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override
|
|
|
|
{
|
2018-06-06 11:20:27 +00:00
|
|
|
auto & column_unique = static_cast<const IColumnUnique&>(rhs);
|
|
|
|
return getNestedColumn()->compareAt(n, m, *column_unique.getNestedColumn(), nan_direction_hint);
|
2018-03-29 14:04:24 +00:00
|
|
|
}
|
2018-06-06 11:20:27 +00:00
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); }
|
|
|
|
bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); }
|
|
|
|
bool isFixedAndContiguous() const override { return column_holder->isFixedAndContiguous(); }
|
|
|
|
size_t sizeOfValueIfFixed() const override { return column_holder->sizeOfValueIfFixed(); }
|
|
|
|
bool isNumeric() const override { return column_holder->isNumeric(); }
|
|
|
|
|
|
|
|
size_t byteSize() const override { return column_holder->byteSize(); }
|
2018-04-17 10:45:05 +00:00
|
|
|
size_t allocatedBytes() const override
|
|
|
|
{
|
2018-06-06 11:20:27 +00:00
|
|
|
return column_holder->allocatedBytes()
|
|
|
|
+ (index ? index->getBufferSizeInBytes() : 0)
|
2018-06-26 17:07:08 +00:00
|
|
|
+ (cached_null_mask ? cached_null_mask->allocatedBytes() : 0);
|
2018-06-06 11:20:27 +00:00
|
|
|
}
|
|
|
|
void forEachSubcolumn(IColumn::ColumnCallback callback) override
|
|
|
|
{
|
2018-06-26 17:07:08 +00:00
|
|
|
callback(column_holder);
|
2018-07-09 18:19:03 +00:00
|
|
|
index = nullptr;
|
2018-04-17 10:45:05 +00:00
|
|
|
}
|
2018-03-14 10:47:13 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
using IndexMapType = HashMap<StringRefWrapper<ColumnType>, UInt64, StringRefHash>;
|
2018-03-14 10:47:13 +00:00
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
ColumnPtr column_holder;
|
2018-04-17 12:41:16 +00:00
|
|
|
|
2018-06-26 17:07:08 +00:00
|
|
|
/// For DataTypeNullable, stores null map.
|
|
|
|
mutable ColumnPtr cached_null_mask;
|
2018-07-09 18:19:03 +00:00
|
|
|
mutable ColumnPtr cached_column_nullable;
|
2018-04-17 12:41:16 +00:00
|
|
|
|
2018-03-14 10:47:13 +00:00
|
|
|
/// Lazy initialized.
|
2018-03-19 11:45:17 +00:00
|
|
|
std::unique_ptr<IndexMapType> index;
|
|
|
|
|
|
|
|
bool is_nullable;
|
2018-03-14 10:47:13 +00:00
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
size_t numSpecialValues() const { return is_nullable ? 2 : 1; }
|
|
|
|
|
|
|
|
void buildIndex();
|
2018-03-29 14:04:24 +00:00
|
|
|
ColumnType * getRawColumnPtr() { return static_cast<ColumnType *>(column_holder->assumeMutable().get()); }
|
2018-06-26 17:07:08 +00:00
|
|
|
const ColumnType * getRawColumnPtr() const { return static_cast<const ColumnType *>(column_holder.get()); }
|
2018-06-28 18:12:29 +00:00
|
|
|
UInt64 insertIntoMap(const StringRefWrapper<ColumnType> & ref, UInt64 value);
|
2018-03-14 10:47:13 +00:00
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename IndexType>
|
|
|
|
MutableColumnPtr uniqueInsertRangeImpl(
|
2018-06-07 18:14:37 +00:00
|
|
|
const IColumn & src,
|
|
|
|
size_t start,
|
|
|
|
size_t length,
|
2018-07-09 18:19:03 +00:00
|
|
|
size_t num_added_rows,
|
2018-06-28 18:12:29 +00:00
|
|
|
typename ColumnVector<IndexType>::MutablePtr && positions_column,
|
2018-06-07 18:14:37 +00:00
|
|
|
ColumnType * overflowed_keys,
|
2018-07-09 18:19:03 +00:00
|
|
|
IndexMapType * secondary_index,
|
2018-06-07 18:14:37 +00:00
|
|
|
size_t max_dictionary_size);
|
2018-03-14 10:47:13 +00:00
|
|
|
};
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
MutableColumnPtr ColumnUnique<ColumnType>::cloneEmpty() const
|
|
|
|
{
|
|
|
|
return ColumnUnique<ColumnType>::create(column_holder->cloneResized(numSpecialValues()), is_nullable);
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
ColumnUnique<ColumnType>::ColumnUnique(const IDataType & type) : is_nullable(type.isNullable())
|
2018-04-17 12:41:16 +00:00
|
|
|
{
|
2018-06-26 17:07:08 +00:00
|
|
|
const auto & holder_type = is_nullable ? *static_cast<const DataTypeNullable &>(type).getNestedType() : type;
|
|
|
|
column_holder = holder_type.createColumn()->cloneResized(numSpecialValues());
|
2018-04-17 12:41:16 +00:00
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
ColumnUnique<ColumnType>::ColumnUnique(MutableColumnPtr && holder, bool is_nullable)
|
2018-06-26 17:07:08 +00:00
|
|
|
: column_holder(std::move(holder)), is_nullable(is_nullable)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
2018-06-26 17:07:08 +00:00
|
|
|
if (column_holder->size() < numSpecialValues())
|
|
|
|
throw Exception("Too small holder column for ColumnUnique.", ErrorCodes::ILLEGAL_COLUMN);
|
2018-03-19 11:45:17 +00:00
|
|
|
if (column_holder->isColumnNullable())
|
2018-06-26 17:07:08 +00:00
|
|
|
throw Exception("Holder column for ColumnUnique can't be nullable.", ErrorCodes::ILLEGAL_COLUMN);
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
2018-07-09 18:19:03 +00:00
|
|
|
const ColumnPtr & ColumnUnique<ColumnType>::getNestedColumn() const
|
2018-04-17 12:41:16 +00:00
|
|
|
{
|
|
|
|
if (is_nullable)
|
|
|
|
{
|
2018-06-26 17:07:08 +00:00
|
|
|
size_t size = getRawColumnPtr()->size();
|
|
|
|
if (!cached_null_mask)
|
|
|
|
{
|
|
|
|
ColumnUInt8::MutablePtr null_mask = ColumnUInt8::create(size, UInt8(0));
|
|
|
|
null_mask->getData()[getNullValueIndex()] = 1;
|
|
|
|
cached_null_mask = std::move(null_mask);
|
2018-07-09 18:19:03 +00:00
|
|
|
cached_column_nullable = ColumnNullable::create(column_holder, cached_null_mask);
|
2018-06-26 17:07:08 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if (cached_null_mask->size() != size)
|
|
|
|
{
|
|
|
|
MutableColumnPtr null_mask = (*std::move(cached_null_mask)).mutate();
|
|
|
|
static_cast<ColumnUInt8 &>(*null_mask).getData().resize_fill(size);
|
|
|
|
cached_null_mask = std::move(null_mask);
|
2018-07-09 18:19:03 +00:00
|
|
|
cached_column_nullable = ColumnNullable::create(column_holder, cached_null_mask);
|
2018-06-26 17:07:08 +00:00
|
|
|
}
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
return cached_column_nullable;
|
2018-04-17 12:41:16 +00:00
|
|
|
}
|
|
|
|
return column_holder;
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
size_t ColumnUnique<ColumnType>::getNullValueIndex() const
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
|
|
|
if (!is_nullable)
|
2018-06-26 17:07:08 +00:00
|
|
|
throw Exception("ColumnUnique can't contain null values.", ErrorCodes::LOGICAL_ERROR);
|
2018-03-19 11:45:17 +00:00
|
|
|
|
|
|
|
return 0;
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
void ColumnUnique<ColumnType>::buildIndex()
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
|
|
|
if (index)
|
|
|
|
return;
|
|
|
|
|
|
|
|
auto column = getRawColumnPtr();
|
|
|
|
index = std::make_unique<IndexMapType>();
|
|
|
|
|
|
|
|
for (auto row : ext::range(numSpecialValues(), column->size()))
|
|
|
|
{
|
2018-03-29 14:04:24 +00:00
|
|
|
(*index)[StringRefWrapper<ColumnType>(column, row)] = row;
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
2018-07-09 18:19:03 +00:00
|
|
|
UInt64 ColumnUnique<ColumnType>::insertIntoMap(const StringRefWrapper<ColumnType> & ref, UInt64 value)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
|
|
|
if (!index)
|
|
|
|
buildIndex();
|
|
|
|
|
2018-03-29 14:04:24 +00:00
|
|
|
using IteratorType = typename IndexMapType::iterator;
|
|
|
|
IteratorType it;
|
2018-03-19 11:45:17 +00:00
|
|
|
bool inserted;
|
|
|
|
index->emplace(ref, it, inserted);
|
|
|
|
|
|
|
|
if (inserted)
|
|
|
|
it->second = value;
|
|
|
|
|
|
|
|
return it->second;
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
size_t ColumnUnique<ColumnType>::uniqueInsert(const Field & x)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
|
|
|
if (x.getType() == Field::Types::Null)
|
|
|
|
return getNullValueIndex();
|
|
|
|
|
|
|
|
auto column = getRawColumnPtr();
|
2018-07-09 18:19:03 +00:00
|
|
|
auto prev_size = static_cast<UInt64>(column->size());
|
2018-03-19 11:45:17 +00:00
|
|
|
|
|
|
|
if ((*column)[getDefaultValueIndex()] == x)
|
|
|
|
return getDefaultValueIndex();
|
|
|
|
|
|
|
|
column->insert(x);
|
2018-05-21 12:29:52 +00:00
|
|
|
auto pos = insertIntoMap(StringRefWrapper<ColumnType>(column, prev_size), prev_size);
|
2018-03-19 11:45:17 +00:00
|
|
|
if (pos != prev_size)
|
|
|
|
column->popBack(1);
|
|
|
|
|
2018-05-21 16:21:15 +00:00
|
|
|
return pos;
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
size_t ColumnUnique<ColumnType>::uniqueInsertFrom(const IColumn & src, size_t n)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
2018-05-03 16:28:14 +00:00
|
|
|
if (is_nullable && src.isNullAt(n))
|
|
|
|
return getNullValueIndex();
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
if (auto * nullable = typeid_cast<const ColumnNullable *>(&src))
|
|
|
|
return uniqueInsertFrom(nullable->getNestedColumn(), n);
|
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
auto ref = src.getDataAt(n);
|
|
|
|
return uniqueInsertData(ref.data, ref.size);
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
size_t ColumnUnique<ColumnType>::uniqueInsertData(const char * pos, size_t length)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
2018-05-21 16:21:15 +00:00
|
|
|
if (!index)
|
|
|
|
buildIndex();
|
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
auto column = getRawColumnPtr();
|
|
|
|
|
2018-03-29 14:04:24 +00:00
|
|
|
if (column->getDataAt(getDefaultValueIndex()) == StringRef(pos, length))
|
2018-03-19 11:45:17 +00:00
|
|
|
return getDefaultValueIndex();
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
UInt64 size = column->size();
|
2018-05-21 16:21:15 +00:00
|
|
|
auto iter = index->find(StringRefWrapper<ColumnType>(StringRef(pos, length)));
|
2018-03-19 11:45:17 +00:00
|
|
|
|
2018-05-21 16:21:15 +00:00
|
|
|
if (iter == index->end())
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
2018-03-29 14:04:24 +00:00
|
|
|
column->insertData(pos, length);
|
2018-05-21 16:21:15 +00:00
|
|
|
return insertIntoMap(StringRefWrapper<ColumnType>(column, size), size);
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
|
2018-05-21 16:21:15 +00:00
|
|
|
return iter->second;
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
size_t ColumnUnique<ColumnType>::uniqueInsertDataWithTerminatingZero(const char * pos, size_t length)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
|
|
|
if (std::is_same<ColumnType, ColumnString>::value)
|
2018-03-29 14:04:24 +00:00
|
|
|
return uniqueInsertData(pos, length - 1);
|
2018-03-19 11:45:17 +00:00
|
|
|
|
|
|
|
if (column_holder->valuesHaveFixedSize())
|
2018-03-29 14:04:24 +00:00
|
|
|
return uniqueInsertData(pos, length);
|
2018-03-19 11:45:17 +00:00
|
|
|
|
|
|
|
/// Don't know if data actually has terminating zero. So, insert it firstly.
|
|
|
|
|
|
|
|
auto column = getRawColumnPtr();
|
|
|
|
size_t prev_size = column->size();
|
2018-03-29 14:04:24 +00:00
|
|
|
column->insertDataWithTerminatingZero(pos, length);
|
2018-03-19 11:45:17 +00:00
|
|
|
|
|
|
|
if (column->compareAt(getDefaultValueIndex(), prev_size, *column, 1) == 0)
|
|
|
|
{
|
|
|
|
column->popBack(1);
|
|
|
|
return getDefaultValueIndex();
|
|
|
|
}
|
|
|
|
|
2018-05-21 12:29:52 +00:00
|
|
|
auto position = insertIntoMap(StringRefWrapper<ColumnType>(column, prev_size), prev_size);
|
2018-03-29 14:04:24 +00:00
|
|
|
if (position != prev_size)
|
2018-03-19 11:45:17 +00:00
|
|
|
column->popBack(1);
|
|
|
|
|
2018-03-29 14:04:24 +00:00
|
|
|
return static_cast<size_t>(position);
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
size_t ColumnUnique<ColumnType>::uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
|
|
|
auto column = getRawColumnPtr();
|
|
|
|
size_t prev_size = column->size();
|
|
|
|
new_pos = column->deserializeAndInsertFromArena(pos);
|
|
|
|
|
|
|
|
if (column->compareAt(getDefaultValueIndex(), prev_size, *column, 1) == 0)
|
|
|
|
{
|
|
|
|
column->popBack(1);
|
|
|
|
return getDefaultValueIndex();
|
|
|
|
}
|
|
|
|
|
2018-05-21 12:29:52 +00:00
|
|
|
auto index_pos = insertIntoMap(StringRefWrapper<ColumnType>(column, prev_size), prev_size);
|
2018-03-19 11:45:17 +00:00
|
|
|
if (index_pos != prev_size)
|
|
|
|
column->popBack(1);
|
|
|
|
|
|
|
|
return static_cast<size_t>(index_pos);
|
|
|
|
}
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
template <typename IndexType>
|
|
|
|
static void checkIndexes(const ColumnVector<IndexType> & indexes, size_t max_dictionary_size)
|
|
|
|
{
|
|
|
|
auto & data = indexes.getData();
|
|
|
|
for (size_t i = 0; i < data.size(); ++i)
|
|
|
|
{
|
|
|
|
if (data[i] >= max_dictionary_size)
|
|
|
|
{
|
|
|
|
throw Exception("Found index " + toString(data[i]) + " at position " + toString(i)
|
|
|
|
+ " which is grated or equal than dictionary size " + toString(max_dictionary_size),
|
|
|
|
ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
template <typename IndexType>
|
|
|
|
MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeImpl(
|
2018-06-07 18:14:37 +00:00
|
|
|
const IColumn & src,
|
|
|
|
size_t start,
|
|
|
|
size_t length,
|
2018-07-09 18:19:03 +00:00
|
|
|
size_t num_added_rows,
|
2018-06-28 18:12:29 +00:00
|
|
|
typename ColumnVector<IndexType>::MutablePtr && positions_column,
|
2018-06-07 18:14:37 +00:00
|
|
|
ColumnType * overflowed_keys,
|
2018-07-09 18:19:03 +00:00
|
|
|
IndexMapType * secondary_index,
|
2018-06-07 18:14:37 +00:00
|
|
|
size_t max_dictionary_size)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
|
|
|
if (!index)
|
|
|
|
buildIndex();
|
|
|
|
|
|
|
|
const ColumnType * src_column;
|
|
|
|
const NullMap * null_map = nullptr;
|
2018-06-28 18:12:29 +00:00
|
|
|
auto & positions = positions_column->getData();
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
auto updatePosition = [&](UInt64 & next_position) -> MutableColumnPtr
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
constexpr auto next_size = NumberTraits::nextSize(sizeof(IndexType));
|
|
|
|
using SuperiorIndexType = typename NumberTraits::Construct<false, false, next_size>::Type;
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
++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);
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
auto expanded_column = ColumnVector<SuperiorIndexType>::create(length);
|
2018-06-28 18:12:29 +00:00
|
|
|
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,
|
2018-07-09 18:19:03 +00:00
|
|
|
start,
|
|
|
|
length,
|
|
|
|
num_added_rows,
|
2018-06-28 18:12:29 +00:00
|
|
|
std::move(expanded_column),
|
|
|
|
overflowed_keys,
|
2018-07-09 18:19:03 +00:00
|
|
|
secondary_index,
|
2018-06-28 18:12:29 +00:00
|
|
|
max_dictionary_size);
|
|
|
|
}
|
|
|
|
|
|
|
|
return nullptr;
|
|
|
|
};
|
2018-03-19 11:45:17 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
if (auto nullable_column = typeid_cast<const ColumnNullable *>(&src))
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
src_column = typeid_cast<const ColumnType *>(&nullable_column->getNestedColumn());
|
2018-03-19 11:45:17 +00:00
|
|
|
null_map = &nullable_column->getNullMapData();
|
|
|
|
}
|
|
|
|
else
|
2018-07-09 18:19:03 +00:00
|
|
|
src_column = typeid_cast<const ColumnType *>(&src);
|
2018-03-19 11:45:17 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
if (src_column == nullptr)
|
|
|
|
throw Exception("Invalid column type for ColumnUnique::insertRangeFrom. Expected " + column_holder->getName() +
|
|
|
|
", got " + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
2018-06-07 18:14:37 +00:00
|
|
|
|
2018-03-19 11:45:17 +00:00
|
|
|
auto column = getRawColumnPtr();
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
UInt64 next_position = column->size();
|
2018-07-09 18:19:03 +00:00
|
|
|
if (secondary_index)
|
|
|
|
next_position += secondary_index->size();
|
|
|
|
|
|
|
|
for (; num_added_rows < length; ++num_added_rows)
|
2018-03-19 11:45:17 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
auto row = start + num_added_rows;
|
2018-03-19 11:45:17 +00:00
|
|
|
|
2018-04-17 12:41:16 +00:00
|
|
|
if (null_map && (*null_map)[row])
|
2018-07-09 18:19:03 +00:00
|
|
|
positions[num_added_rows] = getNullValueIndex();
|
2018-04-17 12:41:16 +00:00
|
|
|
else if (column->compareAt(getDefaultValueIndex(), row, *src_column, 1) == 0)
|
2018-07-09 18:19:03 +00:00
|
|
|
positions[num_added_rows] = getDefaultValueIndex();
|
2018-03-19 11:45:17 +00:00
|
|
|
else
|
|
|
|
{
|
2018-03-29 14:04:24 +00:00
|
|
|
auto it = index->find(StringRefWrapper<ColumnType>(src_column, row));
|
2018-03-19 11:45:17 +00:00
|
|
|
if (it == index->end())
|
|
|
|
{
|
2018-06-07 18:14:37 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
if (overflowed_keys && next_position >= max_dictionary_size)
|
2018-06-07 18:14:37 +00:00
|
|
|
{
|
|
|
|
auto jt = secondary_index->find(StringRefWrapper<ColumnType>(src_column, row));
|
|
|
|
if (jt == secondary_index->end())
|
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
positions[num_added_rows] = next_position;
|
2018-06-07 18:14:37 +00:00
|
|
|
auto ref = src_column->getDataAt(row);
|
|
|
|
overflowed_keys->insertData(ref.data, ref.size);
|
|
|
|
(*secondary_index)[StringRefWrapper<ColumnType>(src_column, row)] = next_position;
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
if (auto res = updatePosition(next_position))
|
2018-06-28 18:12:29 +00:00
|
|
|
return res;
|
2018-06-07 18:14:37 +00:00
|
|
|
}
|
|
|
|
else
|
2018-07-09 18:19:03 +00:00
|
|
|
positions[num_added_rows] = jt->second;
|
2018-06-07 18:14:37 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
positions[num_added_rows] = next_position;
|
2018-06-07 18:14:37 +00:00
|
|
|
auto ref = src_column->getDataAt(row);
|
|
|
|
column->insertData(ref.data, ref.size);
|
|
|
|
(*index)[StringRefWrapper<ColumnType>(column, next_position)] = next_position;
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
if (auto res = updatePosition(next_position))
|
2018-06-28 18:12:29 +00:00
|
|
|
return res;
|
2018-06-07 18:14:37 +00:00
|
|
|
}
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
else
|
2018-07-09 18:19:03 +00:00
|
|
|
positions[num_added_rows] = it->second;
|
2018-03-19 11:45:17 +00:00
|
|
|
}
|
|
|
|
}
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
checkIndexes(*positions_column, column->size() + (overflowed_keys ? overflowed_keys->size() : 0));
|
|
|
|
|
|
|
|
return std::move(positions_column);
|
2018-06-07 18:14:37 +00:00
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
MutableColumnPtr ColumnUnique<ColumnType>::uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length)
|
2018-06-07 18:14:37 +00:00
|
|
|
{
|
2018-06-28 18:12:29 +00:00
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
auto callForType = [this, &src, start, length](auto x) -> MutableColumnPtr
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
size_t size = getRawColumnPtr()->size();
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
using IndexType = decltype(x);
|
|
|
|
if (size <= std::numeric_limits<IndexType>::max())
|
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
auto positions = ColumnVector<IndexType>::create(length);
|
|
|
|
return this->uniqueInsertRangeImpl<IndexType>(src, start, length, 0,
|
|
|
|
std::move(positions), nullptr, nullptr, 0);
|
2018-06-28 18:12:29 +00:00
|
|
|
}
|
2018-06-07 18:14:37 +00:00
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
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);
|
2018-03-19 11:45:17 +00:00
|
|
|
|
|
|
|
return positions_column;
|
|
|
|
}
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
template <typename ColumnType>
|
|
|
|
IColumnUnique::IndexesWithOverflow ColumnUnique<ColumnType>::uniqueInsertRangeWithOverflow(
|
2018-06-07 18:14:37 +00:00
|
|
|
const IColumn & src,
|
|
|
|
size_t start,
|
|
|
|
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)
|
|
|
|
throw Exception("Invalid keys type for ColumnUnique.", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
|
2018-07-09 18:19:03 +00:00
|
|
|
auto callForType = [this, &src, start, length, overflowed_keys_ptr, max_dictionary_size](auto x) -> MutableColumnPtr
|
2018-06-28 18:12:29 +00:00
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
size_t size = getRawColumnPtr()->size();
|
|
|
|
|
2018-06-28 18:12:29 +00:00
|
|
|
using IndexType = decltype(x);
|
|
|
|
if (size <= std::numeric_limits<IndexType>::max())
|
|
|
|
{
|
2018-07-09 18:19:03 +00:00
|
|
|
auto positions = ColumnVector<IndexType>::create(length);
|
|
|
|
IndexMapType secondary_index;
|
|
|
|
return this->uniqueInsertRangeImpl<IndexType>(src, start, length, 0, std::move(positions),
|
|
|
|
overflowed_keys_ptr, &secondary_index, max_dictionary_size);
|
2018-06-28 18:12:29 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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);
|
2018-06-07 18:14:37 +00:00
|
|
|
|
|
|
|
IColumnUnique::IndexesWithOverflow indexes_with_overflow;
|
|
|
|
indexes_with_overflow.indexes = std::move(positions_column);
|
|
|
|
indexes_with_overflow.overflowed_keys = std::move(overflowed_keys);
|
|
|
|
return indexes_with_overflow;
|
|
|
|
}
|
|
|
|
|
|
|
|
};
|