mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Updated tests
This commit is contained in:
parent
e5b85953e8
commit
f1d2f2a9e1
@ -805,7 +805,7 @@ void ColumnArray::getPermutation(PermutationSortDirection direction, Permutation
|
||||
getPermutationImpl(limit, res, ComparatorAscendingStable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort());
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
|
||||
getPermutationImpl(limit, res, ComparatorDescendingUnstable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort());
|
||||
else
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
|
||||
getPermutationImpl(limit, res, ComparatorDescendingStable(*this, nan_direction_hint), DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
@ -820,7 +820,7 @@ void ColumnArray::updatePermutation(PermutationSortDirection direction, Permutat
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorAscendingStable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingUnstable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
else
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingStable(*this, nan_direction_hint), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
@ -833,7 +833,7 @@ void ColumnArray::getPermutationWithCollation(const Collator & collator, Permuta
|
||||
getPermutationImpl(limit, res, ComparatorAscendingStable(*this, nan_direction_hint, &collator), DefaultSort(), DefaultPartialSort());
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
|
||||
getPermutationImpl(limit, res, ComparatorDescendingUnstable(*this, nan_direction_hint, &collator), DefaultSort(), DefaultPartialSort());
|
||||
else
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
|
||||
getPermutationImpl(limit, res, ComparatorDescendingStable(*this, nan_direction_hint, &collator), DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
@ -848,7 +848,7 @@ void ColumnArray::updatePermutationWithCollation(const Collator & collator, Perm
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorAscendingStable(*this, nan_direction_hint, &collator), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingUnstable(*this, nan_direction_hint, &collator), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
else
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingStable(*this, nan_direction_hint, &collator), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
|
@ -173,7 +173,7 @@ void ColumnFixedString::getPermutation(IColumn::PermutationSortDirection directi
|
||||
}
|
||||
|
||||
void ColumnFixedString::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const
|
||||
size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const
|
||||
{
|
||||
auto comparator_equal = ComparatorEqual(*this);
|
||||
|
||||
|
@ -409,11 +409,8 @@ void ColumnLowCardinality::updatePermutation(IColumn::PermutationSortDirection d
|
||||
auto comparator = [this, ascending, stability, nan_direction_hint](size_t lhs, size_t rhs)
|
||||
{
|
||||
int ret = getDictionary().compareAt(getIndexes().getUInt(lhs), getIndexes().getUInt(rhs), getDictionary(), nan_direction_hint);
|
||||
if (stability == IColumn::PermutationSortStability::Stable)
|
||||
{
|
||||
if (unlikely(ret == 0))
|
||||
return lhs < rhs;
|
||||
}
|
||||
if (unlikely(stability == IColumn::PermutationSortStability::Stable && ret == 0))
|
||||
return lhs < rhs;
|
||||
|
||||
if (ascending)
|
||||
return ret < 0;
|
||||
@ -449,11 +446,8 @@ void ColumnLowCardinality::updatePermutationWithCollation(const Collator & colla
|
||||
|
||||
int ret = nested_column->compareAtWithCollation(lhs_index, rhs_index, *nested_column, nan_direction_hint, collator);
|
||||
|
||||
if (stability == IColumn::PermutationSortStability::Stable)
|
||||
{
|
||||
if (unlikely(ret == 0))
|
||||
return lhs < rhs;
|
||||
}
|
||||
if (unlikely(stability == IColumn::PermutationSortStability::Stable && ret == 0))
|
||||
return lhs < rhs;
|
||||
|
||||
if (ascending)
|
||||
return ret < 0;
|
||||
|
@ -312,6 +312,10 @@ void ColumnNullable::getPermutationImpl(IColumn::PermutationSortDirection direct
|
||||
else
|
||||
limit = std::min(res_size, limit);
|
||||
|
||||
/// For stable sort we must process all NULL values
|
||||
if (unlikely(stability == IColumn::PermutationSortStability::Stable))
|
||||
limit = res_size;
|
||||
|
||||
if (is_nulls_last)
|
||||
{
|
||||
/// Shift all NULL values to the end.
|
||||
@ -328,10 +332,6 @@ void ColumnNullable::getPermutationImpl(IColumn::PermutationSortDirection direct
|
||||
|
||||
++read_idx;
|
||||
|
||||
/// For stable sort we must process all NULL values
|
||||
if (stability == IColumn::PermutationSortStability::Stable)
|
||||
limit = res_size;
|
||||
|
||||
/// Invariants:
|
||||
/// write_idx < read_idx
|
||||
/// write_idx points to NULL
|
||||
@ -352,16 +352,9 @@ void ColumnNullable::getPermutationImpl(IColumn::PermutationSortDirection direct
|
||||
++read_idx;
|
||||
}
|
||||
|
||||
if (stability == IColumn::PermutationSortStability::Stable)
|
||||
if (unlikely(stability == IColumn::PermutationSortStability::Stable) && write_idx != res_size)
|
||||
{
|
||||
ssize_t nulls_start_index = limit - 1;
|
||||
|
||||
while (nulls_start_index >= 0 && isNullAt(res[nulls_start_index])) {
|
||||
--nulls_start_index;
|
||||
}
|
||||
|
||||
++nulls_start_index;
|
||||
::sort(res.begin() + nulls_start_index, res.begin() + limit);
|
||||
::sort(res.begin() + write_idx, res.begin() + res_size);
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -389,14 +382,9 @@ void ColumnNullable::getPermutationImpl(IColumn::PermutationSortDirection direct
|
||||
--read_idx;
|
||||
}
|
||||
|
||||
if (stability == IColumn::PermutationSortStability::Stable)
|
||||
if (unlikely(stability == IColumn::PermutationSortStability::Stable) && write_idx != 0)
|
||||
{
|
||||
size_t nulls_end_index = 0;
|
||||
|
||||
while (nulls_end_index < limit && isNullAt(res[nulls_end_index]))
|
||||
++nulls_end_index;
|
||||
|
||||
::sort(res.begin(), res.begin() + nulls_end_index);
|
||||
::sort(res.begin(), res.begin() + write_idx + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -460,7 +448,7 @@ void ColumnNullable::updatePermutationImpl(IColumn::PermutationSortDirection dir
|
||||
if (first != write_idx)
|
||||
new_ranges.emplace_back(first, write_idx);
|
||||
|
||||
/// We have a range [write_idx, list) of NULL values
|
||||
/// We have a range [write_idx, last) of NULL values
|
||||
if (write_idx != last)
|
||||
null_ranges.emplace_back(write_idx, last);
|
||||
}
|
||||
@ -512,6 +500,13 @@ void ColumnNullable::updatePermutationImpl(IColumn::PermutationSortDirection dir
|
||||
getNestedColumn().updatePermutation(direction, stability, limit, null_direction_hint, res, new_ranges);
|
||||
|
||||
equal_ranges = std::move(new_ranges);
|
||||
|
||||
if (unlikely(stability == PermutationSortStability::Stable)) {
|
||||
for (auto & null_range : null_ranges) {
|
||||
::sort(res.begin() + null_range.first, res.begin() + null_range.second);
|
||||
}
|
||||
}
|
||||
|
||||
std::move(null_ranges.begin(), null_ranges.end(), std::back_inserter(equal_ranges));
|
||||
}
|
||||
|
||||
|
@ -552,6 +552,12 @@ void ColumnSparse::getPermutationImpl(IColumn::PermutationSortDirection directio
|
||||
void ColumnSparse::getPermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int null_direction_hint, Permutation & res) const
|
||||
{
|
||||
if (unlikely(stability == IColumn::PermutationSortStability::Stable)) {
|
||||
auto this_full = convertToFullColumnIfSparse();
|
||||
this_full->getPermutation(direction, stability, limit, null_direction_hint, res);
|
||||
return;
|
||||
}
|
||||
|
||||
return getPermutationImpl(direction, stability, limit, null_direction_hint, res, nullptr);
|
||||
}
|
||||
|
||||
|
@ -339,7 +339,7 @@ void ColumnString::getPermutation(PermutationSortDirection direction, Permutatio
|
||||
getPermutationImpl(limit, res, ComparatorAscendingStable(*this), DefaultSort(), DefaultPartialSort());
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
|
||||
getPermutationImpl(limit, res, ComparatorDescendingUnstable(*this), DefaultSort(), DefaultPartialSort());
|
||||
else
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
|
||||
getPermutationImpl(limit, res, ComparatorDescendingStable(*this), DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
@ -354,12 +354,12 @@ void ColumnString::updatePermutation(PermutationSortDirection direction, Permuta
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorAscendingStable(*this), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingUnstable(*this), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
else
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
|
||||
updatePermutationImpl(limit, res, equal_ranges, ComparatorDescendingStable(*this), comparator_equal, DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
void ColumnString::getPermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability,
|
||||
size_t limit, int, Permutation & res) const
|
||||
size_t limit, int /*nan_direction_hint*/, Permutation & res) const
|
||||
{
|
||||
if (direction == IColumn::PermutationSortDirection::Ascending && stability == IColumn::PermutationSortStability::Unstable)
|
||||
getPermutationImpl(limit, res, ComparatorAscendingUnstable(*this, &collator), DefaultSort(), DefaultPartialSort());
|
||||
@ -367,12 +367,12 @@ void ColumnString::getPermutationWithCollation(const Collator & collator, Permut
|
||||
getPermutationImpl(limit, res, ComparatorAscendingStable(*this, &collator), DefaultSort(), DefaultPartialSort());
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Unstable)
|
||||
getPermutationImpl(limit, res, ComparatorDescendingUnstable(*this, &collator), DefaultSort(), DefaultPartialSort());
|
||||
else
|
||||
else if (direction == IColumn::PermutationSortDirection::Descending && stability == IColumn::PermutationSortStability::Stable)
|
||||
getPermutationImpl(limit, res, ComparatorDescendingStable(*this, &collator), DefaultSort(), DefaultPartialSort());
|
||||
}
|
||||
|
||||
void ColumnString::updatePermutationWithCollation(const Collator & collator, PermutationSortDirection direction, PermutationSortStability stability,
|
||||
size_t limit, int, Permutation & res, EqualRanges & equal_ranges) const
|
||||
size_t limit, int /*nan_direction_hint*/, Permutation & res, EqualRanges & equal_ranges) const
|
||||
{
|
||||
auto comparator_equal = ComparatorEqual(*this, &collator);
|
||||
|
||||
|
@ -89,8 +89,6 @@ public:
|
||||
}
|
||||
|
||||
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override;
|
||||
void updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability,
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override;
|
||||
|
||||
void getExtremes(Field & min, Field & max) const override { column_holder->getExtremes(min, max); }
|
||||
bool valuesHaveFixedSize() const override { return column_holder->valuesHaveFixedSize(); }
|
||||
@ -432,46 +430,6 @@ int ColumnUnique<ColumnType>::compareAt(size_t n, size_t m, const IColumn & rhs,
|
||||
return getNestedColumn()->compareAt(n, m, *column_unique.getNestedColumn(), nan_direction_hint);
|
||||
}
|
||||
|
||||
template <typename ColumnType>
|
||||
void ColumnUnique<ColumnType>::updatePermutation(IColumn::PermutationSortDirection direction, IColumn::PermutationSortStability stability, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const
|
||||
{
|
||||
(void)(stability);
|
||||
bool reverse = direction == IColumn::PermutationSortDirection::Descending;
|
||||
|
||||
if (equal_ranges.empty())
|
||||
return;
|
||||
|
||||
bool found_null_value_index = false;
|
||||
for (size_t i = 0; i < equal_ranges.size() && !found_null_value_index; ++i)
|
||||
{
|
||||
auto & [first, last] = equal_ranges[i];
|
||||
for (auto j = first; j < last; ++j)
|
||||
{
|
||||
if (res[j] == getNullValueIndex())
|
||||
{
|
||||
if ((nan_direction_hint > 0) != reverse)
|
||||
{
|
||||
std::swap(res[j], res[last - 1]);
|
||||
--last;
|
||||
}
|
||||
else
|
||||
{
|
||||
std::swap(res[j], res[first]);
|
||||
++first;
|
||||
}
|
||||
if (last - first <= 1)
|
||||
{
|
||||
equal_ranges.erase(equal_ranges.begin() + i);
|
||||
}
|
||||
found_null_value_index = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
getNestedColumn()->updatePermutation(direction, stability, limit, nan_direction_hint, res, equal_ranges);
|
||||
}
|
||||
|
||||
template <typename IndexType>
|
||||
static void checkIndexes(const ColumnVector<IndexType> & indexes, size_t max_dictionary_size)
|
||||
{
|
||||
|
@ -220,10 +220,10 @@ void ColumnVector<T>::getPermutation(IColumn::PermutationSortDirection direction
|
||||
bool sort_is_stable = stability == IColumn::PermutationSortStability::Stable;
|
||||
|
||||
/// TODO: LSD RadixSort is currently not stable if direction is descending, or value is floating point
|
||||
bool should_use_radix_sort_for_stable_sort = (sort_is_stable && ascending && !std::is_floating_point_v<T>) || !sort_is_stable;
|
||||
bool use_radix_sort = (sort_is_stable && ascending && !std::is_floating_point_v<T>) || !sort_is_stable;
|
||||
|
||||
/// Thresholds on size. Lower threshold is arbitrary. Upper threshold is chosen by the type for histogram counters.
|
||||
if (s >= 256 && s <= std::numeric_limits<UInt32>::max() && should_use_radix_sort_for_stable_sort)
|
||||
if (s >= 256 && s <= std::numeric_limits<UInt32>::max() && use_radix_sort)
|
||||
{
|
||||
PaddedPODArray<ValueWithIndex<T>> pairs(s);
|
||||
for (UInt32 i = 0; i < UInt32(s); ++i)
|
||||
|
@ -313,8 +313,8 @@ public:
|
||||
|
||||
/** Returns a permutation that sorts elements of this column,
|
||||
* i.e. perm[i]-th element of source column should be i-th element of sorted column.
|
||||
* direction - permutation direction Ascending, Descending.
|
||||
* sort_type - permutation sort type Unstable, Stable.
|
||||
* direction - permutation direction.
|
||||
* stability - stability of result permutation.
|
||||
* limit - if isn't 0, then only first limit elements of the result column could be sorted.
|
||||
* nan_direction_hint - see above.
|
||||
*/
|
||||
|
@ -248,16 +248,28 @@ struct ComparatorEqualHelperImpl : public ComparatorBase
|
||||
};
|
||||
|
||||
template <typename ComparatorBase>
|
||||
using ComparatorAscendingUnstableImpl = ComparatorHelperImpl<ComparatorBase, IColumn::PermutationSortDirection::Ascending, IColumn::PermutationSortStability::Unstable>;
|
||||
using ComparatorAscendingUnstableImpl = ComparatorHelperImpl<
|
||||
ComparatorBase,
|
||||
IColumn::PermutationSortDirection::Ascending,
|
||||
IColumn::PermutationSortStability::Unstable>;
|
||||
|
||||
template <typename ComparatorBase>
|
||||
using ComparatorAscendingStableImpl = ComparatorHelperImpl<ComparatorBase, IColumn::PermutationSortDirection::Ascending, IColumn::PermutationSortStability::Stable>;
|
||||
using ComparatorAscendingStableImpl = ComparatorHelperImpl<
|
||||
ComparatorBase,
|
||||
IColumn::PermutationSortDirection::Ascending,
|
||||
IColumn::PermutationSortStability::Stable>;
|
||||
|
||||
template <typename ComparatorBase>
|
||||
using ComparatorDescendingUnstableImpl = ComparatorHelperImpl<ComparatorBase, IColumn::PermutationSortDirection::Descending, IColumn::PermutationSortStability::Unstable>;
|
||||
using ComparatorDescendingUnstableImpl = ComparatorHelperImpl<
|
||||
ComparatorBase,
|
||||
IColumn::PermutationSortDirection::Descending,
|
||||
IColumn::PermutationSortStability::Unstable>;
|
||||
|
||||
template <typename ComparatorBase>
|
||||
using ComparatorDescendingStableImpl = ComparatorHelperImpl<ComparatorBase, IColumn::PermutationSortDirection::Descending, IColumn::PermutationSortStability::Stable>;
|
||||
using ComparatorDescendingStableImpl = ComparatorHelperImpl<
|
||||
ComparatorBase,
|
||||
IColumn::PermutationSortDirection::Descending,
|
||||
IColumn::PermutationSortStability::Stable>;
|
||||
|
||||
template <typename ComparatorBase>
|
||||
using ComparatorEqualImpl = ComparatorEqualHelperImpl<ComparatorBase>;
|
||||
|
@ -160,6 +160,12 @@ public:
|
||||
throw Exception("Method getPermutation is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
void updatePermutation(PermutationSortDirection, PermutationSortStability,
|
||||
size_t, int, Permutation &, EqualRanges &) const override
|
||||
{
|
||||
throw Exception("Method getPermutation is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
std::vector<MutableColumnPtr> scatter(IColumn::ColumnIndex, const IColumn::Selector &) const override
|
||||
{
|
||||
throw Exception("Method scatter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
@ -2,25 +2,30 @@
|
||||
|
||||
#include <vector>
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Columns/ColumnUnique.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnSparse.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
|
||||
|
||||
using namespace DB;
|
||||
|
||||
IColumn::Permutation stableGetColumnPermutation(const IColumn & column, IColumn::PermutationSortDirection direction, size_t limit = 0, int nan_direction_hint = 0)
|
||||
void stableGetColumnPermutation(const IColumn & column, IColumn::PermutationSortDirection direction,
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & out_permutation)
|
||||
{
|
||||
(void)(limit);
|
||||
|
||||
IColumn::Permutation out_permutation;
|
||||
|
||||
size_t size = column.size();
|
||||
out_permutation.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
@ -34,16 +39,12 @@ IColumn::Permutation stableGetColumnPermutation(const IColumn & column, IColumn:
|
||||
else
|
||||
return res > 0;
|
||||
});
|
||||
|
||||
return out_permutation;
|
||||
}
|
||||
|
||||
IColumn::Permutation columnGetPermutation(const IColumn & column, IColumn::PermutationSortDirection direction, size_t limit = 0, int nan_direction_hint = 0)
|
||||
void columnGetPermutation(const IColumn & column, IColumn::PermutationSortDirection direction,
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & out_permutation)
|
||||
{
|
||||
IColumn::Permutation column_permutation;
|
||||
column.getPermutation(direction, IColumn::PermutationSortStability::Stable, limit, nan_direction_hint, column_permutation);
|
||||
|
||||
return column_permutation;
|
||||
column.getPermutation(direction, IColumn::PermutationSortStability::Stable, limit, nan_direction_hint, out_permutation);
|
||||
}
|
||||
|
||||
void printColumn(const IColumn & column)
|
||||
@ -59,15 +60,6 @@ void printColumn(const IColumn & column)
|
||||
std::cout << std::endl;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
struct IndexInRangeValueTransform
|
||||
{
|
||||
Field operator()(size_t range_index, size_t index_in_range) const
|
||||
{
|
||||
return Field(static_cast<T>(range_index * index_in_range));
|
||||
}
|
||||
};
|
||||
|
||||
template <typename ValueTransform>
|
||||
void generateRanges(std::vector<std::vector<Field>> & ranges, size_t range_size, ValueTransform value_transform)
|
||||
{
|
||||
@ -99,8 +91,6 @@ void insertRangesIntoColumn(std::vector<std::vector<Field>> & ranges, const std:
|
||||
}
|
||||
|
||||
void assertPermutationsWithLimit(const IColumn::Permutation & lhs, const IColumn::Permutation & rhs, size_t limit) {
|
||||
ASSERT_EQ(lhs.size(), rhs.size());
|
||||
|
||||
if (limit == 0) {
|
||||
limit = lhs.size();
|
||||
}
|
||||
@ -110,40 +100,30 @@ void assertPermutationsWithLimit(const IColumn::Permutation & lhs, const IColumn
|
||||
}
|
||||
}
|
||||
|
||||
void assertColumnPermutation(const IColumn & column, IColumn::PermutationSortDirection direction, size_t limit = 0, int nan_direction_hint = 0)
|
||||
void assertColumnPermutation(
|
||||
const IColumn & column, IColumn::PermutationSortDirection direction,
|
||||
size_t limit, int nan_direction_hint, IColumn::Permutation & actual_permutation, IColumn::Permutation & expected_permutation)
|
||||
{
|
||||
// std::cout << "Limit " << limit << std::endl;
|
||||
|
||||
auto expected = stableGetColumnPermutation(column, direction, limit, nan_direction_hint);
|
||||
auto actual = columnGetPermutation(column, direction, limit, nan_direction_hint);
|
||||
stableGetColumnPermutation(column, direction, limit, nan_direction_hint, expected_permutation);
|
||||
columnGetPermutation(column, direction, limit, nan_direction_hint, actual_permutation);
|
||||
|
||||
if (limit == 0) {
|
||||
limit = actual.size();
|
||||
limit = actual_permutation.size();
|
||||
}
|
||||
|
||||
// std::cout << "Column" << std::endl;
|
||||
// printColumn(column);
|
||||
|
||||
// std::cout << "Expected " << std::endl;
|
||||
// for (size_t i = 0; i < limit; ++i) {
|
||||
// std::cout << expected[i] << ' ';
|
||||
// }
|
||||
// std::cout << std::endl;
|
||||
|
||||
// std::cout << "Actual " << std::endl;
|
||||
// for (size_t i = 0; i < limit; ++i) {
|
||||
// std::cout << actual[i] << ' ';
|
||||
// }
|
||||
// std::cout << std::endl;
|
||||
|
||||
assertPermutationsWithLimit(actual, expected, limit);
|
||||
assertPermutationsWithLimit(actual_permutation, expected_permutation, limit);
|
||||
}
|
||||
|
||||
template <typename ColumnCreateFunc, typename ValueTransform>
|
||||
void assertColumnPermutations(ColumnCreateFunc column_create_func, ValueTransform value_transform)
|
||||
{
|
||||
static constexpr size_t ranges_size = 3;
|
||||
static const std::vector<size_t> range_sizes = { 5, 50, 500, 5000 };
|
||||
static const std::vector<size_t> range_sizes = {
|
||||
1,
|
||||
5,
|
||||
50,
|
||||
500
|
||||
};
|
||||
|
||||
std::vector<std::vector<Field>> ranges(ranges_size);
|
||||
std::vector<size_t> ranges_permutations(ranges_size);
|
||||
@ -151,6 +131,9 @@ void assertColumnPermutations(ColumnCreateFunc column_create_func, ValueTransfor
|
||||
ranges_permutations[i] = i;
|
||||
}
|
||||
|
||||
IColumn::Permutation actual_permutation;
|
||||
IColumn::Permutation expected_permutation;
|
||||
|
||||
for (const auto & range_size : range_sizes) {
|
||||
generateRanges(ranges, range_size, value_transform);
|
||||
std::sort(ranges_permutations.begin(), ranges_permutations.end());
|
||||
@ -161,8 +144,6 @@ void assertColumnPermutations(ColumnCreateFunc column_create_func, ValueTransfor
|
||||
auto & column = *column_ptr;
|
||||
insertRangesIntoColumn(ranges, ranges_permutations, column);
|
||||
|
||||
// printColumn(column);
|
||||
|
||||
static constexpr size_t limit_parts = 4;
|
||||
|
||||
size_t column_size = column.size();
|
||||
@ -170,18 +151,18 @@ void assertColumnPermutations(ColumnCreateFunc column_create_func, ValueTransfor
|
||||
|
||||
for (size_t limit = 0; limit < column_size; limit += column_limit_part)
|
||||
{
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Ascending, limit, -1);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Ascending, limit, 1);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Ascending, limit, -1, actual_permutation, expected_permutation);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Ascending, limit, 1, actual_permutation, expected_permutation);
|
||||
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Descending, limit, -1);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Descending, limit, 1);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Descending, limit, -1, actual_permutation, expected_permutation);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Descending, limit, 1, actual_permutation, expected_permutation);
|
||||
}
|
||||
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Ascending, 0, -1);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Ascending, 0, 1);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Ascending, 0, -1, actual_permutation, expected_permutation);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Ascending, 0, 1, actual_permutation, expected_permutation);
|
||||
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Descending, 0, -1);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Descending, 0, 1);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Descending, 0, -1, actual_permutation, expected_permutation);
|
||||
assertColumnPermutation(column, IColumn::PermutationSortDirection::Descending, 0, 1, actual_permutation, expected_permutation);
|
||||
|
||||
if (!std::next_permutation(ranges_permutations.begin(), ranges_permutations.end()))
|
||||
break;
|
||||
@ -189,13 +170,21 @@ void assertColumnPermutations(ColumnCreateFunc column_create_func, ValueTransfor
|
||||
}
|
||||
}
|
||||
|
||||
struct IndexInRangeInt64Transform
|
||||
{
|
||||
Field operator()(size_t range_index, size_t index_in_range) const
|
||||
{
|
||||
return Field(static_cast<Int64>(range_index * index_in_range));
|
||||
}
|
||||
};
|
||||
|
||||
TEST(StablePermutation, ColumnVectorInteger)
|
||||
{
|
||||
auto create_column = []() {
|
||||
return ColumnVector<Int64>::create();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeValueTransform<Int64>());
|
||||
assertColumnPermutations(create_column, IndexInRangeInt64Transform());
|
||||
}
|
||||
|
||||
struct IndexInRangeFloat64Transform
|
||||
@ -223,6 +212,24 @@ TEST(StablePermutation, ColumnVectorFloat)
|
||||
assertColumnPermutations(create_column, IndexInRangeFloat64Transform());
|
||||
}
|
||||
|
||||
struct IndexInRangeDecimal64Transform
|
||||
{
|
||||
Field operator()(size_t range_index, size_t index_in_range) const
|
||||
{
|
||||
return Field(static_cast<Decimal64>(range_index * index_in_range));
|
||||
}
|
||||
};
|
||||
|
||||
TEST(StablePermutation, ColumnVectorDecimal)
|
||||
{
|
||||
auto create_column = []() {
|
||||
return ColumnDecimal<Decimal64>::create(0, 0);
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeDecimal64Transform());
|
||||
}
|
||||
|
||||
|
||||
struct IndexInRangeStringTransform
|
||||
{
|
||||
Field operator()(size_t range_index, size_t index_in_range) const
|
||||
@ -293,7 +300,6 @@ struct IndexInRangeToNullTransform
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
TEST(StablePermutation, ColumnNullable)
|
||||
{
|
||||
{
|
||||
@ -304,9 +310,9 @@ TEST(StablePermutation, ColumnNullable)
|
||||
return nullable_data_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeNullableTransform<IndexInRangeValueTransform<Int64>>());
|
||||
assertColumnPermutations(create_column, IndexInRangeNullableTransform<IndexInRangeInt64Transform>());
|
||||
assertColumnPermutations(create_column, IndexInRangeToNullTransform());
|
||||
assertColumnPermutations(create_column, IndexInRangeValueTransform<Int64>());
|
||||
assertColumnPermutations(create_column, IndexInRangeInt64Transform());
|
||||
}
|
||||
{
|
||||
auto float_data_type = std::make_shared<DataTypeFloat64>();
|
||||
@ -318,6 +324,173 @@ TEST(StablePermutation, ColumnNullable)
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeNullableTransform<IndexInRangeFloat64Transform>());
|
||||
assertColumnPermutations(create_column, IndexInRangeToNullTransform());
|
||||
assertColumnPermutations(create_column, IndexInRangeValueTransform<Float64>());
|
||||
assertColumnPermutations(create_column, IndexInRangeFloat64Transform());
|
||||
}
|
||||
}
|
||||
|
||||
TEST(StablePermutation, ColumnLowCardinality)
|
||||
{
|
||||
{
|
||||
auto int_data_type = std::make_shared<DataTypeInt64>();
|
||||
auto low_cardinality_data_type = std::make_shared<DataTypeLowCardinality>(std::move(int_data_type));
|
||||
|
||||
auto create_column = [&]() {
|
||||
return low_cardinality_data_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeInt64Transform());
|
||||
}
|
||||
{
|
||||
auto int_data_type = std::make_shared<DataTypeInt64>();
|
||||
auto nullable_data_type = std::make_shared<DataTypeNullable>(std::move(int_data_type));
|
||||
auto low_cardinality_data_type = std::make_shared<DataTypeLowCardinality>(nullable_data_type);
|
||||
|
||||
auto create_column = [&]() {
|
||||
return low_cardinality_data_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeNullableTransform<IndexInRangeInt64Transform>());
|
||||
assertColumnPermutations(create_column, IndexInRangeToNullTransform());
|
||||
assertColumnPermutations(create_column, IndexInRangeInt64Transform());
|
||||
}
|
||||
{
|
||||
auto float_data_type = std::make_shared<DataTypeFloat64>();
|
||||
auto nullable_data_type = std::make_shared<DataTypeNullable>(std::move(float_data_type));
|
||||
auto low_cardinality_data_type = std::make_shared<DataTypeLowCardinality>(nullable_data_type);
|
||||
|
||||
auto create_column = [&]() {
|
||||
return low_cardinality_data_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeNullableTransform<IndexInRangeFloat64Transform>());
|
||||
assertColumnPermutations(create_column, IndexInRangeToNullTransform());
|
||||
assertColumnPermutations(create_column, IndexInRangeFloat64Transform());
|
||||
}
|
||||
}
|
||||
|
||||
template <typename FirstValueTransform, typename SecondValueTransform>
|
||||
struct TupleTransform
|
||||
{
|
||||
Field operator()(size_t range_index, size_t index_in_range) const
|
||||
{
|
||||
Field first_value = first_value_transform(range_index, index_in_range);
|
||||
Field second_value = second_value_transform(range_index, index_in_range);
|
||||
|
||||
return Tuple{std::move(first_value), std::move(second_value)};
|
||||
}
|
||||
|
||||
FirstValueTransform first_value_transform;
|
||||
SecondValueTransform second_value_transform;
|
||||
};
|
||||
|
||||
TEST(StablePermutation, ColumnTuple)
|
||||
{
|
||||
{
|
||||
auto int_data_type = std::make_shared<DataTypeInt64>();
|
||||
auto float_data_type = std::make_shared<DataTypeFloat64>();
|
||||
DataTypes tuple_data_types = {int_data_type, float_data_type};
|
||||
auto tuple_type = std::make_shared<DataTypeTuple>(tuple_data_types);
|
||||
|
||||
auto create_column = [&]() {
|
||||
return tuple_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeInt64Transform, IndexInRangeFloat64Transform>());
|
||||
}
|
||||
{
|
||||
auto int_data_type = std::make_shared<DataTypeInt64>();
|
||||
auto float_type = std::make_shared<DataTypeFloat64>();
|
||||
auto nullable_data_type = std::make_shared<DataTypeNullable>(int_data_type);
|
||||
DataTypes tuple_data_types = {nullable_data_type, float_type};
|
||||
auto tuple_type = std::make_shared<DataTypeTuple>(tuple_data_types);
|
||||
|
||||
auto create_column = [&]() {
|
||||
return tuple_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeNullableTransform<IndexInRangeInt64Transform>, IndexInRangeFloat64Transform>());
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeNullableTransform<IndexInRangeToNullTransform>, IndexInRangeFloat64Transform>());
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeNullableTransform<IndexInRangeInt64Transform>, IndexInRangeFloat64Transform>());
|
||||
}
|
||||
{
|
||||
auto int_data_type = std::make_shared<DataTypeInt64>();
|
||||
auto float_type = std::make_shared<DataTypeFloat64>();
|
||||
auto nullable_data_type = std::make_shared<DataTypeNullable>(int_data_type);
|
||||
DataTypes tuple_data_types = {float_type, nullable_data_type};
|
||||
auto tuple_type = std::make_shared<DataTypeTuple>(tuple_data_types);
|
||||
|
||||
auto create_column = [&]() {
|
||||
return tuple_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeFloat64Transform, IndexInRangeNullableTransform<IndexInRangeInt64Transform>>());
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeFloat64Transform, IndexInRangeNullableTransform<IndexInRangeToNullTransform>>());
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeFloat64Transform, IndexInRangeNullableTransform<IndexInRangeInt64Transform>>());
|
||||
}
|
||||
{
|
||||
auto float_data_type = std::make_shared<DataTypeFloat64>();
|
||||
auto nullable_data_type = std::make_shared<DataTypeNullable>(float_data_type);
|
||||
DataTypes tuple_data_types = {nullable_data_type, float_data_type};
|
||||
auto tuple_type = std::make_shared<DataTypeTuple>(tuple_data_types);
|
||||
|
||||
auto create_column = [&]() {
|
||||
return tuple_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeNullableTransform<IndexInRangeFloat64Transform>, IndexInRangeFloat64Transform>());
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeNullableTransform<IndexInRangeToNullTransform>, IndexInRangeFloat64Transform>());
|
||||
assertColumnPermutations(create_column, TupleTransform<IndexInRangeNullableTransform<IndexInRangeFloat64Transform>, IndexInRangeFloat64Transform>());
|
||||
}
|
||||
}
|
||||
|
||||
template <typename FirstValueTransform, typename SecondValueTransform>
|
||||
struct MapTransform
|
||||
{
|
||||
Field operator()(size_t range_index, size_t index_in_range) const
|
||||
{
|
||||
Field first_value = first_value_transform(range_index, index_in_range);
|
||||
Field second_value = second_value_transform(range_index, index_in_range);
|
||||
|
||||
return Map{Tuple{std::move(first_value), std::move(second_value)}};
|
||||
}
|
||||
|
||||
FirstValueTransform first_value_transform;
|
||||
SecondValueTransform second_value_transform;
|
||||
};
|
||||
|
||||
TEST(StablePermutation, ColumnMap)
|
||||
{
|
||||
{
|
||||
auto int_data_type = std::make_shared<DataTypeInt64>();
|
||||
auto float_data_type = std::make_shared<DataTypeFloat64>();
|
||||
auto map_type = std::make_shared<DataTypeMap>(int_data_type, float_data_type);
|
||||
|
||||
auto create_column = [&]() {
|
||||
return map_type->createColumn();
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, MapTransform<IndexInRangeInt64Transform, IndexInRangeFloat64Transform>());
|
||||
}
|
||||
}
|
||||
|
||||
TEST(StablePermutation, ColumnSparse)
|
||||
{
|
||||
{
|
||||
auto int_data_type = std::make_shared<DataTypeInt64>();
|
||||
|
||||
auto create_column = [&]() {
|
||||
return ColumnSparse::create(int_data_type->createColumn());
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeInt64Transform());
|
||||
}
|
||||
{
|
||||
auto float_data_type = std::make_shared<DataTypeFloat64>();
|
||||
|
||||
auto create_column = [&]() {
|
||||
return ColumnSparse::create(float_data_type->createColumn());
|
||||
};
|
||||
|
||||
assertColumnPermutations(create_column, IndexInRangeFloat64Transform());
|
||||
}
|
||||
}
|
||||
|
@ -38,4 +38,4 @@
|
||||
<drop_query>DROP TABLE IF EXISTS {integer_primary_key_table_name}</drop_query>
|
||||
<drop_query>DROP TABLE IF EXISTS {string_primary_key_table_name}</drop_query>
|
||||
|
||||
</test>
|
||||
</test>
|
||||
|
Loading…
Reference in New Issue
Block a user