mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Replace std::iota with DB::iota where possible
This commit is contained in:
parent
e692b0a5bd
commit
bda6104f84
@ -14,8 +14,9 @@
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/iota.h>
|
||||
#include <base/types.h>
|
||||
|
||||
#include <boost/math/distributions/normal.hpp>
|
||||
@ -48,7 +49,7 @@ struct LargestTriangleThreeBucketsData : public StatisticalSample<Float64, Float
|
||||
// sort the this->x and this->y in ascending order of this->x using index
|
||||
std::vector<size_t> index(this->x.size());
|
||||
|
||||
std::iota(index.begin(), index.end(), 0);
|
||||
iota(index.data(), index.size(), size_t(0));
|
||||
::sort(index.begin(), index.end(), [&](size_t i1, size_t i2) { return this->x[i1] < this->x[i2]; });
|
||||
|
||||
SampleX temp_x{};
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Common/ArenaAllocator.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -30,7 +31,7 @@ std::pair<RanksArray, Float64> computeRanksAndTieCorrection(const Values & value
|
||||
const size_t size = values.size();
|
||||
/// Save initial positions, than sort indices according to the values.
|
||||
std::vector<size_t> indexes(size);
|
||||
std::iota(indexes.begin(), indexes.end(), 0);
|
||||
iota(indexes.data(), indexes.size(), size_t(0));
|
||||
std::sort(indexes.begin(), indexes.end(),
|
||||
[&] (size_t lhs, size_t rhs) { return values[lhs] < values[rhs]; });
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Analyzer/Passes/FuseFunctionsPass.h>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
@ -184,7 +185,7 @@ FunctionNodePtr createFusedQuantilesNode(std::vector<QueryTreeNodePtr *> & nodes
|
||||
{
|
||||
/// Sort nodes and parameters in ascending order of quantile level
|
||||
std::vector<size_t> permutation(nodes.size());
|
||||
std::iota(permutation.begin(), permutation.end(), 0);
|
||||
iota(permutation.data(), permutation.size(), size_t(0));
|
||||
std::sort(permutation.begin(), permutation.end(), [&](size_t i, size_t j) { return parameters[i].get<Float64>() < parameters[j].get<Float64>(); });
|
||||
|
||||
std::vector<QueryTreeNodePtr *> new_nodes;
|
||||
|
@ -2,6 +2,7 @@
|
||||
#include <Columns/ColumnObject.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Common/iota.h>
|
||||
#include <DataTypes/ObjectUtils.h>
|
||||
#include <DataTypes/getLeastSupertype.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
@ -838,7 +839,7 @@ MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const
|
||||
void ColumnObject::getPermutation(PermutationSortDirection, PermutationSortStability, size_t, int, Permutation & res) const
|
||||
{
|
||||
res.resize(num_rows);
|
||||
std::iota(res.begin(), res.end(), 0);
|
||||
iota(res.data(), res.size(), size_t(0));
|
||||
}
|
||||
|
||||
void ColumnObject::compareColumn(const IColumn & rhs, size_t rhs_row_num,
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <Columns/ColumnSparse.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <pcg_random.hpp>
|
||||
#include <gtest/gtest.h>
|
||||
@ -191,7 +192,7 @@ TEST(ColumnSparse, Permute)
|
||||
auto [sparse_src, full_src] = createColumns(n, k);
|
||||
|
||||
IColumn::Permutation perm(n);
|
||||
std::iota(perm.begin(), perm.end(), 0);
|
||||
iota(perm.data(), perm.size(), size_t(0));
|
||||
std::shuffle(perm.begin(), perm.end(), rng);
|
||||
|
||||
auto sparse_dst = sparse_src->permute(perm, limit);
|
||||
|
@ -1,6 +1,33 @@
|
||||
#include <base/defines.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/TargetSpecific.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
MULTITARGET_FUNCTION_AVX2_SSE42(
|
||||
MULTITARGET_FUNCTION_HEADER(template <iota_supported_types T> void NO_INLINE),
|
||||
iotaImpl, MULTITARGET_FUNCTION_BODY((T * begin, size_t count, T first_value) /// NOLINT
|
||||
{
|
||||
for (size_t i = 0; i < count; i++)
|
||||
*(begin + i) = static_cast<T>(first_value + i);
|
||||
})
|
||||
)
|
||||
|
||||
template <iota_supported_types T>
|
||||
void iota(T * begin, size_t count, T first_value)
|
||||
{
|
||||
#if USE_MULTITARGET_CODE
|
||||
if (isArchSupported(TargetArch::AVX2))
|
||||
return iotaImplAVX2(begin, count, first_value);
|
||||
|
||||
if (isArchSupported(TargetArch::SSE42))
|
||||
return iotaImplSSE42(begin, count, first_value);
|
||||
#endif
|
||||
return iotaImpl(begin, count, first_value);
|
||||
}
|
||||
|
||||
template void iota(UInt8 * begin, size_t count, UInt8 first_value);
|
||||
template void iota(UInt32 * begin, size_t count, UInt32 first_value);
|
||||
template void iota(UInt64 * begin, size_t count, UInt64 first_value);
|
||||
}
|
||||
|
@ -1,9 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/defines.h>
|
||||
#include <base/types.h>
|
||||
#include <Common/Concepts.h>
|
||||
#include <Common/TargetSpecific.h>
|
||||
|
||||
/// This is a replacement for std::iota to use dynamic dispatch
|
||||
/// Note that is only defined for containers with contiguous memory only
|
||||
@ -13,30 +11,12 @@ namespace DB
|
||||
|
||||
/// Make sure to add any new type to the extern declaration at the end of the file and instantiate it in iota.cpp
|
||||
template <typename T>
|
||||
concept iota_supported_types = (is_any_of<T, UInt64>);
|
||||
concept iota_supported_types = (is_any_of<T, UInt8, UInt32, UInt64>);
|
||||
|
||||
MULTITARGET_FUNCTION_AVX2_SSE42(
|
||||
MULTITARGET_FUNCTION_HEADER(template <iota_supported_types T> void NO_INLINE),
|
||||
iotaImpl, MULTITARGET_FUNCTION_BODY((T * begin, size_t count, T first_value) /// NOLINT
|
||||
{
|
||||
for (size_t i = 0; i < count; i++)
|
||||
*(begin + i) = first_value + i;
|
||||
})
|
||||
)
|
||||
|
||||
template <iota_supported_types T>
|
||||
void iota(T * begin, size_t count, T first_value)
|
||||
{
|
||||
#if USE_MULTITARGET_CODE
|
||||
if (isArchSupported(TargetArch::AVX2))
|
||||
return iotaImplAVX2(begin, count, first_value);
|
||||
|
||||
if (isArchSupported(TargetArch::SSE42))
|
||||
return iotaImplSSE42(begin, count, first_value);
|
||||
#endif
|
||||
return iotaImpl(begin, count, first_value);
|
||||
}
|
||||
template <iota_supported_types T> void iota(T * begin, size_t count, T first_value);
|
||||
|
||||
extern template void iota(UInt8 * begin, size_t count, UInt8 first_value);
|
||||
extern template void iota(UInt32 * begin, size_t count, UInt32 first_value);
|
||||
extern template void iota(UInt64 * begin, size_t count, UInt64 first_value);
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/HashTable/HashSet.h>
|
||||
#include <Common/HashTable/Hash.h>
|
||||
#include <Common/iota.h>
|
||||
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
@ -20,7 +21,7 @@ namespace
|
||||
std::vector<UInt64> getVectorWithNumbersUpToN(size_t n)
|
||||
{
|
||||
std::vector<UInt64> res(n);
|
||||
std::iota(res.begin(), res.end(), 0);
|
||||
iota(res.data(), res.size(), size_t(0));
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Dictionaries/IDictionary.h>
|
||||
#include <Common/CurrentThread.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/scope_guard_safe.h>
|
||||
#include <Common/ConcurrentBoundedQueue.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
@ -53,7 +54,7 @@ public:
|
||||
LOG_TRACE(dictionary.log, "Will load the dictionary using {} threads (with {} backlog)", shards, backlog);
|
||||
|
||||
shards_slots.resize(shards);
|
||||
std::iota(shards_slots.begin(), shards_slots.end(), 0);
|
||||
iota(shards_slots.data(), shards_slots.size(), UInt64(0));
|
||||
|
||||
for (size_t shard = 0; shard < shards; ++shard)
|
||||
{
|
||||
|
@ -5,6 +5,7 @@
|
||||
|
||||
#include <base/sort.h>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
@ -507,7 +508,7 @@ const IColumn * unrollSimplePolygons(const ColumnPtr & column, Offset & offset)
|
||||
if (!ptr_polygons)
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected a column containing arrays of points");
|
||||
offset.ring_offsets.assign(ptr_polygons->getOffsets());
|
||||
std::iota(offset.polygon_offsets.begin(), offset.polygon_offsets.end(), 1);
|
||||
iota<IColumn::Offsets::value_type>(offset.polygon_offsets.data(), offset.polygon_offsets.size(), IColumn::Offsets::value_type(1));
|
||||
offset.multi_polygon_offsets.assign(offset.polygon_offsets);
|
||||
|
||||
return ptr_polygons->getDataPtr().get();
|
||||
|
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
@ -184,7 +185,7 @@ public:
|
||||
{
|
||||
setBoundingBox();
|
||||
std::vector<size_t> order(polygons.size());
|
||||
std::iota(order.begin(), order.end(), 0);
|
||||
iota(order.data(), order.size(), size_t(0));
|
||||
root = makeCell(min_x, min_y, max_x, max_y, order);
|
||||
}
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
@ -80,7 +81,7 @@ public:
|
||||
const size_t cur_samples = std::min(num_elements, samples);
|
||||
|
||||
indices.resize(num_elements);
|
||||
std::iota(indices.begin(), indices.end(), prev_array_offset);
|
||||
iota(indices.data(), indices.size(), prev_array_offset);
|
||||
std::shuffle(indices.begin(), indices.end(), rng);
|
||||
|
||||
for (UInt64 i = 0; i < cur_samples; i++)
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/shuffle.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -150,7 +151,7 @@ ColumnPtr FunctionArrayShuffleImpl<Traits>::executeGeneric(const ColumnArray & a
|
||||
size_t size = offsets.size();
|
||||
size_t nested_size = array.getData().size();
|
||||
IColumn::Permutation permutation(nested_size);
|
||||
std::iota(std::begin(permutation), std::end(permutation), 0);
|
||||
iota(permutation.data(), permutation.size(), IColumn::Permutation::value_type(0));
|
||||
|
||||
ColumnArray::Offset current_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Common/iota.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
@ -31,7 +32,7 @@ struct TranslateImpl
|
||||
if (map_from.size() != map_to.size())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second and third arguments must be the same length");
|
||||
|
||||
std::iota(map.begin(), map.end(), 0);
|
||||
iota(map.data(), map.size(), UInt8(0));
|
||||
|
||||
for (size_t i = 0; i < map_from.size(); ++i)
|
||||
{
|
||||
@ -129,7 +130,7 @@ struct TranslateUTF8Impl
|
||||
if (map_from_size != map_to_size)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Second and third arguments must be the same length");
|
||||
|
||||
std::iota(map_ascii.begin(), map_ascii.end(), 0);
|
||||
iota(map_ascii.data(), map_ascii.size(), UInt32(0));
|
||||
|
||||
const UInt8 * map_from_ptr = reinterpret_cast<const UInt8 *>(map_from.data());
|
||||
const UInt8 * map_from_end = map_from_ptr + map_from.size();
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <memory>
|
||||
#include <thread>
|
||||
|
||||
#include <Common/iota.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
@ -788,7 +789,7 @@ TEST_F(FileCacheTest, writeBuffer)
|
||||
|
||||
/// get random permutation of indexes
|
||||
std::vector<size_t> indexes(data.size());
|
||||
std::iota(indexes.begin(), indexes.end(), 0);
|
||||
iota(indexes.data(), indexes.size(), size_t(0));
|
||||
std::shuffle(indexes.begin(), indexes.end(), rng);
|
||||
|
||||
for (auto i : indexes)
|
||||
|
Loading…
Reference in New Issue
Block a user