2017-04-01 09:19:00 +00:00
|
|
|
#include <Interpreters/sortBlock.h>
|
2012-07-17 20:05:36 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Columns/ColumnString.h>
|
2019-12-05 15:31:45 +00:00
|
|
|
#include <Columns/ColumnConst.h>
|
2020-10-22 20:23:44 +00:00
|
|
|
#include <Columns/ColumnNullable.h>
|
|
|
|
#include <Columns/ColumnLowCardinality.h>
|
2022-01-27 13:42:08 +00:00
|
|
|
#include <Columns/ColumnTuple.h>
|
2017-07-13 20:58:19 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2019-12-06 22:22:12 +00:00
|
|
|
#include <Functions/FunctionHelpers.h>
|
2017-03-12 13:01:19 +00:00
|
|
|
|
2019-02-02 13:52:20 +00:00
|
|
|
#include <pdqsort.h>
|
2012-07-17 20:05:36 +00:00
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int BAD_COLLATION;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2019-12-06 22:22:12 +00:00
|
|
|
static bool isCollationRequired(const SortColumnDescription & description)
|
2018-10-04 10:24:51 +00:00
|
|
|
{
|
2019-12-06 22:22:12 +00:00
|
|
|
return description.collator != nullptr;
|
2018-10-04 10:24:51 +00:00
|
|
|
}
|
|
|
|
|
2022-01-27 13:42:08 +00:00
|
|
|
/// Column with description for sort
|
|
|
|
struct ColumnWithSortDescription
|
|
|
|
{
|
|
|
|
const IColumn * column = nullptr;
|
|
|
|
const SortColumnDescription * description = nullptr;
|
|
|
|
|
|
|
|
/// It means, that this column is ColumnConst
|
|
|
|
bool column_const = false;
|
|
|
|
};
|
|
|
|
using ColumnsWithSortDescriptions = std::vector<ColumnWithSortDescription>;
|
|
|
|
|
|
|
|
void flattenTupleColumnRecursively(
|
|
|
|
ColumnsWithSortDescriptions & res, const ColumnTuple * tuple, const SortColumnDescription * description, bool is_constant)
|
|
|
|
{
|
|
|
|
for (const auto & column : tuple->getColumns())
|
|
|
|
{
|
|
|
|
if (const auto * subtuple = typeid_cast<const ColumnTuple *>(column.get()))
|
|
|
|
flattenTupleColumnRecursively(res, subtuple, description, is_constant);
|
|
|
|
else
|
|
|
|
res.emplace_back(ColumnWithSortDescription{column.get(), description, is_constant});
|
|
|
|
}
|
|
|
|
}
|
2018-10-04 10:24:51 +00:00
|
|
|
|
|
|
|
ColumnsWithSortDescriptions getColumnsWithSortDescription(const Block & block, const SortDescription & description)
|
2016-11-20 04:47:51 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = description.size();
|
|
|
|
ColumnsWithSortDescriptions res;
|
|
|
|
res.reserve(size);
|
2016-11-20 04:47:51 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
|
|
|
const IColumn * column = !description[i].column_name.empty()
|
|
|
|
? block.getByName(description[i].column_name).column.get()
|
|
|
|
: block.safeGetByPosition(description[i].column_number).column.get();
|
2016-11-20 04:47:51 +00:00
|
|
|
|
2022-01-28 20:49:58 +00:00
|
|
|
if (isCollationRequired(description[i]))
|
|
|
|
{
|
|
|
|
if (!column->isCollationSupported())
|
|
|
|
throw Exception(
|
|
|
|
"Collations could be specified only for String, LowCardinality(String), Nullable(String) or for Array or Tuple, "
|
|
|
|
"containing them.",
|
|
|
|
ErrorCodes::BAD_COLLATION);
|
|
|
|
}
|
2022-01-27 13:42:08 +00:00
|
|
|
if (const auto * tuple = typeid_cast<const ColumnTuple *>(column))
|
|
|
|
flattenTupleColumnRecursively(res, tuple, &description[i], isColumnConst(*column));
|
|
|
|
else
|
|
|
|
res.emplace_back(ColumnWithSortDescription{column, &description[i], isColumnConst(*column)});
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
return res;
|
2016-11-20 04:47:51 +00:00
|
|
|
}
|
|
|
|
|
2018-10-04 14:55:02 +00:00
|
|
|
struct PartialSortingLess
|
|
|
|
{
|
|
|
|
const ColumnsWithSortDescriptions & columns;
|
|
|
|
|
|
|
|
explicit PartialSortingLess(const ColumnsWithSortDescriptions & columns_) : columns(columns_) {}
|
|
|
|
|
|
|
|
bool operator() (size_t a, size_t b) const
|
|
|
|
{
|
2020-03-09 00:28:05 +00:00
|
|
|
for (const auto & elem : columns)
|
2018-10-04 14:55:02 +00:00
|
|
|
{
|
2019-12-06 22:22:12 +00:00
|
|
|
int res;
|
2020-03-09 00:28:05 +00:00
|
|
|
if (elem.column_const)
|
2019-12-06 22:22:12 +00:00
|
|
|
res = 0;
|
|
|
|
else
|
2022-01-27 13:42:08 +00:00
|
|
|
res = elem.description->direction * elem.column->compareAt(a, b, *elem.column, elem.description->nulls_direction);
|
2018-10-04 14:55:02 +00:00
|
|
|
if (res < 0)
|
|
|
|
return true;
|
|
|
|
else if (res > 0)
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2013-05-28 16:56:05 +00:00
|
|
|
struct PartialSortingLessWithCollation
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
const ColumnsWithSortDescriptions & columns;
|
|
|
|
|
2019-12-06 22:22:12 +00:00
|
|
|
explicit PartialSortingLessWithCollation(const ColumnsWithSortDescriptions & columns_)
|
|
|
|
: columns(columns_)
|
|
|
|
{
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
bool operator() (size_t a, size_t b) const
|
|
|
|
{
|
2020-03-09 00:28:05 +00:00
|
|
|
for (const auto & elem : columns)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
int res;
|
2019-12-06 22:22:12 +00:00
|
|
|
|
2020-03-09 00:28:05 +00:00
|
|
|
if (elem.column_const)
|
2019-12-09 01:15:42 +00:00
|
|
|
{
|
2019-12-06 22:22:12 +00:00
|
|
|
res = 0;
|
2019-12-09 01:15:42 +00:00
|
|
|
}
|
2022-01-27 13:42:08 +00:00
|
|
|
else if (isCollationRequired(*elem.description))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2022-01-27 13:42:08 +00:00
|
|
|
res = elem.column->compareAtWithCollation(a, b, *elem.column, elem.description->nulls_direction, *elem.description->collator);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2022-01-27 13:42:08 +00:00
|
|
|
res = elem.column->compareAt(a, b, *elem.column, elem.description->nulls_direction);
|
|
|
|
res *= elem.description->direction;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (res < 0)
|
|
|
|
return true;
|
|
|
|
else if (res > 0)
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
2012-07-17 20:05:36 +00:00
|
|
|
};
|
|
|
|
|
2019-02-10 15:17:45 +00:00
|
|
|
void sortBlock(Block & block, const SortDescription & description, UInt64 limit)
|
2012-07-17 20:05:36 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!block)
|
|
|
|
return;
|
|
|
|
|
2022-01-27 13:42:08 +00:00
|
|
|
ColumnsWithSortDescriptions columns_with_sort_desc = getColumnsWithSortDescription(block, description);
|
2022-01-28 17:37:39 +00:00
|
|
|
bool all_const = true;
|
|
|
|
for (const auto & column : columns_with_sort_desc)
|
|
|
|
{
|
|
|
|
if (!column.column_const)
|
|
|
|
{
|
|
|
|
all_const = false;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
if (all_const)
|
|
|
|
return;
|
|
|
|
|
2022-01-27 13:42:08 +00:00
|
|
|
IColumn::Permutation perm;
|
2017-04-01 07:20:54 +00:00
|
|
|
/// If only one column to sort by
|
2022-01-27 13:42:08 +00:00
|
|
|
if (columns_with_sort_desc.size() == 1)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2022-01-27 13:42:08 +00:00
|
|
|
bool reverse = columns_with_sort_desc[0].description->direction == -1;
|
|
|
|
const IColumn * column = columns_with_sort_desc[0].column;
|
2019-12-06 22:22:12 +00:00
|
|
|
if (isCollationRequired(description[0]))
|
2022-01-28 20:49:58 +00:00
|
|
|
column->getPermutationWithCollation(*description[0].collator, reverse, limit, description[0].nulls_direction, perm);
|
|
|
|
else
|
2020-05-15 00:01:14 +00:00
|
|
|
{
|
2022-01-27 13:42:08 +00:00
|
|
|
int nan_direction_hint = columns_with_sort_desc[0].description->nulls_direction;
|
2020-09-19 14:20:00 +00:00
|
|
|
column->getPermutation(reverse, limit, nan_direction_hint, perm);
|
2020-05-15 00:01:14 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2019-12-07 13:13:14 +00:00
|
|
|
size_t size = block.rows();
|
2022-01-27 13:42:08 +00:00
|
|
|
perm.resize(size);
|
2019-12-07 13:13:14 +00:00
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
perm[i] = i;
|
|
|
|
|
|
|
|
if (limit >= size)
|
|
|
|
limit = 0;
|
|
|
|
|
2022-01-28 20:49:58 +00:00
|
|
|
EqualRanges ranges;
|
|
|
|
ranges.emplace_back(0, perm.size());
|
2022-01-27 13:42:08 +00:00
|
|
|
for (const auto & column : columns_with_sort_desc)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2022-01-28 20:49:58 +00:00
|
|
|
while (!ranges.empty() && limit && limit <= ranges.back().first)
|
|
|
|
ranges.pop_back();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2022-01-28 20:49:58 +00:00
|
|
|
if (ranges.empty())
|
|
|
|
break;
|
|
|
|
|
|
|
|
if (column.column_const)
|
|
|
|
continue;
|
|
|
|
|
|
|
|
if (isCollationRequired(*column.description))
|
2020-05-12 00:58:58 +00:00
|
|
|
{
|
2022-01-28 20:49:58 +00:00
|
|
|
column.column->updatePermutationWithCollation(
|
|
|
|
*column.description->collator,
|
|
|
|
column.description->direction < 0,
|
|
|
|
limit,
|
|
|
|
column.description->nulls_direction,
|
|
|
|
perm,
|
|
|
|
ranges);
|
2020-05-12 00:58:58 +00:00
|
|
|
}
|
2022-01-28 20:49:58 +00:00
|
|
|
else
|
2020-05-12 00:58:58 +00:00
|
|
|
{
|
2020-05-14 21:00:56 +00:00
|
|
|
column.column->updatePermutation(
|
2022-01-27 13:42:08 +00:00
|
|
|
column.description->direction < 0, limit, column.description->nulls_direction, perm, ranges);
|
2020-05-12 00:58:58 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2022-01-27 13:42:08 +00:00
|
|
|
}
|
|
|
|
size_t columns = block.columns();
|
|
|
|
for (size_t i = 0; i < columns; ++i)
|
2022-01-28 17:37:39 +00:00
|
|
|
block.getByPosition(i).column = block.getByPosition(i).column->permute(perm, limit);
|
2012-07-17 20:05:36 +00:00
|
|
|
}
|
|
|
|
|
2013-11-29 22:10:15 +00:00
|
|
|
|
2015-08-14 02:45:40 +00:00
|
|
|
void stableGetPermutation(const Block & block, const SortDescription & description, IColumn::Permutation & out_permutation)
|
2013-11-29 22:10:15 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!block)
|
|
|
|
return;
|
2013-11-29 22:10:15 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t size = block.rows();
|
|
|
|
out_permutation.resize(size);
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
out_permutation[i] = i;
|
2013-11-29 22:10:15 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ColumnsWithSortDescriptions columns_with_sort_desc = getColumnsWithSortDescription(block, description);
|
2013-11-29 22:10:15 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::stable_sort(out_permutation.begin(), out_permutation.end(), PartialSortingLess(columns_with_sort_desc));
|
2015-08-14 02:45:40 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-11-21 15:47:32 +00:00
|
|
|
bool isAlreadySorted(const Block & block, const SortDescription & description)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!block)
|
|
|
|
return true;
|
2015-11-21 15:47:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t rows = block.rows();
|
2015-11-21 15:47:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ColumnsWithSortDescriptions columns_with_sort_desc = getColumnsWithSortDescription(block, description);
|
2015-11-21 15:47:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
PartialSortingLess less(columns_with_sort_desc);
|
2015-11-21 15:47:32 +00:00
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/** If the rows are not too few, then let's make a quick attempt to verify that the block is not sorted.
|
|
|
|
* Constants - at random.
|
|
|
|
*/
|
2017-04-01 07:20:54 +00:00
|
|
|
static constexpr size_t num_rows_to_try = 10;
|
|
|
|
if (rows > num_rows_to_try * 5)
|
|
|
|
{
|
|
|
|
for (size_t i = 1; i < num_rows_to_try; ++i)
|
|
|
|
{
|
|
|
|
size_t prev_position = rows * (i - 1) / num_rows_to_try;
|
|
|
|
size_t curr_position = rows * i / num_rows_to_try;
|
2015-11-21 15:47:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (less(curr_position, prev_position))
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
2015-11-21 15:47:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (size_t i = 1; i < rows; ++i)
|
|
|
|
if (less(i, i - 1))
|
|
|
|
return false;
|
2015-11-21 15:47:32 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return true;
|
2015-11-21 15:47:32 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2015-08-14 02:45:40 +00:00
|
|
|
void stableSortBlock(Block & block, const SortDescription & description)
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
if (!block)
|
|
|
|
return;
|
2015-08-14 02:45:40 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
IColumn::Permutation perm;
|
|
|
|
stableGetPermutation(block, description, perm);
|
2013-11-29 22:10:15 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
size_t columns = block.columns();
|
|
|
|
for (size_t i = 0; i < columns; ++i)
|
|
|
|
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->permute(perm, 0);
|
2013-11-29 22:10:15 +00:00
|
|
|
}
|
|
|
|
|
2012-07-17 20:05:36 +00:00
|
|
|
}
|