mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Replaced Mersenne Twister to PCG-Random [#CLICKHOUSE-2].
This commit is contained in:
parent
5a8f47a640
commit
cd540c2674
@ -206,6 +206,7 @@ target_link_libraries (dbms
|
||||
target_include_directories (dbms BEFORE PRIVATE ${DIVIDE_INCLUDE_DIR})
|
||||
target_include_directories (dbms BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
|
||||
target_include_directories (dbms PUBLIC ${DBMS_INCLUDE_DIR})
|
||||
target_include_directories (dbms PUBLIC ${PCG_RANDOM_INCLUDE_DIR})
|
||||
|
||||
# only for copy_headers.sh:
|
||||
target_include_directories (dbms PRIVATE ${COMMON_INCLUDE_DIR})
|
||||
|
@ -29,4 +29,3 @@ list(REMOVE_ITEM clickhouse_aggregate_functions_headers
|
||||
add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_sources})
|
||||
target_link_libraries(clickhouse_aggregate_functions dbms)
|
||||
target_include_directories (clickhouse_aggregate_functions PRIVATE ${COMMON_INCLUDE_DIR})
|
||||
target_include_directories (clickhouse_aggregate_functions PRIVATE ${PCG_RANDOM_INCLUDE_DIR})
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <list>
|
||||
#include <memory>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
#include <unordered_map>
|
||||
#include <sys/mman.h>
|
||||
#include <boost/intrusive/list.hpp>
|
||||
@ -160,7 +161,7 @@ private:
|
||||
|
||||
mutable std::mutex mutex;
|
||||
|
||||
std::mt19937_64 rng {static_cast<std::mt19937_64::result_type>(randomSeed())};
|
||||
pcg64 rng{randomSeed()};
|
||||
|
||||
struct Chunk : private boost::noncopyable
|
||||
{
|
||||
|
@ -1,9 +1,11 @@
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
#include <functional>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/StringUtils.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -137,9 +139,8 @@ struct ZooKeeperArgs
|
||||
}
|
||||
|
||||
/// Shuffle the hosts to distribute the load among ZooKeeper nodes.
|
||||
std::random_device rd;
|
||||
std::mt19937 g(rd());
|
||||
std::shuffle(hosts_strings.begin(), hosts_strings.end(), g);
|
||||
pcg64 rng(randomSeed());
|
||||
std::shuffle(hosts_strings.begin(), hosts_strings.end(), rng);
|
||||
|
||||
for (auto & host : hosts_strings)
|
||||
{
|
||||
|
@ -1,6 +1,7 @@
|
||||
#include <iostream>
|
||||
#include <cstring>
|
||||
#include <thread>
|
||||
#include <pcg_random.hpp>
|
||||
#include <Common/ArrayCache.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
@ -51,7 +52,7 @@ int main(int argc, char ** argv)
|
||||
{
|
||||
threads.emplace_back([&]
|
||||
{
|
||||
std::mt19937 generator(randomSeed());
|
||||
pcg64 generator(randomSeed());
|
||||
|
||||
for (size_t i = 0; i < num_iterations; ++i)
|
||||
{
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <common/Types.h>
|
||||
#include <common/ThreadPool.h>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
#include <thread>
|
||||
#include <atomic>
|
||||
#include <iomanip>
|
||||
@ -23,7 +24,7 @@ TEST(Common, RWLockFIFO_1)
|
||||
static auto fifo_lock = RWLockFIFO::create();
|
||||
|
||||
static thread_local std::random_device rd;
|
||||
static thread_local std::mt19937 gen(rd());
|
||||
static thread_local pcg64 gen(rd());
|
||||
|
||||
auto func = [&] (size_t threads, int round) {
|
||||
for (int i = 0; i < cycles; ++i)
|
||||
@ -85,7 +86,7 @@ TEST(Common, RWLockFIFO_Recursive)
|
||||
static auto fifo_lock = RWLockFIFO::create();
|
||||
|
||||
static thread_local std::random_device rd;
|
||||
static thread_local std::mt19937 gen(rd());
|
||||
static thread_local pcg64 gen(rd());
|
||||
|
||||
std::thread t1([&] () {
|
||||
for (int i = 0; i < 2 * cycles; ++i)
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <DataStreams/ConcatBlockInputStream.h>
|
||||
|
||||
@ -20,7 +21,7 @@ BlockInputStreams narrowBlockInputStreams(BlockInputStreams & inputs, size_t wid
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
distribution[i] = i % width;
|
||||
|
||||
std::mt19937 generator(randomSeed());
|
||||
pcg64 generator(randomSeed());
|
||||
std::shuffle(distribution.begin(), distribution.end(), generator);
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
|
@ -68,7 +68,7 @@ CacheDictionary::CacheDictionary(const std::string & name, const DictionaryStruc
|
||||
size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))},
|
||||
size_overlap_mask{this->size - 1},
|
||||
cells{this->size},
|
||||
rnd_engine{randomSeed()}
|
||||
rnd_engine(randomSeed())
|
||||
{
|
||||
if (!this->source_ptr->supportsSelectiveLoad())
|
||||
throw Exception{
|
||||
|
@ -13,7 +13,7 @@
|
||||
#include <vector>
|
||||
#include <map>
|
||||
#include <tuple>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
#include <shared_mutex>
|
||||
|
||||
|
||||
@ -260,7 +260,7 @@ private:
|
||||
Attribute * hierarchical_attribute = nullptr;
|
||||
std::unique_ptr<ArenaWithFreeLists> string_arena;
|
||||
|
||||
mutable std::mt19937_64 rnd_engine;
|
||||
mutable pcg64 rnd_engine;
|
||||
|
||||
mutable size_t bytes_allocated = 0;
|
||||
mutable std::atomic<size_t> element_count{0};
|
||||
|
@ -57,7 +57,7 @@ ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const std::string & name, c
|
||||
: name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime),
|
||||
size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))},
|
||||
size_overlap_mask{this->size - 1},
|
||||
rnd_engine{randomSeed()}
|
||||
rnd_engine(randomSeed())
|
||||
{
|
||||
if (!this->source_ptr->supportsSelectiveLoad())
|
||||
throw Exception{
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <atomic>
|
||||
#include <chrono>
|
||||
#include <map>
|
||||
#include <random>
|
||||
#include <tuple>
|
||||
#include <vector>
|
||||
#include <shared_mutex>
|
||||
@ -19,6 +18,7 @@
|
||||
#include <ext/bit_cast.h>
|
||||
#include <ext/map.h>
|
||||
#include <ext/scope_guard.h>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
@ -721,7 +721,7 @@ private:
|
||||
std::unique_ptr<SmallObjectPool> fixed_size_keys_pool = key_size_is_fixed ? std::make_unique<SmallObjectPool>(key_size) : nullptr;
|
||||
std::unique_ptr<ArenaWithFreeLists> string_arena;
|
||||
|
||||
mutable std::mt19937_64 rnd_engine;
|
||||
mutable pcg64 rnd_engine;
|
||||
|
||||
mutable size_t bytes_allocated = 0;
|
||||
mutable std::atomic<size_t> element_count{0};
|
||||
|
@ -1,6 +1,5 @@
|
||||
#include <map>
|
||||
#include <set>
|
||||
#include <random>
|
||||
|
||||
#include <boost/functional/hash/hash.hpp>
|
||||
#include <Poco/Mutex.h>
|
||||
@ -9,6 +8,7 @@
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
#include <Common/Macros.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
@ -175,7 +175,7 @@ struct ContextShared
|
||||
|
||||
Context::ApplicationType application_type = Context::ApplicationType::SERVER;
|
||||
|
||||
std::mt19937_64 rng{randomSeed()};
|
||||
pcg64 rng{randomSeed()};
|
||||
|
||||
ContextShared()
|
||||
{
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include <Common/getFQDNOrHostName.h>
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
@ -34,6 +35,9 @@
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <Poco/Timestamp.h>
|
||||
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -655,7 +659,7 @@ void DDLWorker::processTaskAlter(
|
||||
bool alter_executed_by_any_replica = false;
|
||||
{
|
||||
auto lock = createSimpleZooKeeperLock(zookeeper, shard_path, "lock", task.host_id_str);
|
||||
std::mt19937 rng(StringRefHash{}(task.host_id_str) + reinterpret_cast<intptr_t>(&rng));
|
||||
pcg64 rng(randomSeed());
|
||||
|
||||
for (int num_tries = 0; num_tries < 10; ++num_tries)
|
||||
{
|
||||
|
@ -13,7 +13,8 @@
|
||||
#include <thread>
|
||||
#include <unordered_map>
|
||||
#include <chrono>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -74,7 +75,7 @@ private:
|
||||
*/
|
||||
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times;
|
||||
|
||||
std::mt19937_64 rnd_engine{randomSeed()};
|
||||
pcg64 rnd_engine{randomSeed()};
|
||||
|
||||
Context & context;
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Interpreters/Quota.h>
|
||||
|
||||
#include <set>
|
||||
#include <random>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -159,7 +160,7 @@ void QuotaForInterval::check(
|
||||
}
|
||||
|
||||
|
||||
void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng)
|
||||
void QuotaForIntervals::initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, pcg64 & rng)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys(config_elem, config_keys);
|
||||
@ -250,7 +251,7 @@ String QuotaForIntervals::toString() const
|
||||
}
|
||||
|
||||
|
||||
void Quota::loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng)
|
||||
void Quota::loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, pcg64 & rng)
|
||||
{
|
||||
name = name_;
|
||||
|
||||
@ -308,7 +309,7 @@ QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_na
|
||||
|
||||
void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
std::mt19937 rng;
|
||||
pcg64 rng;
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys config_keys;
|
||||
config.keys("quotas", config_keys);
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <cstring>
|
||||
#include <unordered_map>
|
||||
#include <memory>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
#include <Poco/Timespan.h>
|
||||
|
||||
@ -191,7 +191,7 @@ public:
|
||||
return cont.empty();
|
||||
}
|
||||
|
||||
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng);
|
||||
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config, pcg64 & rng);
|
||||
|
||||
/// Set maximum values (limits) from passed argument.
|
||||
/// Remove intervals that does not exist in argument. Add intervals from argument, that we don't have.
|
||||
@ -241,7 +241,7 @@ struct Quota
|
||||
|
||||
bool keyed_by_ip = false;
|
||||
|
||||
void loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, std::mt19937 & rng);
|
||||
void loadFromConfig(const String & config_elem, const String & name_, Poco::Util::AbstractConfiguration & config, pcg64 & rng);
|
||||
QuotaForIntervalsPtr get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip);
|
||||
};
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <fstream>
|
||||
#include <iomanip>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <Poco/Util/Application.h>
|
||||
@ -230,7 +231,7 @@ private:
|
||||
|
||||
void run()
|
||||
{
|
||||
std::mt19937 generator(randomSeed());
|
||||
pcg64 generator(randomSeed());
|
||||
std::uniform_int_distribution<size_t> distribution(0, queries.size() - 1);
|
||||
|
||||
for (size_t i = 0; i < concurrency; ++i)
|
||||
|
@ -2,10 +2,12 @@
|
||||
#include <Common/setThreadName.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/randomSeed.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <Storages/MergeTree/BackgroundProcessingPool.h>
|
||||
|
||||
#include <pcg_random.hpp>
|
||||
#include <random>
|
||||
|
||||
|
||||
@ -115,7 +117,7 @@ void BackgroundProcessingPool::threadFunction()
|
||||
memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool);
|
||||
current_memory_tracker = &memory_tracker;
|
||||
|
||||
std::mt19937 rng(reinterpret_cast<intptr_t>(&rng));
|
||||
pcg64 rng(randomSeed());
|
||||
std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
|
||||
|
||||
while (!shutdown)
|
||||
|
@ -31,11 +31,14 @@
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
#include <future>
|
||||
#include <chrono>
|
||||
#include <cstdlib>
|
||||
#include <ctime>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -1391,19 +1394,8 @@ void ReshardingWorker::executeAttach(LogRecord & log_record)
|
||||
/// Description of tasks for each replica of a shard.
|
||||
/// For fault tolerance purposes, some fields are provided
|
||||
/// to perform attempts on more than one replica if needed.
|
||||
struct ShardTaskInfo
|
||||
struct ShardTaskInfo : private boost::noncopyable
|
||||
{
|
||||
ShardTaskInfo()
|
||||
{
|
||||
rng = std::mt19937(randomSeed());
|
||||
}
|
||||
|
||||
ShardTaskInfo(const ShardTaskInfo &) = delete;
|
||||
ShardTaskInfo & operator=(const ShardTaskInfo &) = delete;
|
||||
|
||||
ShardTaskInfo(ShardTaskInfo &&) = default;
|
||||
ShardTaskInfo & operator=(ShardTaskInfo &&) = default;
|
||||
|
||||
/// one task for each replica
|
||||
std::vector<TaskInfo> shard_tasks;
|
||||
/// index to the replica to be used
|
||||
@ -1411,7 +1403,7 @@ void ReshardingWorker::executeAttach(LogRecord & log_record)
|
||||
/// result of the operation on the current replica
|
||||
bool is_success = false;
|
||||
/// For pseudo-random number generation.
|
||||
std::mt19937 rng;
|
||||
pcg64 rng{randomSeed()};
|
||||
};
|
||||
|
||||
const WeightedZooKeeperPath & weighted_path = current_job.paths[log_record.shard_no];
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <ext/shared_ptr_helper.h>
|
||||
#include <atomic>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/MergeTreeData.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataMerger.h>
|
||||
@ -321,7 +321,7 @@ private:
|
||||
|
||||
Logger * log;
|
||||
|
||||
std::mt19937 rng{randomSeed()};
|
||||
pcg64 rng{randomSeed()};
|
||||
|
||||
StorageReplicatedMergeTree(
|
||||
const String & zookeeper_path_,
|
||||
|
@ -7,12 +7,14 @@
|
||||
#include <iomanip>
|
||||
#include <vector>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
#include <Poco/NumberParser.h>
|
||||
#include <Poco/NumberFormatter.h>
|
||||
#include <Poco/Exception.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
||||
#include <common/ThreadPool.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
@ -50,11 +52,7 @@ void thread(int fd, int mode, size_t min_offset, size_t max_offset, size_t block
|
||||
else
|
||||
buf = &simple_buf[0];
|
||||
|
||||
std::mt19937 rng;
|
||||
|
||||
timespec times;
|
||||
clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×);
|
||||
rng.seed(times.tv_nsec);
|
||||
pcg64 rng(randomSeed());
|
||||
|
||||
for (size_t i = 0; i < count; ++i)
|
||||
{
|
||||
|
@ -13,12 +13,14 @@
|
||||
#include <vector>
|
||||
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
#include <Poco/NumberParser.h>
|
||||
#include <Poco/NumberFormatter.h>
|
||||
#include <Poco/Exception.h>
|
||||
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/randomSeed.h>
|
||||
|
||||
#include <common/ThreadPool.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
@ -77,11 +79,7 @@ int mainImpl(int argc, char ** argv)
|
||||
|
||||
std::vector<char> buf(block_size);
|
||||
|
||||
std::mt19937 rng;
|
||||
|
||||
timespec times;
|
||||
clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×);
|
||||
rng.seed(times.tv_nsec);
|
||||
pcg64 rng(randomSeed());
|
||||
|
||||
Stopwatch watch;
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <iostream>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
#include <cmath>
|
||||
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
@ -42,8 +43,8 @@ struct Models
|
||||
struct Generator
|
||||
{
|
||||
WriteBufferFromFileDescriptor out;
|
||||
std::mt19937_64 random;
|
||||
std::mt19937_64 random_with_seed;
|
||||
pcg64 random;
|
||||
pcg64 random_with_seed;
|
||||
Models models;
|
||||
|
||||
// UInt64 WatchID = random();
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <iostream>
|
||||
#include <random>
|
||||
#include <pcg_random.hpp>
|
||||
|
||||
#include <boost/program_options.hpp>
|
||||
|
||||
@ -47,7 +48,7 @@ try
|
||||
ReadBufferFromFileDescriptor in(STDIN_FILENO);
|
||||
WriteBufferFromFileDescriptor out(STDOUT_FILENO);
|
||||
|
||||
std::mt19937 random;
|
||||
pcg64 random;
|
||||
|
||||
if (options.count("seed"))
|
||||
random.seed(options["seed"].as<UInt64>());
|
||||
|
Loading…
Reference in New Issue
Block a user