Merge pull request #34145 from kitaisreal/bitset-sort-performance-check

pdqsort performance check
This commit is contained in:
Maksim Kita 2022-01-31 12:35:13 +01:00 committed by GitHub
commit 8513f20cfd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
71 changed files with 240 additions and 133 deletions

View File

@ -1,26 +1,42 @@
#pragma once
#include <pdqsort.h>
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wold-style-cast"
#include <miniselect/floyd_rivest_select.h>
template <class RandomIt>
template <typename RandomIt>
void nth_element(RandomIt first, RandomIt nth, RandomIt last)
{
::miniselect::floyd_rivest_select(first, nth, last);
}
template <class RandomIt>
template <typename RandomIt>
void partial_sort(RandomIt first, RandomIt middle, RandomIt last)
{
::miniselect::floyd_rivest_partial_sort(first, middle, last);
}
template <class RandomIt, class Compare>
template <typename RandomIt, typename Compare>
void partial_sort(RandomIt first, RandomIt middle, RandomIt last, Compare compare)
{
::miniselect::floyd_rivest_partial_sort(first, middle, last, compare);
}
#pragma GCC diagnostic pop
template <typename RandomIt, typename Compare>
void sort(RandomIt first, RandomIt last, Compare compare)
{
::pdqsort(first, last, compare);
}
template <typename RandomIt>
void sort(RandomIt first, RandomIt last)
{
using value_type = typename std::iterator_traits<RandomIt>::value_type;
using comparator = std::less<value_type>;
::pdqsort(first, last, comparator());
}

View File

@ -3,6 +3,7 @@
#include <algorithm>
#include <memory>
#include <boost/noncopyable.hpp>
#include <base/sort.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/HashTable/SmallTable.h>
@ -557,7 +558,7 @@ public:
}
if (limit < answer.size())
{
std::nth_element(answer.begin(), answer.begin() + limit, answer.end());
::nth_element(answer.begin(), answer.begin() + limit, answer.end());
answer.resize(limit);
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <base/sort.h>
#include <Common/Arena.h>
#include <Common/NaNUtils.h>
@ -72,7 +74,7 @@ private:
private:
void sort()
{
std::sort(points, points + size,
::sort(points, points + size,
[](const WeightedValue & first, const WeightedValue & second)
{
return first.mean < second.mean;

View File

@ -1,5 +1,7 @@
#pragma once
#include <unordered_set>
#include <AggregateFunctions/AggregateFunctionNull.h>
#include <Columns/ColumnsNumber.h>
@ -7,6 +9,7 @@
#include <Common/ArenaAllocator.h>
#include <Common/assert_cast.h>
#include <base/arithmeticOverflow.h>
#include <base/sort.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
@ -14,8 +17,6 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <unordered_set>
namespace DB
{
@ -67,7 +68,7 @@ struct AggregateFunctionIntervalLengthSumData
/// either sort whole container or do so partially merging ranges afterwards
if (!sorted && !other.sorted)
{
std::sort(std::begin(segments), std::end(segments));
::sort(std::begin(segments), std::end(segments));
}
else
{
@ -76,10 +77,10 @@ struct AggregateFunctionIntervalLengthSumData
const auto end = std::end(segments);
if (!sorted)
std::sort(begin, middle);
::sort(begin, middle);
if (!other.sorted)
std::sort(middle, end);
::sort(middle, end);
std::inplace_merge(begin, middle, end);
}
@ -91,7 +92,7 @@ struct AggregateFunctionIntervalLengthSumData
{
if (!sorted)
{
std::sort(std::begin(segments), std::end(segments));
::sort(std::begin(segments), std::end(segments));
sorted = true;
}
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <unordered_map>
#include <base/sort.h>
#include <AggregateFunctions/AggregateFunctionCombinatorFactory.h>
#include <AggregateFunctions/IAggregateFunction.h>
#include <Columns/ColumnFixedString.h>
@ -226,7 +227,7 @@ public:
{
keys.push_back(it.first);
}
std::sort(keys.begin(), keys.end());
::sort(keys.begin(), keys.end());
// insert using sorted keys to result column
for (auto & key : keys)

View File

@ -1,6 +1,7 @@
#pragma once
#include <base/logger_useful.h>
#include <base/sort.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
@ -142,7 +143,7 @@ public:
auto & array = this->data(place).value;
/// Sort by position; for equal position, sort by weight to get deterministic result.
std::sort(array.begin(), array.end());
::sort(array.begin(), array.end());
for (const auto & point_weight : array)
{

View File

@ -6,6 +6,7 @@
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <base/range.h>
#include <base/sort.h>
#include <Common/PODArray.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
@ -76,7 +77,7 @@ struct AggregateFunctionSequenceMatchData final
{
if (!sorted)
{
std::sort(std::begin(events_list), std::end(events_list), Comparator{});
::sort(std::begin(events_list), std::end(events_list), Comparator{});
sorted = true;
}
}

View File

@ -2,6 +2,7 @@
#include <base/types.h>
#include <base/bit_cast.h>
#include <base/sort.h>
#include <Common/HashTable/HashMap.h>
#include <IO/ReadBuffer.h>
@ -134,7 +135,7 @@ private:
++arr_it;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 threshold = std::ceil(sum_weight * level);
Float64 accumulated = 0;
@ -175,7 +176,7 @@ private:
++arr_it;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
size_t level_index = 0;
Float64 accumulated = 0;

View File

@ -88,7 +88,7 @@ struct QuantileExact : QuantileExactBase<Value, QuantileExact<Value>>
if (!array.empty())
{
size_t n = level < 1 ? level * array.size() : (array.size() - 1);
nth_element(array.begin(), array.begin() + n, array.end()); /// NOTE: You can think of the radix-select algorithm.
::nth_element(array.begin(), array.begin() + n, array.end()); /// NOTE: You can think of the radix-select algorithm.
return array[n];
}
@ -107,7 +107,7 @@ struct QuantileExact : QuantileExactBase<Value, QuantileExact<Value>>
auto level = levels[indices[i]];
size_t n = level < 1 ? level * array.size() : (array.size() - 1);
nth_element(array.begin() + prev_n, array.begin() + n, array.end());
::nth_element(array.begin() + prev_n, array.begin() + n, array.end());
result[indices[i]] = array[n];
prev_n = n;
}
@ -143,7 +143,7 @@ struct QuantileExactExclusive : public QuantileExact<Value>
else if (n < 1)
return static_cast<Float64>(array[0]);
nth_element(array.begin(), array.begin() + n - 1, array.end());
::nth_element(array.begin(), array.begin() + n - 1, array.end());
auto nth_elem = std::min_element(array.begin() + n, array.end());
return static_cast<Float64>(array[n - 1]) + (h - n) * static_cast<Float64>(*nth_elem - array[n - 1]);
@ -172,7 +172,7 @@ struct QuantileExactExclusive : public QuantileExact<Value>
result[indices[i]] = static_cast<Float64>(array[0]);
else
{
nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end());
::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end());
auto nth_elem = std::min_element(array.begin() + n, array.end());
result[indices[i]] = static_cast<Float64>(array[n - 1]) + (h - n) * static_cast<Float64>(*nth_elem - array[n - 1]);
@ -207,7 +207,7 @@ struct QuantileExactInclusive : public QuantileExact<Value>
return static_cast<Float64>(array[array.size() - 1]);
else if (n < 1)
return static_cast<Float64>(array[0]);
nth_element(array.begin(), array.begin() + n - 1, array.end());
::nth_element(array.begin(), array.begin() + n - 1, array.end());
auto nth_elem = std::min_element(array.begin() + n, array.end());
return static_cast<Float64>(array[n - 1]) + (h - n) * static_cast<Float64>(*nth_elem - array[n - 1]);
@ -234,7 +234,7 @@ struct QuantileExactInclusive : public QuantileExact<Value>
result[indices[i]] = static_cast<Float64>(array[0]);
else
{
nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end());
::nth_element(array.begin() + prev_n, array.begin() + n - 1, array.end());
auto nth_elem = std::min_element(array.begin() + n, array.end());
result[indices[i]] = static_cast<Float64>(array[n - 1]) + (h - n) * (static_cast<Float64>(*nth_elem) - array[n - 1]);
@ -263,7 +263,7 @@ struct QuantileExactLow : public QuantileExactBase<Value, QuantileExactLow<Value
if (!array.empty())
{
// sort inputs in ascending order
std::sort(array.begin(), array.end());
::sort(array.begin(), array.end());
// if level is 0.5 then compute the "low" median of the sorted array
// by the method of rounding.
@ -296,7 +296,7 @@ struct QuantileExactLow : public QuantileExactBase<Value, QuantileExactLow<Value
if (!array.empty())
{
// sort inputs in ascending order
std::sort(array.begin(), array.end());
::sort(array.begin(), array.end());
for (size_t i = 0; i < size; ++i)
{
auto level = levels[indices[i]];
@ -345,7 +345,7 @@ struct QuantileExactHigh : public QuantileExactBase<Value, QuantileExactHigh<Val
if (!array.empty())
{
// sort inputs in ascending order
std::sort(array.begin(), array.end());
::sort(array.begin(), array.end());
// if level is 0.5 then compute the "high" median of the sorted array
// by the method of rounding.
@ -370,7 +370,7 @@ struct QuantileExactHigh : public QuantileExactBase<Value, QuantileExactHigh<Val
if (!array.empty())
{
// sort inputs in ascending order
std::sort(array.begin(), array.end());
::sort(array.begin(), array.end());
for (size_t i = 0; i < size; ++i)
{
auto level = levels[indices[i]];

View File

@ -1,5 +1,7 @@
#pragma once
#include <base/sort.h>
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
@ -101,7 +103,7 @@ struct QuantileExactWeighted
++i;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 threshold = std::ceil(sum_weight * level);
Float64 accumulated = 0;
@ -151,7 +153,7 @@ struct QuantileExactWeighted
++i;
}
std::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
::sort(array, array + size, [](const Pair & a, const Pair & b) { return a.first < b.first; });
Float64 accumulated = 0;

View File

@ -90,7 +90,7 @@ namespace detail
/** This function must be called before get-functions. */
void prepare() const
{
std::sort(elems, elems + count);
::sort(elems, elems + count);
}
UInt16 get(double level) const
@ -183,7 +183,7 @@ namespace detail
/// Sorting an array will not be considered a violation of constancy.
auto & array = elems;
nth_element(array.begin(), array.begin() + n, array.end());
::nth_element(array.begin(), array.begin() + n, array.end());
quantile = array[n];
}
@ -204,7 +204,7 @@ namespace detail
? level * elems.size()
: (elems.size() - 1);
nth_element(array.begin() + prev_n, array.begin() + n, array.end());
::nth_element(array.begin() + prev_n, array.begin() + n, array.end());
result[level_index] = array[n];
prev_n = n;

View File

@ -2,6 +2,8 @@
#include <vector>
#include <base/sort.h>
#include <Common/FieldVisitorConvertToNumber.h>
#include <Common/NaNUtils.h>
@ -64,7 +66,7 @@ struct QuantileLevels
permutation[i] = i;
}
std::sort(permutation.begin(), permutation.end(), [this] (size_t a, size_t b) { return levels[a] < levels[b]; });
::sort(permutation.begin(), permutation.end(), [this] (size_t a, size_t b) { return levels[a] < levels[b]; });
}
};

View File

@ -4,6 +4,7 @@
#include <algorithm>
#include <climits>
#include <base/types.h>
#include <base/sort.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
@ -15,6 +16,7 @@
#include <Poco/Exception.h>
#include <pcg_random.hpp>
namespace DB
{
struct Settings;
@ -249,7 +251,7 @@ private:
if (sorted)
return;
sorted = true;
std::sort(samples.begin(), samples.end(), Comparer());
::sort(samples.begin(), samples.end(), Comparer());
}
template <typename ResultType>

View File

@ -5,6 +5,7 @@
#include <climits>
#include <AggregateFunctions/ReservoirSampler.h>
#include <base/types.h>
#include <base/sort.h>
#include <Common/HashTable/Hash.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>
@ -258,7 +259,8 @@ private:
{
if (sorted)
return;
std::sort(samples.begin(), samples.end(), [](const auto & lhs, const auto & rhs) { return lhs.first < rhs.first; });
::sort(samples.begin(), samples.end(), [](const auto & lhs, const auto & rhs) { return lhs.first < rhs.first; });
sorted = true;
}

View File

@ -1,13 +1,17 @@
#pragma once
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Common/ArenaAllocator.h>
#include <numeric>
#include <algorithm>
#include <utility>
#include <base/sort.h>
#include <Common/ArenaAllocator.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
namespace DB
{
struct Settings;
@ -41,7 +45,7 @@ std::pair<RanksArray, Float64> computeRanksAndTieCorrection(const Values & value
/// Save initial positions, than sort indices according to the values.
std::vector<size_t> indexes(size);
std::iota(indexes.begin(), indexes.end(), 0);
std::sort(indexes.begin(), indexes.end(),
::sort(indexes.begin(), indexes.end(),
[&] (size_t lhs, size_t rhs) { return values[lhs] < values[rhs]; });
size_t left = 0;

View File

@ -15,6 +15,7 @@
#include <Parsers/formatAST.h>
#include <Storages/IStorage.h>
#include <base/insertAtEnd.h>
#include <base/sort.h>
#include <boost/range/adaptor/reversed.hpp>
#include <filesystem>
@ -632,7 +633,7 @@ BackupEntries makeBackupEntries(const Elements & elements, const ContextPtr & co
throw Exception("Backup must not be empty", ErrorCodes::BACKUP_IS_EMPTY);
/// Check that all backup entries are unique.
std::sort(
::sort(
backup_entries.begin(),
backup_entries.end(),
[](const std::pair<String, std::unique_ptr<IBackupEntry>> & lhs, const std::pair<String, std::unique_ptr<IBackupEntry>> & rhs)

View File

@ -17,6 +17,7 @@
#include <Common/Exception.h>
#include <Poco/String.h>
#include <algorithm>
#include <base/sort.h>
namespace DB
@ -74,10 +75,10 @@ AvailableCollationLocales::LocalesVector AvailableCollationLocales::getAvailable
result.push_back(name_and_locale.second);
auto comparator = [] (const LocaleAndLanguage & f, const LocaleAndLanguage & s)
{
return f.locale_name < s.locale_name;
};
std::sort(result.begin(), result.end(), comparator);
{
return f.locale_name < s.locale_name;
};
::sort(result.begin(), result.end(), comparator);
return result;
}

View File

@ -16,6 +16,7 @@
#include <Common/WeakHash.h>
#include <Common/HashTable/Hash.h>
#include <base/unaligned.h>
#include <base/sort.h>
#include <cstring> // memcpy
@ -810,9 +811,9 @@ void ColumnArray::getPermutationImpl(size_t limit, Permutation & res, Comparator
auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; };
if (limit)
partial_sort(res.begin(), res.begin() + limit, res.end(), less);
::partial_sort(res.begin(), res.begin() + limit, res.end(), less);
else
std::sort(res.begin(), res.end(), less);
::sort(res.begin(), res.end(), less);
}
void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const

View File

@ -9,7 +9,6 @@
#include <base/sort.h>
#include <base/scope_guard.h>
#include <IO/WriteHelpers.h>
#include <Columns/ColumnsCommon.h>
@ -132,7 +131,7 @@ template <is_decimal T>
void ColumnDecimal<T>::updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_ranges) const
{
auto equals = [this](size_t lhs, size_t rhs) { return data[lhs] == data[rhs]; };
auto sort = [](auto begin, auto end, auto pred) { std::sort(begin, end, pred); };
auto sort = [](auto begin, auto end, auto pred) { ::sort(begin, end, pred); };
auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); };
if (reverse)

View File

@ -169,9 +169,9 @@ protected:
sort_end = res.begin() + limit;
if (reverse)
partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; });
::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] > data[b]; });
else
partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; });
::partial_sort(res.begin(), sort_end, res.end(), [this](size_t a, size_t b) { return data[a] < data[b]; });
}
};

View File

@ -192,9 +192,9 @@ void ColumnFixedString::getPermutation(bool reverse, size_t limit, int /*nan_dir
else
{
if (reverse)
std::sort(res.begin(), res.end(), greater(*this));
::sort(res.begin(), res.end(), greater(*this));
else
std::sort(res.begin(), res.end(), less(*this));
::sort(res.begin(), res.end(), less(*this));
}
}

View File

@ -335,9 +335,9 @@ void ColumnString::getPermutationImpl(size_t limit, Permutation & res, Comparato
auto less = [&cmp](size_t lhs, size_t rhs){ return cmp(lhs, rhs) < 0; };
if (limit)
partial_sort(res.begin(), res.begin() + limit, res.end(), less);
::partial_sort(res.begin(), res.begin() + limit, res.end(), less);
else
std::sort(res.begin(), res.end(), less);
::sort(res.begin(), res.end(), less);
}
void ColumnString::getPermutation(bool reverse, size_t limit, int /*nan_direction_hint*/, Permutation & res) const

View File

@ -1,5 +1,6 @@
#include <Columns/ColumnTuple.h>
#include <base/sort.h>
#include <Columns/IColumnImpl.h>
#include <Columns/ColumnCompressed.h>
#include <Core/Field.h>
@ -384,9 +385,9 @@ void ColumnTuple::getPermutationImpl(size_t limit, Permutation & res, LessOperat
limit = 0;
if (limit)
partial_sort(res.begin(), res.begin() + limit, res.end(), less);
::partial_sort(res.begin(), res.begin() + limit, res.end(), less);
else
std::sort(res.begin(), res.end(), less);
::sort(res.begin(), res.end(), less);
}
void ColumnTuple::updatePermutationImpl(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges, const Collator * collator) const

View File

@ -1,6 +1,5 @@
#include "ColumnVector.h"
#include <pdqsort.h>
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnCompressed.h>
#include <Columns/MaskOperations.h>
@ -118,7 +117,6 @@ struct ColumnVector<T>::equals
bool operator()(size_t lhs, size_t rhs) const { return CompareHelper<T>::equals(parent.data[lhs], parent.data[rhs], nan_direction_hint); }
};
namespace
{
template <typename T>
@ -158,9 +156,9 @@ void ColumnVector<T>::getPermutation(bool reverse, size_t limit, int nan_directi
res[i] = i;
if (reverse)
partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint));
::partial_sort(res.begin(), res.begin() + limit, res.end(), greater(*this, nan_direction_hint));
else
partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint));
::partial_sort(res.begin(), res.begin() + limit, res.end(), less(*this, nan_direction_hint));
}
else
{
@ -204,16 +202,16 @@ void ColumnVector<T>::getPermutation(bool reverse, size_t limit, int nan_directi
res[i] = i;
if (reverse)
pdqsort(res.begin(), res.end(), greater(*this, nan_direction_hint));
::sort(res.begin(), res.end(), greater(*this, nan_direction_hint));
else
pdqsort(res.begin(), res.end(), less(*this, nan_direction_hint));
::sort(res.begin(), res.end(), less(*this, nan_direction_hint));
}
}
template <typename T>
void ColumnVector<T>::updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const
{
auto sort = [](auto begin, auto end, auto pred) { pdqsort(begin, end, pred); };
auto sort = [](auto begin, auto end, auto pred) { ::sort(begin, end, pred); };
auto partial_sort = [](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); };
if (reverse)

View File

@ -528,7 +528,7 @@ protected:
template <typename Derived>
void getIndicesOfNonDefaultRowsImpl(Offsets & indices, size_t from, size_t limit) const;
/// Uses std::sort and partial_sort as default algorithms.
/// Uses sort and partial_sort as default algorithms.
/// Implements 'less' and 'equals' via comparator.
/// If 'less' and 'equals' can be implemented more optimal
/// (e.g. with less number of comparisons), you can use

View File

@ -11,6 +11,7 @@
#include <base/sort.h>
#include <algorithm>
namespace DB
{
namespace ErrorCodes
@ -203,7 +204,7 @@ void IColumn::updatePermutationImpl(
limit, res, equal_ranges,
[&cmp](size_t lhs, size_t rhs) { return cmp(lhs, rhs) < 0; },
[&cmp](size_t lhs, size_t rhs) { return cmp(lhs, rhs) == 0; },
[](auto begin, auto end, auto pred) { std::sort(begin, end, pred); },
[](auto begin, auto end, auto pred) { ::sort(begin, end, pred); },
[](auto begin, auto mid, auto end, auto pred) { ::partial_sort(begin, mid, end, pred); });
}

View File

@ -20,6 +20,7 @@
#include <Common/Exception.h>
#include <Common/getResource.h>
#include <base/errnoToString.h>
#include <base/sort.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
@ -105,7 +106,7 @@ static ElementIdentifier getElementIdentifier(Node * element)
std::string value = node->nodeValue();
attrs_kv.push_back(std::make_pair(name, value));
}
std::sort(attrs_kv.begin(), attrs_kv.end());
::sort(attrs_kv.begin(), attrs_kv.end());
ElementIdentifier res;
res.push_back(element->nodeName());
@ -443,7 +444,7 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string &
}
}
std::sort(files.begin(), files.end());
::sort(files.begin(), files.end());
return files;
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <base/defines.h>
#include <base/sort.h>
#include <vector>
#include <utility>
@ -489,14 +490,14 @@ private:
}
}
std::sort(intervals_sorted_by_left_asc.begin(), intervals_sorted_by_left_asc.end(), [](auto & lhs, auto & rhs)
::sort(intervals_sorted_by_left_asc.begin(), intervals_sorted_by_left_asc.end(), [](auto & lhs, auto & rhs)
{
auto & lhs_interval = getInterval(lhs);
auto & rhs_interval = getInterval(rhs);
return lhs_interval.left < rhs_interval.left;
});
std::sort(intervals_sorted_by_right_desc.begin(), intervals_sorted_by_right_desc.end(), [](auto & lhs, auto & rhs)
::sort(intervals_sorted_by_right_desc.begin(), intervals_sorted_by_right_desc.end(), [](auto & lhs, auto & rhs)
{
auto & lhs_interval = getInterval(lhs);
auto & rhs_interval = getInterval(rhs);
@ -681,7 +682,7 @@ private:
size_t size = points.size();
size_t middle_element_index = size / 2;
std::nth_element(points.begin(), points.begin() + middle_element_index, points.end());
::nth_element(points.begin(), points.begin() + middle_element_index, points.end());
/** We should not get median as average of middle_element_index and middle_element_index - 1
* because we want point in node to intersect some interval.

View File

@ -7,6 +7,7 @@
#include <functional>
#include <base/types.h>
#include <base/scope_guard.h>
#include <base/sort.h>
#include <Common/PoolBase.h>
#include <Common/ProfileEvents.h>
#include <Common/NetException.h>
@ -178,7 +179,7 @@ PoolWithFailoverBase<TNestedPool>::getShuffledPools(
shuffled_pools.reserve(nested_pools.size());
for (size_t i = 0; i < nested_pools.size(); ++i)
shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, 0});
std::sort(
::sort(
shuffled_pools.begin(), shuffled_pools.end(),
[](const ShuffledPool & lhs, const ShuffledPool & rhs)
{

View File

@ -5,6 +5,8 @@
#include <boost/range/adaptor/reversed.hpp>
#include <base/sort.h>
#include <Common/AllocatorWithMemoryTracking.h>
#include <Common/ArenaWithFreeLists.h>
#include <Common/HashTable/Hash.h>
@ -242,7 +244,7 @@ public:
}
}
std::sort(counter_list.begin(), counter_list.end(), [](Counter * l, Counter * r) { return *l > *r; });
::sort(counter_list.begin(), counter_list.end(), [](Counter * l, Counter * r) { return *l > *r; });
if (counter_list.size() > m_capacity)
{

View File

@ -12,6 +12,8 @@
//#include <iostream>
#include <filesystem>
#include <base/sort.h>
/**
ELF object can contain three different places with symbol names and addresses:
@ -498,8 +500,8 @@ void SymbolIndex::update()
{
dl_iterate_phdr(collectSymbols, &data);
std::sort(data.objects.begin(), data.objects.end(), [](const Object & a, const Object & b) { return a.address_begin < b.address_begin; });
std::sort(data.symbols.begin(), data.symbols.end(), [](const Symbol & a, const Symbol & b) { return a.address_begin < b.address_begin; });
::sort(data.objects.begin(), data.objects.end(), [](const Object & a, const Object & b) { return a.address_begin < b.address_begin; });
::sort(data.symbols.begin(), data.symbols.end(), [](const Symbol & a, const Symbol & b) { return a.address_begin < b.address_begin; });
/// We found symbols both from loaded program headers and from ELF symbol tables.
data.symbols.erase(std::unique(data.symbols.begin(), data.symbols.end(), [](const Symbol & a, const Symbol & b)

View File

@ -12,6 +12,7 @@
#include <Columns/ColumnSparse.h>
#include <iterator>
#include <base/sort.h>
namespace DB
@ -538,7 +539,7 @@ Block Block::sortColumns() const
for (auto it = index_by_name.begin(); it != index_by_name.end(); ++it)
sorted_index_by_name[i++] = it;
}
std::sort(sorted_index_by_name.begin(), sorted_index_by_name.end(), [](const auto & lhs, const auto & rhs)
::sort(sorted_index_by_name.begin(), sorted_index_by_name.end(), [](const auto & lhs, const auto & rhs)
{
return lhs->first < rhs->first;
});

View File

@ -1,4 +1,6 @@
#include <Core/NamesAndTypes.h>
#include <base/sort.h>
#include <Common/HashTable/HashMap.h>
#include <DataTypes/DataTypeFactory.h>
#include <IO/ReadBuffer.h>
@ -113,7 +115,7 @@ bool NamesAndTypesList::isSubsetOf(const NamesAndTypesList & rhs) const
{
NamesAndTypes vector(rhs.begin(), rhs.end());
vector.insert(vector.end(), begin(), end());
std::sort(vector.begin(), vector.end());
::sort(vector.begin(), vector.end());
return std::unique(vector.begin(), vector.end()) == vector.begin() + rhs.size();
}
@ -121,16 +123,16 @@ size_t NamesAndTypesList::sizeOfDifference(const NamesAndTypesList & rhs) const
{
NamesAndTypes vector(rhs.begin(), rhs.end());
vector.insert(vector.end(), begin(), end());
std::sort(vector.begin(), vector.end());
::sort(vector.begin(), vector.end());
return (std::unique(vector.begin(), vector.end()) - vector.begin()) * 2 - size() - rhs.size();
}
void NamesAndTypesList::getDifference(const NamesAndTypesList & rhs, NamesAndTypesList & deleted, NamesAndTypesList & added) const
{
NamesAndTypes lhs_vector(begin(), end());
std::sort(lhs_vector.begin(), lhs_vector.end());
::sort(lhs_vector.begin(), lhs_vector.end());
NamesAndTypes rhs_vector(rhs.begin(), rhs.end());
std::sort(rhs_vector.begin(), rhs_vector.end());
::sort(rhs_vector.begin(), rhs_vector.end());
std::set_difference(lhs_vector.begin(), lhs_vector.end(), rhs_vector.begin(), rhs_vector.end(),
std::back_inserter(deleted));

View File

@ -1,5 +1,7 @@
#include <DataTypes/EnumValues.h>
#include <boost/algorithm/string.hpp>
#include <base/sort.h>
namespace DB
{
@ -18,7 +20,7 @@ EnumValues<T>::EnumValues(const Values & values_)
if (values.empty())
throw Exception{"DataTypeEnum enumeration cannot be empty", ErrorCodes::EMPTY_DATA_PASSED};
std::sort(std::begin(values), std::end(values), [] (auto & left, auto & right)
::sort(std::begin(values), std::end(values), [] (auto & left, auto & right)
{
return left.second < right.second;
});

View File

@ -12,6 +12,7 @@
#include <base/logger_useful.h>
#include <base/scope_guard_safe.h>
#include <base/sort.h>
#include <iomanip>
#include <filesystem>
@ -151,7 +152,7 @@ DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const Filt
if (!filter_by_table_name || filter_by_table_name(table_name))
filtered_tables.push_back(table_name);
}
std::sort(filtered_tables.begin(), filtered_tables.end());
::sort(filtered_tables.begin(), filtered_tables.end());
return std::make_unique<DatabaseLazyIterator>(*this, std::move(filtered_tables));
}

View File

@ -142,7 +142,7 @@ ClusterPtr DatabaseReplicated::getClusterImpl() const
"It's possible if the first replica is not fully created yet "
"or if the last replica was just dropped or due to logical error", database_name);
Int32 cversion = stat.cversion;
std::sort(hosts.begin(), hosts.end());
::sort(hosts.begin(), hosts.end());
std::vector<zkutil::ZooKeeper::FutureGet> futures;
futures.reserve(hosts.size());

View File

@ -13,10 +13,12 @@
#include <base/itoa.h>
#include <base/map.h>
#include <base/range.h>
#include <base/sort.h>
#include <Dictionaries/DictionarySource.h>
#include <Dictionaries/DictionaryFactory.h>
#include <Functions/FunctionHelpers.h>
namespace DB
{
namespace ErrorCodes
@ -145,7 +147,7 @@ static void validateKeyTypes(const DataTypes & key_types)
template <typename T, typename Comp>
size_t sortAndUnique(std::vector<T> & vec, Comp comp)
{
std::sort(vec.begin(), vec.end(),
::sort(vec.begin(), vec.end(),
[&](const auto & a, const auto & b) { return comp(a, b) < 0; });
auto new_end = std::unique(vec.begin(), vec.end(),

View File

@ -3,6 +3,8 @@
#include <numeric>
#include <cmath>
#include <base/sort.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeArray.h>
@ -250,7 +252,7 @@ void IPolygonDictionary::loadData()
polygon_ids.emplace_back(polygon, i);
}
std::sort(polygon_ids.begin(), polygon_ids.end(), [& areas](const auto & lhs, const auto & rhs)
::sort(polygon_ids.begin(), polygon_ids.end(), [& areas](const auto & lhs, const auto & rhs)
{
return areas[lhs.second] < areas[rhs.second];
});

View File

@ -3,11 +3,13 @@
#include <Common/ThreadPool.h>
#include <base/logger_useful.h>
#include <base/sort.h>
#include <algorithm>
#include <thread>
#include <numeric>
namespace DB
{
@ -87,7 +89,7 @@ std::vector<Coord> SlabsPolygonIndex::uniqueX(const std::vector<Polygon> & polyg
}
/** Making all_x sorted and distinct */
std::sort(all_x.begin(), all_x.end());
::sort(all_x.begin(), all_x.end());
all_x.erase(std::unique(all_x.begin(), all_x.end()), all_x.end());
return all_x;
@ -104,7 +106,7 @@ void SlabsPolygonIndex::indexBuild(const std::vector<Polygon> & polygons)
}
/** Sorting edges of (left_point, right_point, polygon_id) in that order */
std::sort(all_edges.begin(), all_edges.end(), Edge::compareByLeftPoint);
::sort(all_edges.begin(), all_edges.end(), Edge::compareByLeftPoint);
for (size_t i = 0; i != all_edges.size(); ++i)
all_edges[i].edge_id = i;
@ -298,7 +300,7 @@ bool SlabsPolygonIndex::find(const Point & point, size_t & id) const
} while (pos != 0);
/** Sort all ids and find smallest with odd occurrences */
std::sort(intersections.begin(), intersections.end());
::sort(intersections.begin(), intersections.end());
for (size_t i = 0; i < intersections.size(); i += 2)
{
if (i + 1 == intersections.size() || intersections[i] != intersections[i + 1])

View File

@ -12,6 +12,7 @@
#include <absl/container/flat_hash_set.h>
#include <base/unaligned.h>
#include <base/sort.h>
#include <Common/randomSeed.h>
#include <Common/Arena.h>
#include <Common/ArenaWithFreeLists.h>
@ -24,6 +25,7 @@
#include <Dictionaries/ICacheDictionaryStorage.h>
#include <Dictionaries/DictionaryHelpers.h>
namespace CurrentMetrics
{
extern const Metric Write;
@ -1092,7 +1094,7 @@ private:
}
/// Sort blocks by offset before start async io requests
std::sort(blocks_to_request.begin(), blocks_to_request.end());
::sort(blocks_to_request.begin(), blocks_to_request.end());
file_buffer.fetchBlocks(configuration.read_buffer_blocks_size, blocks_to_request, [&](size_t block_index, char * block_data)
{

View File

@ -36,6 +36,7 @@
# include <IO/WriteBufferFromString.h>
# include <IO/WriteHelpers.h>
# include <base/range.h>
# include <base/sort.h>
# include <google/protobuf/descriptor.h>
# include <google/protobuf/descriptor.pb.h>
# include <boost/algorithm/string.hpp>
@ -2163,7 +2164,7 @@ namespace
for (auto & desc : field_descs_)
field_infos.emplace_back(std::move(desc.column_indices), *desc.field_descriptor, std::move(desc.field_serializer));
std::sort(field_infos.begin(), field_infos.end(),
::sort(field_infos.begin(), field_infos.end(),
[](const FieldInfo & lhs, const FieldInfo & rhs) { return lhs.field_tag < rhs.field_tag; });
for (size_t i : collections::range(field_infos.size()))
@ -2643,7 +2644,7 @@ namespace
missing_column_indices.clear();
missing_column_indices.reserve(column_names.size() - used_column_indices.size());
auto used_column_indices_sorted = std::move(used_column_indices);
std::sort(used_column_indices_sorted.begin(), used_column_indices_sorted.end());
::sort(used_column_indices_sorted.begin(), used_column_indices_sorted.end());
boost::range::set_difference(collections::range(column_names.size()), used_column_indices_sorted,
std::back_inserter(missing_column_indices));
@ -2755,7 +2756,7 @@ namespace
}
/// Shorter suffixes first.
std::sort(out_field_descriptors_with_suffixes.begin(), out_field_descriptors_with_suffixes.end(),
::sort(out_field_descriptors_with_suffixes.begin(), out_field_descriptors_with_suffixes.end(),
[](const std::pair<const FieldDescriptor *, std::string_view /* suffix */> & f1,
const std::pair<const FieldDescriptor *, std::string_view /* suffix */> & f2)
{

View File

@ -17,6 +17,7 @@
#include <type_traits>
#include <array>
#include <base/bit_cast.h>
#include <base/sort.h>
#include <algorithm>
#ifdef __SSE4_1__
@ -738,7 +739,7 @@ private:
for (size_t i = 0; i < boundaries.size(); ++i)
boundary_values[i] = boundaries[i].get<ValueType>();
std::sort(boundary_values.begin(), boundary_values.end());
::sort(boundary_values.begin(), boundary_values.end());
boundary_values.erase(std::unique(boundary_values.begin(), boundary_values.end()), boundary_values.end());
size_t size = src.size();

View File

@ -1,5 +1,6 @@
#include <algorithm>
#include <vector>
#include <base/sort.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include "arrayScalarProduct.h"
@ -112,7 +113,7 @@ public:
sorted_labels[i].label = label;
}
std::sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; });
::sort(sorted_labels.begin(), sorted_labels.end(), [](const auto & lhs, const auto & rhs) { return lhs.score > rhs.score; });
/// We will first calculate non-normalized area.

View File

@ -1,4 +1,5 @@
#include "FunctionArrayMapped.h"
#include <base/sort.h>
#include <Functions/FunctionFactory.h>
@ -49,7 +50,7 @@ struct ArraySortImpl
for (size_t i = 0; i < size; ++i)
{
auto next_offset = offsets[i];
std::sort(&permutation[current_offset], &permutation[next_offset], Less(*mapped));
::sort(&permutation[current_offset], &permutation[next_offset], Less(*mapped));
current_offset = next_offset;
}

View File

@ -1,3 +1,4 @@
#include <base/sort.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
@ -53,7 +54,7 @@ namespace
}
/// We sort the names because the result of the function should not depend on the order of UUIDs.
std::sort(role_names.begin(), role_names.end());
::sort(role_names.begin(), role_names.end());
}
size_t getNumberOfArguments() const override { return 0; }

View File

@ -13,6 +13,7 @@
#include <IO/Operators.h>
#include <stack>
#include <base/sort.h>
#include <Common/JSONBuilder.h>
namespace DB
@ -676,7 +677,7 @@ NameSet ActionsDAG::foldActionsByProjection(
void ActionsDAG::reorderAggregationKeysForProjection(const std::unordered_map<std::string_view, size_t> & key_names_pos_map)
{
std::sort(index.begin(), index.end(), [&key_names_pos_map](const Node * lhs, const Node * rhs)
::sort(index.begin(), index.end(), [&key_names_pos_map](const Node * lhs, const Node * rhs)
{
return key_names_pos_map.find(lhs->result_name)->second < key_names_pos_map.find(rhs->result_name)->second;
});

View File

@ -1,6 +1,7 @@
#include <future>
#include <Poco/Util/Application.h>
#include <base/sort.h>
#include <Common/Stopwatch.h>
#include <Common/setThreadName.h>
#include <Common/formatReadable.h>
@ -2167,7 +2168,7 @@ ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedData
if (non_empty_data.size() > 1)
{
/// Sort the states in descending order so that the merge is more efficient (since all states are merged into the first).
std::sort(non_empty_data.begin(), non_empty_data.end(),
::sort(non_empty_data.begin(), non_empty_data.end(),
[](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs)
{
return lhs->sizeWithoutOverflowRow() > rhs->sizeWithoutOverflowRow();

View File

@ -12,8 +12,10 @@
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/Application.h>
#include <base/range.h>
#include <base/sort.h>
#include <boost/range/algorithm_ext/erase.hpp>
namespace DB
{
@ -305,11 +307,11 @@ void Clusters::updateClusters(const Poco::Util::AbstractConfiguration & new_conf
Poco::Util::AbstractConfiguration::Keys deleted_keys;
if (old_config)
{
std::sort(new_config_keys.begin(), new_config_keys.end());
::sort(new_config_keys.begin(), new_config_keys.end());
Poco::Util::AbstractConfiguration::Keys old_config_keys;
old_config->keys(config_prefix, old_config_keys);
std::sort(old_config_keys.begin(), old_config_keys.end());
::sort(old_config_keys.begin(), old_config_keys.end());
std::set_difference(
old_config_keys.begin(), old_config_keys.end(), new_config_keys.begin(), new_config_keys.end(), std::back_inserter(deleted_keys));

View File

@ -1,4 +1,5 @@
#include <Interpreters/DDLTask.h>
#include <base/sort.h>
#include <Common/DNSResolver.h>
#include <Common/isLocalAddress.h>
#include <IO/WriteHelpers.h>
@ -14,6 +15,7 @@
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Databases/DatabaseReplicated.h>
namespace DB
{
@ -324,7 +326,7 @@ String DDLTask::getShardID() const
Strings replica_names;
for (const Cluster::Address & address : shard_addresses)
replica_names.emplace_back(address.readableString());
std::sort(replica_names.begin(), replica_names.end());
::sort(replica_names.begin(), replica_names.end());
String res;
for (auto it = replica_names.begin(); it != replica_names.end(); ++it)

View File

@ -28,6 +28,7 @@
#include <base/sleep.h>
#include <base/getFQDNOrHostName.h>
#include <base/logger_useful.h>
#include <base/sort.h>
#include <random>
#include <pcg_random.hpp>
#include <base/scope_guard_safe.h>
@ -221,7 +222,7 @@ DDLTaskPtr DDLWorker::initAndCheckTask(const String & entry_name, String & out_r
static void filterAndSortQueueNodes(Strings & all_nodes)
{
all_nodes.erase(std::remove_if(all_nodes.begin(), all_nodes.end(), [] (const String & s) { return !startsWith(s, "query-"); }), all_nodes.end());
std::sort(all_nodes.begin(), all_nodes.end());
::sort(all_nodes.begin(), all_nodes.end());
}
void DDLWorker::scheduleTasks(bool reinitialized)

View File

@ -16,6 +16,7 @@
#include <Columns/ColumnSet.h>
#include <queue>
#include <stack>
#include <base/sort.h>
#include <Common/JSONBuilder.h>
#include <Core/SettingsEnums.h>
@ -735,7 +736,7 @@ void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run)
}
else
{
std::sort(execution_context.inputs_pos.rbegin(), execution_context.inputs_pos.rend());
::sort(execution_context.inputs_pos.rbegin(), execution_context.inputs_pos.rend());
for (auto input : execution_context.inputs_pos)
if (input >= 0)
block.erase(input);

View File

@ -6,6 +6,7 @@
#include <stack>
#include <base/logger_useful.h>
#include <base/sort.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnVector.h>
@ -576,7 +577,10 @@ void ActionsDAG::compileFunctions(size_t min_count_to_compile_expression, const
/** Sort nodes before compilation using their children size to avoid compiling subexpression before compile parent expression.
* This is needed to avoid compiling expression more than once with different names because of compilation order.
*/
std::sort(nodes_to_compile.begin(), nodes_to_compile.end(), [&](const Node * lhs, const Node * rhs) { return node_to_data[lhs].children_size > node_to_data[rhs].children_size; });
::sort(nodes_to_compile.begin(), nodes_to_compile.end(), [&](const Node * lhs, const Node * rhs)
{
return node_to_data[lhs].children_size > node_to_data[rhs].children_size;
});
for (auto & node : nodes_to_compile)
{

View File

@ -72,6 +72,7 @@
#include <Core/Field.h>
#include <Core/ProtocolDefines.h>
#include <base/types.h>
#include <base/sort.h>
#include <Columns/Collator.h>
#include <Common/FieldVisitorsAccurateComparison.h>
#include <Common/FieldVisitorToString.h>
@ -2269,7 +2270,7 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan)
for (const auto & [_, w] : query_analyzer->windowDescriptions())
windows_sorted.push_back(&w);
std::sort(windows_sorted.begin(), windows_sorted.end(), windowDescriptionComparator);
::sort(windows_sorted.begin(), windows_sorted.end(), windowDescriptionComparator);
const Settings & settings = context->getSettingsRef();
for (size_t i = 0; i < windows_sorted.size(); ++i)

View File

@ -9,6 +9,8 @@
#include <deque>
#include <base/sort.h>
namespace DB
{
@ -180,7 +182,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
{
auto & equalities = chain.second;
auto & equality_functions = equalities.functions;
std::sort(equality_functions.begin(), equality_functions.end());
::sort(equality_functions.begin(), equality_functions.end());
}
}
@ -237,7 +239,7 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
}
/// Sort the literals so that they are specified in the same order in the IN expression.
std::sort(tuple.begin(), tuple.end());
::sort(tuple.begin(), tuple.end());
/// Get the expression `expr` from the chain `expr = x1 OR ... OR expr = xN`
ASTPtr equals_expr_lhs;

View File

@ -1,15 +1,18 @@
#pragma once
#include <Common/Arena.h>
#include <Columns/IColumn.h>
#include <Interpreters/asof.h>
#include <optional>
#include <variant>
#include <list>
#include <mutex>
#include <algorithm>
#include <base/sort.h>
#include <Common/Arena.h>
#include <Columns/IColumn.h>
#include <Interpreters/asof.h>
namespace DB
{
@ -199,7 +202,7 @@ private:
if (!sorted.load(std::memory_order_relaxed))
{
if (!array.empty())
std::sort(array.begin(), array.end(), (ascending ? less : greater));
::sort(array.begin(), array.end(), (ascending ? less : greater));
sorted.store(true, std::memory_order_release);
}

View File

@ -25,8 +25,10 @@
#include <Storages/MergeTree/KeyCondition.h>
#include <base/range.h>
#include <base/sort.h>
#include <DataTypes/DataTypeLowCardinality.h>
namespace DB
{
@ -405,7 +407,7 @@ void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) c
MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector<KeyTuplePositionMapping> && indexes_mapping_)
: has_all_keys(set_elements.size() == indexes_mapping_.size()), indexes_mapping(std::move(indexes_mapping_))
{
std::sort(indexes_mapping.begin(), indexes_mapping.end(),
::sort(indexes_mapping.begin(), indexes_mapping.end(),
[](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r)
{
return std::tie(l.key_index, l.tuple_index) < std::tie(r.key_index, r.tuple_index);

View File

@ -18,6 +18,7 @@
#include <Processors/Sinks/EmptySink.h>
#include <QueryPipeline/Pipe.h>
#include <filesystem>
#include <base/sort.h>
namespace fs = std::filesystem;
@ -124,7 +125,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, ContextPtr context,
use_local_default_database = true;
}
}
std::sort(shard_default_databases.begin(), shard_default_databases.end());
::sort(shard_default_databases.begin(), shard_default_databases.end());
shard_default_databases.erase(std::unique(shard_default_databases.begin(), shard_default_databases.end()), shard_default_databases.end());
assert(use_local_default_database || !shard_default_databases.empty());

View File

@ -1,5 +1,8 @@
#include <Processors/DelayedPortsProcessor.h>
#include <base/sort.h>
namespace DB
{
@ -18,7 +21,7 @@ InputPorts createInputPorts(
return InputPorts(num_ports, header);
InputPorts res;
std::sort(delayed_ports.begin(), delayed_ports.end());
::sort(delayed_ports.begin(), delayed_ports.end());
size_t next_delayed_port = 0;
for (size_t i = 0; i < num_ports; ++i)
{

View File

@ -28,6 +28,7 @@
#include <Processors/Formats/Impl/ConstantExpressionTemplate.h>
#include <Parsers/ExpressionElementParsers.h>
#include <boost/functional/hash.hpp>
#include <base/sort.h>
namespace DB
@ -299,7 +300,7 @@ ConstantExpressionTemplate::TemplateStructure::TemplateStructure(LiteralsInfo &
{
null_as_default = null_as_default_;
std::sort(replaced_literals.begin(), replaced_literals.end(), [](const LiteralInfo & a, const LiteralInfo & b)
::sort(replaced_literals.begin(), replaced_literals.end(), [](const LiteralInfo & a, const LiteralInfo & b)
{
return a.literal->begin.value() < b.literal->begin.value();
});

View File

@ -10,9 +10,11 @@
#include <unordered_map>
#include <fmt/format.h>
#include <base/sort.h>
#include <Poco/Util/AbstractConfiguration.h>
using namespace std::literals;
namespace DB::ErrorCodes
@ -286,7 +288,7 @@ std::string buildTaggedRegex(std::string regexp_str)
else
regexp_str = "[\\?&]";
std::sort(std::begin(tags), std::end(tags)); /* sorted tag keys */
::sort(std::begin(tags), std::end(tags)); /* sorted tag keys */
regexp_str += fmt::format(
"{}{}",
fmt::join(tags, "&(.*&)?"),
@ -419,7 +421,7 @@ appendGraphitePattern(
/// retention should be in descending order of age.
if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll
std::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions);
::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions);
patterns.emplace_back(pattern);
return patterns.back();

View File

@ -26,6 +26,7 @@
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/TreeRewriter.h>
#include <base/logger_useful.h>
#include <base/sort.h>
#include <Common/JSONBuilder.h>
namespace ProfileEvents
@ -1015,7 +1016,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
/// Skip this if final was used, because such columns were already added from PK.
std::vector<String> add_columns = result.sampling.filter_expression->getRequiredColumns().getNames();
column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end());
std::sort(column_names_to_read.begin(), column_names_to_read.end());
::sort(column_names_to_read.begin(), column_names_to_read.end());
column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()),
column_names_to_read.end());
}
@ -1039,7 +1040,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
if (!data.merging_params.version_column.empty())
column_names_to_read.push_back(data.merging_params.version_column);
std::sort(column_names_to_read.begin(), column_names_to_read.end());
::sort(column_names_to_read.begin(), column_names_to_read.end());
column_names_to_read.erase(std::unique(column_names_to_read.begin(), column_names_to_read.end()), column_names_to_read.end());
pipe = spreadMarkRangesAmongStreamsFinal(

View File

@ -19,6 +19,8 @@
#include <boost/algorithm/string/join.hpp>
#include <iterator>
#include <regex>
#include <base/sort.h>
namespace fs = std::filesystem;
@ -425,7 +427,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart(
}
if (!capability.empty())
{
std::sort(capability.begin(), capability.end());
::sort(capability.begin(), capability.end());
capability.erase(std::unique(capability.begin(), capability.end()), capability.end());
const String & remote_fs_metadata = boost::algorithm::join(capability, ", ");
uri.addQueryParameter("remote_fs_metadata", remote_fs_metadata);

View File

@ -1,6 +1,7 @@
#pragma once
#include <base/logger_useful.h>
#include <base/sort.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Core/BackgroundSchedulePool.h>
@ -48,7 +49,7 @@ void checkNoOldLeaders(Poco::Logger * log, ZooKeeper & zookeeper, const String p
}
else
{
std::sort(potential_leaders.begin(), potential_leaders.end());
::sort(potential_leaders.begin(), potential_leaders.end());
if (potential_leaders.front() == persistent_multiple_leaders)
return;

View File

@ -67,6 +67,7 @@
#include <boost/algorithm/string/replace.hpp>
#include <base/insertAtEnd.h>
#include <base/sort.h>
#include <algorithm>
#include <iomanip>
@ -1622,7 +1623,7 @@ size_t MergeTreeData::clearOldWriteAheadLogs()
if (all_block_numbers_on_disk.empty())
return 0;
std::sort(all_block_numbers_on_disk.begin(), all_block_numbers_on_disk.end());
::sort(all_block_numbers_on_disk.begin(), all_block_numbers_on_disk.end());
block_numbers_on_disk.push_back(all_block_numbers_on_disk[0]);
for (size_t i = 1; i < all_block_numbers_on_disk.size(); ++i)
{

View File

@ -7,6 +7,8 @@
#include <random>
#include <unordered_set>
#include <base/sort.h>
namespace DB
{
@ -110,7 +112,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs()
if (entries.empty())
return;
std::sort(entries.begin(), entries.end());
::sort(entries.begin(), entries.end());
String min_saved_record_log_str = entries[
entries.size() > storage_settings->max_replicated_logs_to_keep
@ -443,7 +445,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
}
}
std::sort(timed_blocks.begin(), timed_blocks.end(), NodeWithStat::greaterByTime);
::sort(timed_blocks.begin(), timed_blocks.end(), NodeWithStat::greaterByTime);
}
@ -476,7 +478,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldMutations()
}
Strings entries = zookeeper->getChildren(storage.zookeeper_path + "/mutations");
std::sort(entries.begin(), entries.end());
::sort(entries.begin(), entries.end());
/// Do not remove entries that are greater than `min_pointer` (they are not done yet).
entries.erase(std::upper_bound(entries.begin(), entries.end(), padIndex(min_pointer)), entries.end());

View File

@ -2,14 +2,15 @@
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/ReplicatedMergeTreeLogEntry.h>
#include <base/types.h>
#include <base/sort.h>
#include <optional>
#include <mutex>
#include <city.h>
#include <algorithm>
#include <atomic>
namespace DB
{
@ -125,7 +126,7 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState()
auto zookeeper = storage.getZooKeeper();
auto all_replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas");
std::sort(all_replicas.begin(), all_replicas.end());
::sort(all_replicas.begin(), all_replicas.end());
std::vector<String> active_replicas_tmp;
int current_replica_index_tmp = -1;

View File

@ -9,6 +9,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <Common/CurrentMetrics.h>
#include <Parsers/formatAST.h>
#include <base/sort.h>
namespace DB
@ -145,7 +146,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
LOG_DEBUG(log, "Having {} queue entries to load, {} entries already loaded.", (to_remove_it - children.begin()), (children.end() - to_remove_it));
children.erase(to_remove_it, children.end());
std::sort(children.begin(), children.end());
::sort(children.begin(), children.end());
zkutil::AsyncResponses<Coordination::GetResponse> futures;
futures.reserve(children.size());
@ -600,7 +601,7 @@ int32_t ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper
if (!log_entries.empty())
{
std::sort(log_entries.begin(), log_entries.end());
::sort(log_entries.begin(), log_entries.end());
for (size_t entry_idx = 0, num_entries = log_entries.size(); entry_idx < num_entries;)
{

View File

@ -1,5 +1,7 @@
#include "PostgreSQLReplicationHandler.h"
#include <base/sort.h>
#include <Common/setThreadName.h>
#include <Parsers/ASTTableOverrides.h>
#include <Processors/Transforms/PostgreSQLSource.h>
@ -698,7 +700,7 @@ std::set<String> PostgreSQLReplicationHandler::fetchRequiredTables()
}
NameSet diff;
std::sort(expected_tables.begin(), expected_tables.end());
::sort(expected_tables.begin(), expected_tables.end());
std::set_symmetric_difference(expected_tables.begin(), expected_tables.end(),
result_tables.begin(), result_tables.end(),
std::inserter(diff, diff.begin()));

View File

@ -29,6 +29,7 @@
#include <Poco/Logger.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <base/logger_useful.h>
#include <base/sort.h>
#include <rocksdb/db.h>
#include <rocksdb/table.h>
@ -457,7 +458,7 @@ Pipe StorageEmbeddedRocksDB::read(
if (keys->empty())
return {};
std::sort(keys->begin(), keys->end());
::sort(keys->begin(), keys->end());
keys->erase(std::unique(keys->begin(), keys->end()), keys->end());
Pipes pipes;

View File

@ -2,6 +2,8 @@
#include <optional>
#include <base/sort.h>
#include <Databases/IDatabase.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
@ -1184,7 +1186,7 @@ std::vector<StorageMergeTree::PartVersionWithName> StorageMergeTree::getSortedPa
getUpdatedDataVersion(part, currently_processing_in_background_mutex_lock),
part->name
});
std::sort(part_versions_with_names.begin(), part_versions_with_names.end());
::sort(part_versions_with_names.begin(), part_versions_with_names.end());
return part_versions_with_names;
}

View File

@ -10,6 +10,8 @@
#include <Common/thread_local_rng.h>
#include <Common/typeid_cast.h>
#include <base/sort.h>
#include <Storages/AlterCommands.h>
#include <Storages/PartitionCommands.h>
#include <Storages/ColumnsDescription.h>
@ -2304,7 +2306,7 @@ void StorageReplicatedMergeTree::cloneReplica(const String & source_replica, Coo
}
}
std::sort(source_queue_names.begin(), source_queue_names.end());
::sort(source_queue_names.begin(), source_queue_names.end());
struct QueueEntryInfo
{