Support collate in LowCardinality(String) and Nullable(String) and refactor ColumnString

This commit is contained in:
Pavel Kruglov 2020-10-22 23:23:44 +03:00
parent 11fc6fd8c9
commit 2892252b36
11 changed files with 342 additions and 208 deletions

View File

@ -1,5 +1,6 @@
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnString.h>
#include <DataStreams/ColumnGathererStream.h>
#include <DataTypes/NumberTraits.h>
#include <Common/HashTable/HashMap.h>
@ -14,6 +15,7 @@ namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int LOGICAL_ERROR;
extern const int BAD_COLLATION;
}
namespace
@ -295,14 +297,24 @@ void ColumnLowCardinality::compareColumn(const IColumn & rhs, size_t rhs_row_num
compare_results, direction, nan_direction_hint);
}
void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator) const
{
if (limit == 0)
limit = size();
size_t unique_limit = getDictionary().size();
Permutation unique_perm;
getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm);
if (collator)
{
/// Collations are supported only for ColumnString
const ColumnString * column_string = checkAndGetColumn<ColumnString>(getDictionary().getNestedColumn().get());
if (!column_string)
throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION);
column_string->getPermutationWithCollation(*collator, reverse, unique_limit, unique_perm);
}
else
getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm);
/// TODO: optimize with sse.
@ -330,7 +342,8 @@ void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_di
}
}
void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const
template <typename Cmp>
void ColumnLowCardinality::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const
{
if (equal_ranges.empty())
return;
@ -345,20 +358,17 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan
EqualRanges new_ranges;
SCOPE_EXIT({equal_ranges = std::move(new_ranges);});
auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; };
for (size_t i = 0; i < number_of_ranges; ++i)
{
const auto& [first, last] = equal_ranges[i];
if (reverse)
std::sort(res.begin() + first, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b)
{return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) > 0; });
else
std::sort(res.begin() + first, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b)
{return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) < 0; });
std::sort(res.begin() + first, res.begin() + last, less);
auto new_first = first;
for (auto j = first + 1; j < last; ++j)
{
if (compareAt(res[new_first], res[j], *this, nan_direction_hint) != 0)
if (comparator(res[new_first], res[j]) != 0)
{
if (j - new_first > 1)
new_ranges.emplace_back(new_first, j);
@ -379,17 +389,12 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan
/// Since then we are working inside the interval.
if (reverse)
std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b)
{return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) > 0; });
else
std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, [this, nan_direction_hint](size_t a, size_t b)
{return getDictionary().compareAt(getIndexes().getUInt(a), getIndexes().getUInt(b), getDictionary(), nan_direction_hint) < 0; });
std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less);
auto new_first = first;
for (auto j = first + 1; j < limit; ++j)
{
if (getDictionary().compareAt(getIndexes().getUInt(res[new_first]), getIndexes().getUInt(res[j]), getDictionary(), nan_direction_hint) != 0)
if (comparator(res[new_first],res[j]) != 0)
{
if (j - new_first > 1)
new_ranges.emplace_back(new_first, j);
@ -401,7 +406,7 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan
auto new_last = limit;
for (auto j = limit; j < last; ++j)
{
if (getDictionary().compareAt(getIndexes().getUInt(res[new_first]), getIndexes().getUInt(res[j]), getDictionary(), nan_direction_hint) == 0)
if (comparator(res[new_first], res[j]) == 0)
{
std::swap(res[new_last], res[j]);
++new_last;
@ -412,6 +417,43 @@ void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan
}
}
void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
{
getPermutationImpl(reverse, limit, nan_direction_hint, res);
}
void ColumnLowCardinality::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const
{
auto comparator = [this, nan_direction_hint, reverse](size_t lhs, size_t rhs)
{
int ret = getDictionary().compareAt(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), getDictionary(), nan_direction_hint);
return reverse ? -ret : ret;
};
updatePermutationImpl(limit, res, equal_ranges, comparator);
}
void ColumnLowCardinality::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
{
getPermutationImpl(reverse, limit, nan_direction_hint, res, &collator);
}
void ColumnLowCardinality::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const
{
/// Collations are supported only for ColumnString
const ColumnString * column_string = checkAndGetColumn<ColumnString>(getDictionary().getNestedColumn().get());
if (!column_string)
throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION);
auto comparator = [this, &column_string, &collator, reverse](size_t lhs, size_t rhs)
{
int ret = column_string->compareAtWithCollation(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), *column_string, collator);
return reverse ? -ret : ret;
};
updatePermutationImpl(limit, res, equal_ranges, comparator);
}
std::vector<MutableColumnPtr> ColumnLowCardinality::scatter(ColumnIndex num_columns, const Selector & selector) const
{
auto columns = getIndexes().scatter(num_columns, selector);

View File

@ -31,6 +31,11 @@ class ColumnLowCardinality final : public COWHelper<IColumn, ColumnLowCardinalit
ColumnLowCardinality(MutableColumnPtr && column_unique, MutableColumnPtr && indexes, bool is_shared = false);
ColumnLowCardinality(const ColumnLowCardinality & other) = default;
void getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator = nullptr) const;
template <typename Cmp>
void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const;
public:
/** Create immutable column using immutable arguments. This arguments may be shared with other columns.
* Use IColumn::mutate in order to make mutable column and mutate shared nested columns.
@ -129,6 +134,10 @@ public:
void updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_range) const override;
void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const;
void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges& equal_range) const;
ColumnPtr replicate(const Offsets & offsets) const override
{
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets));

View File

@ -6,6 +6,7 @@
#include <Common/WeakHash.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
#include <DataStreams/ColumnGathererStream.h>
@ -17,6 +18,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_COLUMN;
extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT;
extern const int BAD_COLLATION;
}
@ -256,10 +258,21 @@ void ColumnNullable::compareColumn(const IColumn & rhs, size_t rhs_row_num,
compare_results, direction, nan_direction_hint);
}
void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const
void ColumnNullable::getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator) const
{
/// Cannot pass limit because of unknown amount of NULLs.
getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res);
if (collator)
{
/// Collations are supported only for ColumnString
const ColumnString * column_string = checkAndGetColumn<ColumnString>(&getNestedColumn());
if (!column_string)
throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION);
column_string->getPermutationWithCollation(*collator, reverse, 0, res);
}
else
getNestedColumn().getPermutation(reverse, 0, null_direction_hint, res);
if ((null_direction_hint > 0) != reverse)
{
@ -329,7 +342,7 @@ void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_directi
}
}
void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const
void ColumnNullable::updatePermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator) const
{
if (equal_ranges.empty())
return;
@ -432,12 +445,42 @@ void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_dire
}
}
getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges);
if (collator)
{
/// Collations are supported only for ColumnString
const ColumnString * column_string = checkAndGetColumn<ColumnString>(getNestedColumn());
if (!column_string)
throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION);
column_string->updatePermutationWithCollation(*collator, reverse, limit, null_direction_hint, res, new_ranges);
}
else
getNestedColumn().updatePermutation(reverse, limit, null_direction_hint, res, new_ranges);
equal_ranges = std::move(new_ranges);
std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges));
}
void ColumnNullable::getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const
{
getPermutationImpl(reverse, limit, null_direction_hint, res);
}
void ColumnNullable::updatePermutation(bool reverse, size_t limit, int null_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const
{
updatePermutationImpl(reverse, limit, null_direction_hint, res, equal_ranges);
}
void ColumnNullable::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const
{
getPermutationImpl(reverse, limit, null_direction_hint, res, &collator);
}
void ColumnNullable::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const
{
updatePermutationImpl(reverse, limit, null_direction_hint, res, equal_range, &collator);
}
void ColumnNullable::gather(ColumnGathererStream & gatherer)
{
gatherer.gather(*this);

View File

@ -6,6 +6,7 @@
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
class Collator;
namespace DB
{
@ -30,6 +31,11 @@ private:
ColumnNullable(MutableColumnPtr && nested_column_, MutableColumnPtr && null_map_);
ColumnNullable(const ColumnNullable &) = default;
void getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator = nullptr) const;
void updatePermutationImpl(
bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_ranges, const Collator * collator = nullptr) const;
public:
/** Create immutable column using immutable arguments. This arguments may be shared with other columns.
* Use IColumn::mutate in order to make mutable column and mutate shared nested columns.
@ -94,6 +100,9 @@ public:
int direction, int nan_direction_hint) const override;
void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override;
void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override;
void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const;
void updatePermutationWithCollation(
const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges& equal_range) const;
void reserve(size_t n) override;
size_t byteSize() const override;
size_t allocatedBytes() const override;

View File

@ -284,22 +284,23 @@ void ColumnString::compareColumn(
compare_results, direction, nan_direction_hint);
}
template <bool positive>
struct ColumnString::less
struct ColumnString::cmp
{
const ColumnString & parent;
explicit less(const ColumnString & parent_) : parent(parent_) {}
bool operator()(size_t lhs, size_t rhs) const
bool reverse;
cmp(const ColumnString & parent_, bool reverse_=false) : parent(parent_), reverse(reverse_) {}
int operator()(size_t lhs, size_t rhs) const
{
int res = memcmpSmallAllowOverflow15(
parent.chars.data() + parent.offsetAt(lhs), parent.sizeAt(lhs) - 1,
parent.chars.data() + parent.offsetAt(rhs), parent.sizeAt(rhs) - 1);
return positive ? (res < 0) : (res > 0);
return reverse ? -res : res;
}
};
void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const
template <typename Cmp>
void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const
{
size_t s = offsets.size();
res.resize(s);
@ -309,23 +310,16 @@ void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_directio
if (limit >= s)
limit = 0;
auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; };
if (limit)
{
if (reverse)
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<false>(*this));
else
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less<true>(*this));
}
std::partial_sort(res.begin(), res.begin() + limit, res.end(), less);
else
{
if (reverse)
std::sort(res.begin(), res.end(), less<false>(*this));
else
std::sort(res.begin(), res.end(), less<true>(*this));
}
std::sort(res.begin(), res.end(), less);
}
void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const
template <typename Cmp>
void ColumnString::updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const
{
if (equal_ranges.empty())
return;
@ -340,21 +334,17 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc
if (limit)
--number_of_ranges;
auto less = [&comparator](size_t lhs, size_t rhs){ return comparator(lhs, rhs) < 0; };
for (size_t i = 0; i < number_of_ranges; ++i)
{
const auto & [first, last] = equal_ranges[i];
if (reverse)
std::sort(res.begin() + first, res.begin() + last, less<false>(*this));
else
std::sort(res.begin() + first, res.begin() + last, less<true>(*this));
std::sort(res.begin() + first, res.begin() + last, less);
size_t new_first = first;
for (size_t j = first + 1; j < last; ++j)
{
if (memcmpSmallAllowOverflow15(
chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1,
chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) != 0)
if (comparator(res[j], res[new_first]) != 0)
{
if (j - new_first > 1)
new_ranges.emplace_back(new_first, j);
@ -375,17 +365,12 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc
/// Since then we are working inside the interval.
if (reverse)
std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less<false>(*this));
else
std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less<true>(*this));
std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, less);
size_t new_first = first;
for (size_t j = first + 1; j < limit; ++j)
{
if (memcmpSmallAllowOverflow15(
chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1,
chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) != 0)
if (comparator(res[j], res[new_first]) != 0)
{
if (j - new_first > 1)
new_ranges.emplace_back(new_first, j);
@ -395,9 +380,7 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc
size_t new_last = limit;
for (size_t j = limit; j < last; ++j)
{
if (memcmpSmallAllowOverflow15(
chars.data() + offsetAt(res[j]), sizeAt(res[j]) - 1,
chars.data() + offsetAt(res[new_first]), sizeAt(res[new_first]) - 1) == 0)
if (comparator(res[j], res[new_first]) == 0)
{
std::swap(res[j], res[new_last]);
++new_last;
@ -408,6 +391,45 @@ void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direc
}
}
void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const
{
getPermutationImpl(limit, res, cmp(*this, reverse));
}
void ColumnString::updatePermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const
{
updatePermutationImpl(limit, res, equal_ranges, cmp(*this, reverse));
}
struct ColumnString::cmpWithCollation
{
const ColumnString & parent;
const Collator & collator;
bool reverse;
cmpWithCollation(const ColumnString & parent_, const Collator & collator_, bool reverse_=false) : parent(parent_), collator(collator_), reverse(reverse_) {}
int operator()(size_t lhs, size_t rhs) const
{
int res = collator.compare(
reinterpret_cast<const char *>(&parent.chars[parent.offsetAt(lhs)]), parent.sizeAt(lhs),
reinterpret_cast<const char *>(&parent.chars[parent.offsetAt(rhs)]), parent.sizeAt(rhs));
return reverse ? -res : res;
}
};
void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const
{
getPermutationImpl(limit, res, cmpWithCollation(*this, collator, reverse));
}
void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const
{
updatePermutationImpl(limit, res, equal_ranges, cmpWithCollation(*this, collator, reverse));
}
ColumnPtr ColumnString::replicate(const Offsets & replicate_offsets) const
{
size_t col_size = size();
@ -476,13 +498,13 @@ void ColumnString::getExtremes(Field & min, Field & max) const
size_t min_idx = 0;
size_t max_idx = 0;
less<true> less_op(*this);
cmp cmp_op(*this);
for (size_t i = 1; i < col_size; ++i)
{
if (less_op(i, min_idx))
if (cmp_op(i, min_idx) < 0)
min_idx = i;
else if (less_op(max_idx, i))
else if (cmp_op(max_idx, i) < 0)
max_idx = i;
}
@ -500,134 +522,6 @@ int ColumnString::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs
reinterpret_cast<const char *>(&rhs.chars[rhs.offsetAt(m)]), rhs.sizeAt(m));
}
template <bool positive>
struct ColumnString::lessWithCollation
{
const ColumnString & parent;
const Collator & collator;
lessWithCollation(const ColumnString & parent_, const Collator & collator_) : parent(parent_), collator(collator_) {}
bool operator()(size_t lhs, size_t rhs) const
{
int res = collator.compare(
reinterpret_cast<const char *>(&parent.chars[parent.offsetAt(lhs)]), parent.sizeAt(lhs),
reinterpret_cast<const char *>(&parent.chars[parent.offsetAt(rhs)]), parent.sizeAt(rhs));
return positive ? (res < 0) : (res > 0);
}
};
void ColumnString::getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const
{
size_t s = offsets.size();
res.resize(s);
for (size_t i = 0; i < s; ++i)
res[i] = i;
if (limit >= s)
limit = 0;
if (limit)
{
if (reverse)
std::partial_sort(res.begin(), res.begin() + limit, res.end(), lessWithCollation<false>(*this, collator));
else
std::partial_sort(res.begin(), res.begin() + limit, res.end(), lessWithCollation<true>(*this, collator));
}
else
{
if (reverse)
std::sort(res.begin(), res.end(), lessWithCollation<false>(*this, collator));
else
std::sort(res.begin(), res.end(), lessWithCollation<true>(*this, collator));
}
}
void ColumnString::updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const
{
if (equal_ranges.empty())
return;
if (limit >= size() || limit >= equal_ranges.back().second)
limit = 0;
size_t number_of_ranges = equal_ranges.size();
if (limit)
--number_of_ranges;
EqualRanges new_ranges;
SCOPE_EXIT({equal_ranges = std::move(new_ranges);});
for (size_t i = 0; i < number_of_ranges; ++i)
{
const auto& [first, last] = equal_ranges[i];
if (reverse)
std::sort(res.begin() + first, res.begin() + last, lessWithCollation<false>(*this, collator));
else
std::sort(res.begin() + first, res.begin() + last, lessWithCollation<true>(*this, collator));
auto new_first = first;
for (auto j = first + 1; j < last; ++j)
{
if (collator.compare(
reinterpret_cast<const char *>(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]),
reinterpret_cast<const char *>(&chars[offsetAt(res[j])]), sizeAt(res[j])) != 0)
{
if (j - new_first > 1)
new_ranges.emplace_back(new_first, j);
new_first = j;
}
}
if (last - new_first > 1)
new_ranges.emplace_back(new_first, last);
}
if (limit)
{
const auto & [first, last] = equal_ranges.back();
if (limit < first || limit > last)
return;
/// Since then we are working inside the interval.
if (reverse)
std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, lessWithCollation<false>(*this, collator));
else
std::partial_sort(res.begin() + first, res.begin() + limit, res.begin() + last, lessWithCollation<true>(*this, collator));
auto new_first = first;
for (auto j = first + 1; j < limit; ++j)
{
if (collator.compare(
reinterpret_cast<const char *>(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]),
reinterpret_cast<const char *>(&chars[offsetAt(res[j])]), sizeAt(res[j])) != 0)
{
if (j - new_first > 1)
new_ranges.emplace_back(new_first, j);
new_first = j;
}
}
auto new_last = limit;
for (auto j = limit; j < last; ++j)
{
if (collator.compare(
reinterpret_cast<const char *>(&chars[offsetAt(res[new_first])]), sizeAt(res[new_first]),
reinterpret_cast<const char *>(&chars[offsetAt(res[j])]), sizeAt(res[j])) == 0)
{
std::swap(res[new_last], res[j]);
++new_last;
}
}
if (new_last - new_first > 1)
new_ranges.emplace_back(new_first, new_last);
}
}
void ColumnString::protect()
{
getChars().protect();

View File

@ -42,15 +42,19 @@ private:
/// Size of i-th element, including terminating zero.
size_t ALWAYS_INLINE sizeAt(ssize_t i) const { return offsets[i] - offsets[i - 1]; }
template <bool positive>
struct less;
struct cmp;
template <bool positive>
struct lessWithCollation;
struct cmpWithCollation;
ColumnString() = default;
ColumnString(const ColumnString & src);
template <typename Cmp>
void getPermutationImpl(size_t limit, Permutation & res, Cmp comparator) const;
template <typename Cmp>
void updatePermutationImpl(size_t limit, Permutation & res, EqualRanges & equal_ranges, Cmp comparator) const;
public:
const char * getFamilyName() const override { return "String"; }
TypeIndex getDataType() const override { return TypeIndex::String; }
@ -233,12 +237,12 @@ public:
void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override;
void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override;
void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const override;
/// Sorting with respect of collation.
void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, Permutation & res) const;
void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges& equal_range) const;
void updatePermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const;
ColumnPtr replicate(const Offsets & replicate_offsets) const override;

View File

@ -2,6 +2,8 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnLowCardinality.h>
#include <Common/typeid_cast.h>
#include <Functions/FunctionHelpers.h>
@ -20,6 +22,24 @@ static bool isCollationRequired(const SortColumnDescription & description)
return description.collator != nullptr;
}
static bool isCollationSupported(const IColumn * column)
{
if (column->getDataType() == TypeIndex::String)
return true;
if (column->getDataType() == TypeIndex::Nullable)
{
const ColumnNullable * column_nullable = assert_cast<const ColumnNullable *>(column);
return isCollationSupported(&column_nullable->getNestedColumn());
}
if (column->getDataType() == TypeIndex::LowCardinality)
{
const ColumnLowCardinality * column_low_cardinality = assert_cast<const ColumnLowCardinality *>(column);
return isCollationSupported(column_low_cardinality->getDictionary().getNestedColumn().get());
}
return false;
}
ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description)
{
@ -101,7 +121,6 @@ struct PartialSortingLessWithCollation
}
};
void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
{
if (!block)
@ -120,14 +139,18 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
bool is_column_const = false;
if (isCollationRequired(description[0]))
{
/// it it's real string column, than we need sort
/// Check if column supports collations
if (!isCollationSupported(column))
throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION);
if (const ColumnString * column_string = checkAndGetColumn<ColumnString>(column))
column_string->getPermutationWithCollation(*description[0].collator, reverse, limit, perm);
else if (checkAndGetColumnConstData<ColumnString>(column))
else if (const ColumnNullable * column_nullable = checkAndGetColumn<ColumnNullable>(column))
column_nullable->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm);
else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn<ColumnLowCardinality>(column))
column_low_cardinality->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm);
else if (isColumnConst(*column))
is_column_const = true;
else
throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION);
}
else if (!isColumnConst(*column))
{
@ -163,8 +186,8 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
const IColumn * column = columns_with_sort_desc[i].column;
if (isCollationRequired(description[i]))
{
if (!checkAndGetColumn<ColumnString>(column) && !checkAndGetColumnConstData<ColumnString>(column))
throw Exception("Collations could be specified only for String columns.", ErrorCodes::BAD_COLLATION);
if (!isCollationSupported(column))
throw Exception("Collations could be specified only for String columns or columns where nested column is String.", ErrorCodes::BAD_COLLATION);
need_collation = true;
}
@ -187,10 +210,20 @@ void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
if (isCollationRequired(column.description))
{
const ColumnString & column_string = assert_cast<const ColumnString &>(*column.column);
column_string.updatePermutationWithCollation(
*column.description.collator,
column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges);
if (const ColumnString * column_string = checkAndGetColumn<ColumnString>(column.column))
column_string->updatePermutationWithCollation(
*column.description.collator,
column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges);
else if (const ColumnNullable * column_nullable = checkAndGetColumn<ColumnNullable>(column.column))
column_nullable->updatePermutationWithCollation(
*column.description.collator,
column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges);
else if (const ColumnLowCardinality * column_low_cardinality = checkAndGetColumn<ColumnLowCardinality>(column.column))
column_low_cardinality->updatePermutationWithCollation(
*column.description.collator,
column.description.direction < 0, limit, column.description.nulls_direction, perm, ranges);
}
else
{

View File

@ -0,0 +1,28 @@
Order by without collate
1 Ё
2 А
2 Я
1 а
2 я
1 ё
Order by with collate
1 а
2 А
1 ё
1 Ё
2 я
2 Я
Order by tuple without collate
1 Ё
1 а
1 ё
2 А
2 Я
2 я
Order by tuple with collate
1 а
1 ё
1 Ё
2 А
2 я
2 Я

View File

@ -0,0 +1,18 @@
DROP TABLE IF EXISTS test_collate;
CREATE TABLE test_collate (x UInt32, s LowCardinality(String)) ENGINE=Memory();
INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (2, 'А'), (2, 'я'), (2, 'Я');
SELECT 'Order by without collate';
SELECT * FROM test_collate ORDER BY s;
SELECT 'Order by with collate';
SELECT * FROM test_collate ORDER BY s COLLATE 'ru';
SELECT 'Order by tuple without collate';
SELECT * FROM test_collate ORDER BY x, s;
SELECT 'Order by tuple with collate';
SELECT * FROM test_collate ORDER BY x, s COLLATE 'ru';
DROP TABLE test_collate;

View File

@ -0,0 +1,36 @@
Order by without collate
1 Ё
2 А
2 Я
1 а
2 я
1 ё
1 \N
2 \N
Order by with collate
1 а
2 А
1 ё
1 Ё
2 я
2 Я
1 \N
2 \N
Order by tuple without collate
1 Ё
1 а
1 ё
1 \N
2 А
2 Я
2 я
2 \N
Order by tuple with collate
1 а
1 ё
1 Ё
1 \N
2 А
2 я
2 Я
2 \N

View File

@ -0,0 +1,18 @@
DROP TABLE IF EXISTS test_collate;
CREATE TABLE test_collate (x UInt32, s Nullable(String)) ENGINE=Memory();
INSERT INTO test_collate VALUES (1, 'Ё'), (1, 'ё'), (1, 'а'), (1, null), (2, 'А'), (2, 'я'), (2, 'Я'), (2, null);
SELECT 'Order by without collate';
SELECT * FROM test_collate ORDER BY s;
SELECT 'Order by with collate';
SELECT * FROM test_collate ORDER BY s COLLATE 'ru';
SELECT 'Order by tuple without collate';
SELECT * FROM test_collate ORDER BY x, s;
SELECT 'Order by tuple with collate';
SELECT * FROM test_collate ORDER BY x, s COLLATE 'ru';
DROP TABLE test_collate;