Merge pull request #6197 from yandex/thread_local_rng

Less number of "clock_gettime" calls; fixed ABI compatibility between debug/release in Allocator.
This commit is contained in:
alexey-milovidov 2019-07-31 04:36:30 +03:00 committed by GitHub
commit 7fd6204114
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 28 additions and 41 deletions

View File

@ -11,7 +11,7 @@
#endif
#include <pcg_random.hpp>
#include <Common/randomSeed.h>
#include <Common/thread_local_rng.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#include <malloc.h>
@ -86,10 +86,8 @@ struct RandomHint
{
void * mmap_hint()
{
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(rng));
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(thread_local_rng));
}
private:
pcg64 rng{randomSeed()};
};
}

View File

@ -1,7 +1,6 @@
#include "QueryProfiler.h"
#include <random>
#include <pcg_random.hpp>
#include <common/Pipe.h>
#include <common/phdr_cache.h>
#include <common/config_common.h>
@ -10,7 +9,7 @@
#include <common/logger_useful.h>
#include <Common/CurrentThread.h>
#include <Common/Exception.h>
#include <Common/randomSeed.h>
#include <Common/thread_local_rng.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
@ -63,7 +62,6 @@ namespace
constexpr size_t QUERY_ID_MAX_LEN = 1024;
thread_local size_t write_trace_iteration = 0;
thread_local pcg64 rng{randomSeed()};
void writeTraceInfo(TimerType timer_type, int /* sig */, siginfo_t * info, void * context)
{
@ -161,7 +159,7 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const Int32 thread_id, const
/// It will allow to sample short queries even if timer period is large.
/// (For example, with period of 1 second, query with 50 ms duration will be sampled with 1 / 20 probability).
/// It also helps to avoid interference (moire).
UInt32 period_rand = std::uniform_int_distribution<UInt32>(0, period)(rng);
UInt32 period_rand = std::uniform_int_distribution<UInt32>(0, period)(thread_local_rng);
struct timespec interval{.tv_sec = period / TIMER_PRECISION, .tv_nsec = period % TIMER_PRECISION};
struct timespec offset{.tv_sec = period_rand / TIMER_PRECISION, .tv_nsec = period_rand % TIMER_PRECISION};

View File

@ -4,14 +4,13 @@
#include "TestKeeper.h"
#include <random>
#include <pcg_random.hpp>
#include <functional>
#include <boost/algorithm/string.hpp>
#include <common/logger_useful.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/PODArray.h>
#include <Common/randomSeed.h>
#include <Common/thread_local_rng.h>
#include <Common/Exception.h>
#include <Poco/Net/NetException.h>
@ -159,8 +158,7 @@ struct ZooKeeperArgs
}
/// Shuffle the hosts to distribute the load among ZooKeeper nodes.
pcg64 rng(randomSeed());
std::shuffle(hosts_strings.begin(), hosts_strings.end(), rng);
std::shuffle(hosts_strings.begin(), hosts_strings.end(), thread_local_rng);
for (auto & host : hosts_strings)
{

View File

@ -0,0 +1,4 @@
#include <Common/thread_local_rng.h>
#include <Common/randomSeed.h>
thread_local pcg64 thread_local_rng{randomSeed()};

View File

@ -0,0 +1,5 @@
#pragma once
#include <pcg_random.hpp>
/// Fairly good thread-safe random number generator, but probably slow-down thread creation a little.
extern thread_local pcg64 thread_local_rng;

View File

@ -101,6 +101,7 @@ struct PerformanceStatistics
Element data[NUM_ELEMENTS];
/// It's Ok that generator is not seeded.
pcg64 rng;
/// To select from different algorithms we use a kind of "bandits" algorithm.

View File

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

View File

@ -5,12 +5,12 @@
#include <Poco/Mutex.h>
#include <Poco/UUID.h>
#include <Poco/Net/IPAddress.h>
#include <pcg_random.hpp>
#include <Common/Macros.h>
#include <Common/escapeForFileName.h>
#include <Common/setThreadName.h>
#include <Common/Stopwatch.h>
#include <Common/formatReadable.h>
#include <Common/thread_local_rng.h>
#include <Compression/ICompressionCodec.h>
#include <Core/BackgroundSchedulePool.h>
#include <Formats/FormatFactory.h>
@ -205,8 +205,6 @@ struct ContextShared
Context::ApplicationType application_type = Context::ApplicationType::SERVER;
pcg64 rng{randomSeed()};
/// vector of xdbc-bridge commands, they will be killed when Context will be destroyed
std::vector<std::unique_ptr<ShellCommand>> bridge_commands;
@ -1172,12 +1170,8 @@ void Context::setCurrentQueryId(const String & query_id)
} words;
} random;
{
auto lock = getLock();
random.words.a = shared->rng();
random.words.b = shared->rng();
}
random.words.a = thread_local_rng();
random.words.b = thread_local_rng();
/// Use protected constructor.
struct UUID : Poco::UUID

View File

@ -6,6 +6,7 @@
#include <Common/escapeForFileName.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/typeid_cast.h>
#include <Common/thread_local_rng.h>
#include <Common/ThreadPool.h>
#include <Storages/AlterCommands.h>
@ -162,14 +163,6 @@ static const auto MUTATIONS_FINALIZING_SLEEP_MS = 1 * 1000;
extern const int MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER = 5 * 60;
/** For randomized selection of replicas. */
/// avoid error: non-local variable 'DB::rng' declared '__thread' needs dynamic initialization
#ifndef __APPLE__
thread_local
#endif
pcg64 rng{randomSeed()};
void StorageReplicatedMergeTree::setZooKeeper(zkutil::ZooKeeperPtr zookeeper)
{
std::lock_guard lock(current_zookeeper_mutex);
@ -708,7 +701,7 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
part->columns, part->checksums);
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
std::shuffle(replicas.begin(), replicas.end(), rng);
std::shuffle(replicas.begin(), replicas.end(), thread_local_rng);
bool has_been_already_added = false;
for (const String & replica : replicas)
@ -2445,7 +2438,7 @@ String StorageReplicatedMergeTree::findReplicaHavingPart(const String & part_nam
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
/// Select replicas in uniformly random order.
std::shuffle(replicas.begin(), replicas.end(), rng);
std::shuffle(replicas.begin(), replicas.end(), thread_local_rng);
for (const String & replica : replicas)
{
@ -2470,7 +2463,7 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(LogEntry & entr
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
/// Select replicas in uniformly random order.
std::shuffle(replicas.begin(), replicas.end(), rng);
std::shuffle(replicas.begin(), replicas.end(), thread_local_rng);
for (const String & replica : replicas)
{
@ -2529,7 +2522,7 @@ String StorageReplicatedMergeTree::findReplicaHavingCoveringPart(
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
/// Select replicas in uniformly random order.
std::shuffle(replicas.begin(), replicas.end(), rng);
std::shuffle(replicas.begin(), replicas.end(), thread_local_rng);
String largest_part_found;
String largest_replica_found;

View File

@ -2,8 +2,7 @@
#include <algorithm>
#include <DataTypes/DataTypeString.h>
#include <pcg_random.hpp>
#include <Common/randomSeed.h>
#include <Common/thread_local_rng.h>
extern const char * auto_contributors[];
@ -23,8 +22,7 @@ void StorageSystemContributors::fillData(MutableColumns & res_columns, const Con
for (auto it = auto_contributors; *it; ++it)
contributors.emplace_back(*it);
pcg64 rng(randomSeed());
std::shuffle(contributors.begin(), contributors.end(), rng);
std::shuffle(contributors.begin(), contributors.end(), thread_local_rng);
for (auto & it : contributors)
res_columns[0]->insert(String(it));