Replaced Mersenne Twister to PCG-Random [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-09-10 02:17:38 +03:00
parent 5a8f47a640
commit cd540c2674
24 changed files with 62 additions and 58 deletions

View File

@ -206,6 +206,7 @@ target_link_libraries (dbms
target_include_directories (dbms BEFORE PRIVATE ${DIVIDE_INCLUDE_DIR}) target_include_directories (dbms BEFORE PRIVATE ${DIVIDE_INCLUDE_DIR})
target_include_directories (dbms BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) target_include_directories (dbms BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR})
target_include_directories (dbms PUBLIC ${DBMS_INCLUDE_DIR}) target_include_directories (dbms PUBLIC ${DBMS_INCLUDE_DIR})
target_include_directories (dbms PUBLIC ${PCG_RANDOM_INCLUDE_DIR})
# only for copy_headers.sh: # only for copy_headers.sh:
target_include_directories (dbms PRIVATE ${COMMON_INCLUDE_DIR}) target_include_directories (dbms PRIVATE ${COMMON_INCLUDE_DIR})

View File

@ -29,4 +29,3 @@ list(REMOVE_ITEM clickhouse_aggregate_functions_headers
add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_sources}) add_library(clickhouse_aggregate_functions ${clickhouse_aggregate_functions_sources})
target_link_libraries(clickhouse_aggregate_functions dbms) target_link_libraries(clickhouse_aggregate_functions dbms)
target_include_directories (clickhouse_aggregate_functions PRIVATE ${COMMON_INCLUDE_DIR}) target_include_directories (clickhouse_aggregate_functions PRIVATE ${COMMON_INCLUDE_DIR})
target_include_directories (clickhouse_aggregate_functions PRIVATE ${PCG_RANDOM_INCLUDE_DIR})

View File

@ -5,6 +5,7 @@
#include <list> #include <list>
#include <memory> #include <memory>
#include <random> #include <random>
#include <pcg_random.hpp>
#include <unordered_map> #include <unordered_map>
#include <sys/mman.h> #include <sys/mman.h>
#include <boost/intrusive/list.hpp> #include <boost/intrusive/list.hpp>
@ -160,7 +161,7 @@ private:
mutable std::mutex mutex; mutable std::mutex mutex;
std::mt19937_64 rng {static_cast<std::mt19937_64::result_type>(randomSeed())}; pcg64 rng{randomSeed()};
struct Chunk : private boost::noncopyable struct Chunk : private boost::noncopyable
{ {

View File

@ -1,9 +1,11 @@
#include <random> #include <random>
#include <pcg_random.hpp>
#include <functional> #include <functional>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/ProfileEvents.h> #include <Common/ProfileEvents.h>
#include <Common/StringUtils.h> #include <Common/StringUtils.h>
#include <Common/randomSeed.h>
namespace ProfileEvents namespace ProfileEvents
@ -137,9 +139,8 @@ struct ZooKeeperArgs
} }
/// Shuffle the hosts to distribute the load among ZooKeeper nodes. /// Shuffle the hosts to distribute the load among ZooKeeper nodes.
std::random_device rd; pcg64 rng(randomSeed());
std::mt19937 g(rd()); std::shuffle(hosts_strings.begin(), hosts_strings.end(), rng);
std::shuffle(hosts_strings.begin(), hosts_strings.end(), g);
for (auto & host : hosts_strings) for (auto & host : hosts_strings)
{ {

View File

@ -1,6 +1,7 @@
#include <iostream> #include <iostream>
#include <cstring> #include <cstring>
#include <thread> #include <thread>
#include <pcg_random.hpp>
#include <Common/ArrayCache.h> #include <Common/ArrayCache.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
@ -51,7 +52,7 @@ int main(int argc, char ** argv)
{ {
threads.emplace_back([&] threads.emplace_back([&]
{ {
std::mt19937 generator(randomSeed()); pcg64 generator(randomSeed());
for (size_t i = 0; i < num_iterations; ++i) for (size_t i = 0; i < num_iterations; ++i)
{ {

View File

@ -4,6 +4,7 @@
#include <common/Types.h> #include <common/Types.h>
#include <common/ThreadPool.h> #include <common/ThreadPool.h>
#include <random> #include <random>
#include <pcg_random.hpp>
#include <thread> #include <thread>
#include <atomic> #include <atomic>
#include <iomanip> #include <iomanip>
@ -23,7 +24,7 @@ TEST(Common, RWLockFIFO_1)
static auto fifo_lock = RWLockFIFO::create(); static auto fifo_lock = RWLockFIFO::create();
static thread_local std::random_device rd; 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) { auto func = [&] (size_t threads, int round) {
for (int i = 0; i < cycles; ++i) for (int i = 0; i < cycles; ++i)
@ -85,7 +86,7 @@ TEST(Common, RWLockFIFO_Recursive)
static auto fifo_lock = RWLockFIFO::create(); static auto fifo_lock = RWLockFIFO::create();
static thread_local std::random_device rd; static thread_local std::random_device rd;
static thread_local std::mt19937 gen(rd()); static thread_local pcg64 gen(rd());
std::thread t1([&] () { std::thread t1([&] () {
for (int i = 0; i < 2 * cycles; ++i) for (int i = 0; i < 2 * cycles; ++i)
@ -106,7 +107,7 @@ TEST(Common, RWLockFIFO_Recursive)
std::this_thread::sleep_for(sleep_for); std::this_thread::sleep_for(sleep_for);
auto lock2 = fifo_lock->getLock(RWLockFIFO::Read); auto lock2 = fifo_lock->getLock(RWLockFIFO::Read);
EXPECT_ANY_THROW({fifo_lock->getLock(RWLockFIFO::Write);}); EXPECT_ANY_THROW({fifo_lock->getLock(RWLockFIFO::Write);});
} }

View File

@ -1,4 +1,5 @@
#include <random> #include <random>
#include <pcg_random.hpp>
#include <Common/randomSeed.h> #include <Common/randomSeed.h>
#include <DataStreams/ConcatBlockInputStream.h> #include <DataStreams/ConcatBlockInputStream.h>
@ -20,7 +21,7 @@ BlockInputStreams narrowBlockInputStreams(BlockInputStreams & inputs, size_t wid
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)
distribution[i] = i % width; distribution[i] = i % width;
std::mt19937 generator(randomSeed()); pcg64 generator(randomSeed());
std::shuffle(distribution.begin(), distribution.end(), generator); std::shuffle(distribution.begin(), distribution.end(), generator);
for (size_t i = 0; i < size; ++i) for (size_t i = 0; i < size; ++i)

View File

@ -68,7 +68,7 @@ CacheDictionary::CacheDictionary(const std::string & name, const DictionaryStruc
size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))}, size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))},
size_overlap_mask{this->size - 1}, size_overlap_mask{this->size - 1},
cells{this->size}, cells{this->size},
rnd_engine{randomSeed()} rnd_engine(randomSeed())
{ {
if (!this->source_ptr->supportsSelectiveLoad()) if (!this->source_ptr->supportsSelectiveLoad())
throw Exception{ throw Exception{

View File

@ -13,7 +13,7 @@
#include <vector> #include <vector>
#include <map> #include <map>
#include <tuple> #include <tuple>
#include <random> #include <pcg_random.hpp>
#include <shared_mutex> #include <shared_mutex>
@ -260,7 +260,7 @@ private:
Attribute * hierarchical_attribute = nullptr; Attribute * hierarchical_attribute = nullptr;
std::unique_ptr<ArenaWithFreeLists> string_arena; std::unique_ptr<ArenaWithFreeLists> string_arena;
mutable std::mt19937_64 rnd_engine; mutable pcg64 rnd_engine;
mutable size_t bytes_allocated = 0; mutable size_t bytes_allocated = 0;
mutable std::atomic<size_t> element_count{0}; mutable std::atomic<size_t> element_count{0};

View File

@ -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), : 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{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))},
size_overlap_mask{this->size - 1}, size_overlap_mask{this->size - 1},
rnd_engine{randomSeed()} rnd_engine(randomSeed())
{ {
if (!this->source_ptr->supportsSelectiveLoad()) if (!this->source_ptr->supportsSelectiveLoad())
throw Exception{ throw Exception{

View File

@ -3,7 +3,6 @@
#include <atomic> #include <atomic>
#include <chrono> #include <chrono>
#include <map> #include <map>
#include <random>
#include <tuple> #include <tuple>
#include <vector> #include <vector>
#include <shared_mutex> #include <shared_mutex>
@ -19,6 +18,7 @@
#include <ext/bit_cast.h> #include <ext/bit_cast.h>
#include <ext/map.h> #include <ext/map.h>
#include <ext/scope_guard.h> #include <ext/scope_guard.h>
#include <pcg_random.hpp>
namespace ProfileEvents 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<SmallObjectPool> fixed_size_keys_pool = key_size_is_fixed ? std::make_unique<SmallObjectPool>(key_size) : nullptr;
std::unique_ptr<ArenaWithFreeLists> string_arena; std::unique_ptr<ArenaWithFreeLists> string_arena;
mutable std::mt19937_64 rnd_engine; mutable pcg64 rnd_engine;
mutable size_t bytes_allocated = 0; mutable size_t bytes_allocated = 0;
mutable std::atomic<size_t> element_count{0}; mutable std::atomic<size_t> element_count{0};

View File

@ -1,6 +1,5 @@
#include <map> #include <map>
#include <set> #include <set>
#include <random>
#include <boost/functional/hash/hash.hpp> #include <boost/functional/hash/hash.hpp>
#include <Poco/Mutex.h> #include <Poco/Mutex.h>
@ -9,6 +8,7 @@
#include <Poco/Net/IPAddress.h> #include <Poco/Net/IPAddress.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <pcg_random.hpp>
#include <Common/Macros.h> #include <Common/Macros.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
@ -175,7 +175,7 @@ struct ContextShared
Context::ApplicationType application_type = Context::ApplicationType::SERVER; Context::ApplicationType application_type = Context::ApplicationType::SERVER;
std::mt19937_64 rng{randomSeed()}; pcg64 rng{randomSeed()};
ContextShared() ContextShared()
{ {

View File

@ -21,6 +21,7 @@
#include <Common/getFQDNOrHostName.h> #include <Common/getFQDNOrHostName.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <Common/randomSeed.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
@ -34,6 +35,9 @@
#include <Common/isLocalAddress.h> #include <Common/isLocalAddress.h>
#include <Poco/Timestamp.h> #include <Poco/Timestamp.h>
#include <random>
#include <pcg_random.hpp>
namespace DB namespace DB
{ {
@ -655,7 +659,7 @@ void DDLWorker::processTaskAlter(
bool alter_executed_by_any_replica = false; bool alter_executed_by_any_replica = false;
{ {
auto lock = createSimpleZooKeeperLock(zookeeper, shard_path, "lock", task.host_id_str); 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) for (int num_tries = 0; num_tries < 10; ++num_tries)
{ {

View File

@ -13,7 +13,8 @@
#include <thread> #include <thread>
#include <unordered_map> #include <unordered_map>
#include <chrono> #include <chrono>
#include <random> #include <pcg_random.hpp>
namespace DB namespace DB
{ {
@ -74,7 +75,7 @@ private:
*/ */
std::unordered_map<std::string, std::chrono::system_clock::time_point> update_times; 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; Context & context;

View File

@ -8,6 +8,7 @@
#include <Interpreters/Quota.h> #include <Interpreters/Quota.h>
#include <set> #include <set>
#include <random>
namespace DB 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; Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_elem, 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_; name = name_;
@ -308,7 +309,7 @@ QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_na
void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config) void Quotas::loadFromConfig(Poco::Util::AbstractConfiguration & config)
{ {
std::mt19937 rng; pcg64 rng;
Poco::Util::AbstractConfiguration::Keys config_keys; Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys("quotas", config_keys); config.keys("quotas", config_keys);

View File

@ -3,7 +3,7 @@
#include <cstring> #include <cstring>
#include <unordered_map> #include <unordered_map>
#include <memory> #include <memory>
#include <random> #include <pcg_random.hpp>
#include <Poco/Timespan.h> #include <Poco/Timespan.h>
@ -191,7 +191,7 @@ public:
return cont.empty(); 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. /// Set maximum values (limits) from passed argument.
/// Remove intervals that does not exist in argument. Add intervals from argument, that we don't have. /// 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; 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); QuotaForIntervalsPtr get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip);
}; };

View File

@ -8,6 +8,7 @@
#include <fstream> #include <fstream>
#include <iomanip> #include <iomanip>
#include <random> #include <random>
#include <pcg_random.hpp>
#include <Poco/File.h> #include <Poco/File.h>
#include <Poco/Util/Application.h> #include <Poco/Util/Application.h>
@ -230,7 +231,7 @@ private:
void run() void run()
{ {
std::mt19937 generator(randomSeed()); pcg64 generator(randomSeed());
std::uniform_int_distribution<size_t> distribution(0, queries.size() - 1); std::uniform_int_distribution<size_t> distribution(0, queries.size() - 1);
for (size_t i = 0; i < concurrency; ++i) for (size_t i = 0; i < concurrency; ++i)

View File

@ -2,10 +2,12 @@
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Common/MemoryTracker.h> #include <Common/MemoryTracker.h>
#include <Common/randomSeed.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Storages/MergeTree/BackgroundProcessingPool.h> #include <Storages/MergeTree/BackgroundProcessingPool.h>
#include <pcg_random.hpp>
#include <random> #include <random>
@ -115,7 +117,7 @@ void BackgroundProcessingPool::threadFunction()
memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool);
current_memory_tracker = &memory_tracker; 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))); std::this_thread::sleep_for(std::chrono::duration<double>(std::uniform_real_distribution<double>(0, sleep_seconds_random_part)(rng)));
while (!shutdown) while (!shutdown)

View File

@ -31,11 +31,14 @@
#include <Poco/DirectoryIterator.h> #include <Poco/DirectoryIterator.h>
#include <Poco/File.h> #include <Poco/File.h>
#include <boost/noncopyable.hpp>
#include <future> #include <future>
#include <chrono> #include <chrono>
#include <cstdlib> #include <cstdlib>
#include <ctime> #include <ctime>
#include <random> #include <pcg_random.hpp>
namespace DB namespace DB
{ {
@ -1391,19 +1394,8 @@ void ReshardingWorker::executeAttach(LogRecord & log_record)
/// Description of tasks for each replica of a shard. /// Description of tasks for each replica of a shard.
/// For fault tolerance purposes, some fields are provided /// For fault tolerance purposes, some fields are provided
/// to perform attempts on more than one replica if needed. /// 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 /// one task for each replica
std::vector<TaskInfo> shard_tasks; std::vector<TaskInfo> shard_tasks;
/// index to the replica to be used /// 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 /// result of the operation on the current replica
bool is_success = false; bool is_success = false;
/// For pseudo-random number generation. /// For pseudo-random number generation.
std::mt19937 rng; pcg64 rng{randomSeed()};
}; };
const WeightedZooKeeperPath & weighted_path = current_job.paths[log_record.shard_no]; const WeightedZooKeeperPath & weighted_path = current_job.paths[log_record.shard_no];

View File

@ -2,7 +2,7 @@
#include <ext/shared_ptr_helper.h> #include <ext/shared_ptr_helper.h>
#include <atomic> #include <atomic>
#include <random> #include <pcg_random.hpp>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Storages/MergeTree/MergeTreeData.h> #include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeDataMerger.h> #include <Storages/MergeTree/MergeTreeDataMerger.h>
@ -321,7 +321,7 @@ private:
Logger * log; Logger * log;
std::mt19937 rng{randomSeed()}; pcg64 rng{randomSeed()};
StorageReplicatedMergeTree( StorageReplicatedMergeTree(
const String & zookeeper_path_, const String & zookeeper_path_,

View File

@ -7,12 +7,14 @@
#include <iomanip> #include <iomanip>
#include <vector> #include <vector>
#include <random> #include <random>
#include <pcg_random.hpp>
#include <Poco/NumberParser.h> #include <Poco/NumberParser.h>
#include <Poco/NumberFormatter.h> #include <Poco/NumberFormatter.h>
#include <Poco/Exception.h> #include <Poco/Exception.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/randomSeed.h>
#include <common/ThreadPool.h> #include <common/ThreadPool.h>
#include <Common/Stopwatch.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 else
buf = &simple_buf[0]; buf = &simple_buf[0];
std::mt19937 rng; pcg64 rng(randomSeed());
timespec times;
clock_gettime(CLOCK_THREAD_CPUTIME_ID, &times);
rng.seed(times.tv_nsec);
for (size_t i = 0; i < count; ++i) for (size_t i = 0; i < count; ++i)
{ {

View File

@ -13,12 +13,14 @@
#include <vector> #include <vector>
#include <random> #include <random>
#include <pcg_random.hpp>
#include <Poco/NumberParser.h> #include <Poco/NumberParser.h>
#include <Poco/NumberFormatter.h> #include <Poco/NumberFormatter.h>
#include <Poco/Exception.h> #include <Poco/Exception.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/randomSeed.h>
#include <common/ThreadPool.h> #include <common/ThreadPool.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
@ -77,11 +79,7 @@ int mainImpl(int argc, char ** argv)
std::vector<char> buf(block_size); std::vector<char> buf(block_size);
std::mt19937 rng; pcg64 rng(randomSeed());
timespec times;
clock_gettime(CLOCK_THREAD_CPUTIME_ID, &times);
rng.seed(times.tv_nsec);
Stopwatch watch; Stopwatch watch;

View File

@ -1,5 +1,6 @@
#include <iostream> #include <iostream>
#include <random> #include <random>
#include <pcg_random.hpp>
#include <cmath> #include <cmath>
#include <IO/WriteBufferFromFileDescriptor.h> #include <IO/WriteBufferFromFileDescriptor.h>
@ -42,8 +43,8 @@ struct Models
struct Generator struct Generator
{ {
WriteBufferFromFileDescriptor out; WriteBufferFromFileDescriptor out;
std::mt19937_64 random; pcg64 random;
std::mt19937_64 random_with_seed; pcg64 random_with_seed;
Models models; Models models;
// UInt64 WatchID = random(); // UInt64 WatchID = random();

View File

@ -1,5 +1,6 @@
#include <iostream> #include <iostream>
#include <random> #include <random>
#include <pcg_random.hpp>
#include <boost/program_options.hpp> #include <boost/program_options.hpp>
@ -47,7 +48,7 @@ try
ReadBufferFromFileDescriptor in(STDIN_FILENO); ReadBufferFromFileDescriptor in(STDIN_FILENO);
WriteBufferFromFileDescriptor out(STDOUT_FILENO); WriteBufferFromFileDescriptor out(STDOUT_FILENO);
std::mt19937 random; pcg64 random;
if (options.count("seed")) if (options.count("seed"))
random.seed(options["seed"].as<UInt64>()); random.seed(options["seed"].as<UInt64>());