diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 0feacf7f849..dbf23edc479 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -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}) diff --git a/dbms/src/AggregateFunctions/CMakeLists.txt b/dbms/src/AggregateFunctions/CMakeLists.txt index 4414d89f4a0..f3fb20b6101 100644 --- a/dbms/src/AggregateFunctions/CMakeLists.txt +++ b/dbms/src/AggregateFunctions/CMakeLists.txt @@ -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}) diff --git a/dbms/src/Common/ArrayCache.h b/dbms/src/Common/ArrayCache.h index 38ec9f88bd7..eda20ca6bf2 100644 --- a/dbms/src/Common/ArrayCache.h +++ b/dbms/src/Common/ArrayCache.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -160,7 +161,7 @@ private: mutable std::mutex mutex; - std::mt19937_64 rng {static_cast(randomSeed())}; + pcg64 rng{randomSeed()}; struct Chunk : private boost::noncopyable { diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 05c5c16d3d2..765bf494580 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -1,9 +1,11 @@ #include +#include #include #include #include #include #include +#include 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) { diff --git a/dbms/src/Common/tests/array_cache.cpp b/dbms/src/Common/tests/array_cache.cpp index 8dec433e3d6..b962e86c53c 100644 --- a/dbms/src/Common/tests/array_cache.cpp +++ b/dbms/src/Common/tests/array_cache.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -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) { diff --git a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp index 8dee71fb379..48f04e9fc01 100644 --- a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -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) @@ -106,7 +107,7 @@ TEST(Common, RWLockFIFO_Recursive) std::this_thread::sleep_for(sleep_for); auto lock2 = fifo_lock->getLock(RWLockFIFO::Read); - + EXPECT_ANY_THROW({fifo_lock->getLock(RWLockFIFO::Write);}); } diff --git a/dbms/src/DataStreams/narrowBlockInputStreams.cpp b/dbms/src/DataStreams/narrowBlockInputStreams.cpp index 25927e451e4..07c26bc9b67 100644 --- a/dbms/src/DataStreams/narrowBlockInputStreams.cpp +++ b/dbms/src/DataStreams/narrowBlockInputStreams.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -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) diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index f69fbd607f4..b9d33f5ac16 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -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{ diff --git a/dbms/src/Dictionaries/CacheDictionary.h b/dbms/src/Dictionaries/CacheDictionary.h index 45bfceb16ad..ac0854fc84e 100644 --- a/dbms/src/Dictionaries/CacheDictionary.h +++ b/dbms/src/Dictionaries/CacheDictionary.h @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include @@ -260,7 +260,7 @@ private: Attribute * hierarchical_attribute = nullptr; std::unique_ptr string_arena; - mutable std::mt19937_64 rnd_engine; + mutable pcg64 rnd_engine; mutable size_t bytes_allocated = 0; mutable std::atomic element_count{0}; diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp index b8f5c8c1dfd..37cfc759e6c 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp @@ -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{ diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index 2b806b51316..09e6330efdd 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -19,6 +18,7 @@ #include #include #include +#include namespace ProfileEvents @@ -721,7 +721,7 @@ private: std::unique_ptr fixed_size_keys_pool = key_size_is_fixed ? std::make_unique(key_size) : nullptr; std::unique_ptr string_arena; - mutable std::mt19937_64 rnd_engine; + mutable pcg64 rnd_engine; mutable size_t bytes_allocated = 0; mutable std::atomic element_count{0}; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index a2812495c69..3663597f334 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -1,6 +1,5 @@ #include #include -#include #include #include @@ -9,6 +8,7 @@ #include #include +#include #include #include @@ -175,7 +175,7 @@ struct ContextShared Context::ApplicationType application_type = Context::ApplicationType::SERVER; - std::mt19937_64 rng{randomSeed()}; + pcg64 rng{randomSeed()}; ContextShared() { diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index eda33e983ee..c2afea4a79f 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -34,6 +35,9 @@ #include #include +#include +#include + 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(&rng)); + pcg64 rng(randomSeed()); for (int num_tries = 0; num_tries < 10; ++num_tries) { diff --git a/dbms/src/Interpreters/ExternalDictionaries.h b/dbms/src/Interpreters/ExternalDictionaries.h index 7419f44a627..84488118e19 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.h +++ b/dbms/src/Interpreters/ExternalDictionaries.h @@ -13,7 +13,8 @@ #include #include #include -#include +#include + namespace DB { @@ -74,7 +75,7 @@ private: */ std::unordered_map update_times; - std::mt19937_64 rnd_engine{randomSeed()}; + pcg64 rnd_engine{randomSeed()}; Context & context; diff --git a/dbms/src/Interpreters/Quota.cpp b/dbms/src/Interpreters/Quota.cpp index 4a3d40aa7fa..65fdda86fa5 100644 --- a/dbms/src/Interpreters/Quota.cpp +++ b/dbms/src/Interpreters/Quota.cpp @@ -8,6 +8,7 @@ #include #include +#include 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); diff --git a/dbms/src/Interpreters/Quota.h b/dbms/src/Interpreters/Quota.h index 3331af1cfa2..8df01fe9593 100644 --- a/dbms/src/Interpreters/Quota.h +++ b/dbms/src/Interpreters/Quota.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include @@ -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); }; diff --git a/dbms/src/Server/Benchmark.cpp b/dbms/src/Server/Benchmark.cpp index f3bd262ad6f..8d7f1057a2f 100644 --- a/dbms/src/Server/Benchmark.cpp +++ b/dbms/src/Server/Benchmark.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -230,7 +231,7 @@ private: void run() { - std::mt19937 generator(randomSeed()); + pcg64 generator(randomSeed()); std::uniform_int_distribution distribution(0, queries.size() - 1); for (size_t i = 0; i < concurrency; ++i) diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index c0a7c1c0214..12b7edf32dc 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -2,10 +2,12 @@ #include #include #include +#include #include #include #include +#include #include @@ -115,7 +117,7 @@ void BackgroundProcessingPool::threadFunction() memory_tracker.setMetric(CurrentMetrics::MemoryTrackingInBackgroundProcessingPool); current_memory_tracker = &memory_tracker; - std::mt19937 rng(reinterpret_cast(&rng)); + pcg64 rng(randomSeed()); std::this_thread::sleep_for(std::chrono::duration(std::uniform_real_distribution(0, sleep_seconds_random_part)(rng))); while (!shutdown) diff --git a/dbms/src/Storages/MergeTree/ReshardingWorker.cpp b/dbms/src/Storages/MergeTree/ReshardingWorker.cpp index e0d0086c3fe..08b5ed51504 100644 --- a/dbms/src/Storages/MergeTree/ReshardingWorker.cpp +++ b/dbms/src/Storages/MergeTree/ReshardingWorker.cpp @@ -31,11 +31,14 @@ #include #include +#include + #include #include #include #include -#include +#include + 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 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]; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 0793231b16e..25562c778f1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -321,7 +321,7 @@ private: Logger * log; - std::mt19937 rng{randomSeed()}; + pcg64 rng{randomSeed()}; StorageReplicatedMergeTree( const String & zookeeper_path_, diff --git a/utils/iotest/iotest.cpp b/utils/iotest/iotest.cpp index 2c91e11cd41..439672fc25c 100644 --- a/utils/iotest/iotest.cpp +++ b/utils/iotest/iotest.cpp @@ -7,12 +7,14 @@ #include #include #include +#include #include #include #include #include +#include #include #include @@ -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) { diff --git a/utils/iotest/iotest_nonblock.cpp b/utils/iotest/iotest_nonblock.cpp index 5482e0ddaa7..fdd25860825 100644 --- a/utils/iotest/iotest_nonblock.cpp +++ b/utils/iotest/iotest_nonblock.cpp @@ -13,12 +13,14 @@ #include #include +#include #include #include #include #include +#include #include #include @@ -77,11 +79,7 @@ int mainImpl(int argc, char ** argv) std::vector buf(block_size); - std::mt19937 rng; - - timespec times; - clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×); - rng.seed(times.tv_nsec); + pcg64 rng(randomSeed()); Stopwatch watch; diff --git a/utils/test-data-generator/main.cpp b/utils/test-data-generator/main.cpp index 4d16195cb72..2a0db1e4b3d 100644 --- a/utils/test-data-generator/main.cpp +++ b/utils/test-data-generator/main.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -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(); diff --git a/utils/test-data-generator/markov-model.cpp b/utils/test-data-generator/markov-model.cpp index 08f7a3f72a9..87849f8f398 100644 --- a/utils/test-data-generator/markov-model.cpp +++ b/utils/test-data-generator/markov-model.cpp @@ -1,5 +1,6 @@ #include #include +#include #include @@ -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());