ClickHouse/src/Interpreters/Context.cpp

3091 lines
102 KiB
C++
Raw Normal View History

2015-04-16 06:12:35 +00:00
#include <map>
#include <set>
2019-02-04 23:18:04 +00:00
#include <optional>
#include <memory>
2015-04-16 06:12:35 +00:00
#include <Poco/Mutex.h>
#include <Poco/UUID.h>
#include <Poco/Net/IPAddress.h>
2019-11-04 19:17:27 +00:00
#include <Poco/Util/Application.h>
#include <Common/Macros.h>
#include <Common/escapeForFileName.h>
#include <Common/setThreadName.h>
#include <Common/Stopwatch.h>
#include <Common/formatReadable.h>
2021-05-26 20:37:44 +00:00
#include <Common/Throttler.h>
2019-07-28 15:30:38 +00:00
#include <Common/thread_local_rng.h>
2021-06-14 04:13:35 +00:00
#include <Common/FieldVisitorToString.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Coordination/KeeperDispatcher.h>
2018-12-21 12:17:30 +00:00
#include <Compression/ICompressionCodec.h>
2018-08-20 15:34:37 +00:00
#include <Core/BackgroundSchedulePool.h>
#include <Formats/FormatFactory.h>
#include <Databases/IDatabase.h>
#include <Storages/IStorage.h>
#include <Storages/MarkCache.h>
#include <Storages/MergeTree/MergeList.h>
2020-10-27 12:47:42 +00:00
#include <Storages/MergeTree/ReplicatedFetchList.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeSettings.h>
2018-12-21 12:17:30 +00:00
#include <Storages/CompressionCodecSelector.h>
#include <Storages/StorageS3Settings.h>
#include <Disks/DiskLocal.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <Interpreters/ActionLocksManager.h>
#include <Interpreters/ExternalLoaderXMLConfigRepository.h>
#include <Core/Settings.h>
#include <Core/SettingsQuirks.h>
#include <Access/AccessControlManager.h>
#include <Access/ContextAccess.h>
#include <Access/EnabledRolesInfo.h>
#include <Access/EnabledRowPolicies.h>
#include <Access/QuotaUsage.h>
#include <Access/User.h>
#include <Access/Credentials.h>
#include <Access/SettingsProfile.h>
#include <Access/SettingsProfilesInfo.h>
#include <Access/SettingsConstraintsAndProfileIDs.h>
#include <Access/ExternalAuthenticators.h>
#include <Access/GSSAcceptor.h>
2021-08-18 22:19:14 +00:00
#include <Backups/BackupFactory.h>
2019-09-26 16:12:15 +00:00
#include <Dictionaries/Embedded/GeoDictionariesLoader.h>
#include <Interpreters/EmbeddedDictionaries.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/ExternalUserDefinedExecutableFunctionsLoader.h>
#include <Interpreters/ExternalModelsLoader.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ProcessList.h>
#include <Interpreters/InterserverCredentials.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/InterserverIOHandler.h>
#include <Interpreters/SystemLog.h>
#include <Interpreters/SessionLog.h>
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/DDLTask.h>
2021-08-19 14:09:44 +00:00
#include <Interpreters/Session.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/UncompressedCache.h>
2021-03-28 19:24:28 +00:00
#include <IO/MMappedFileCache.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/parseQuery.h>
2019-07-29 22:26:44 +00:00
#include <Common/StackTrace.h>
#include <Common/Config/ConfigProcessor.h>
#include <Common/Config/AbstractConfigurationComparison.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ShellCommand.h>
2019-07-10 20:47:39 +00:00
#include <Common/TraceCollector.h>
2021-10-02 07:13:14 +00:00
#include <base/logger_useful.h>
#include <base/EnumReflection.h>
#include <Common/RemoteHostFilter.h>
2021-09-01 23:18:09 +00:00
#include <Interpreters/AsynchronousInsertQueue.h>
2020-02-03 12:54:36 +00:00
#include <Interpreters/DatabaseCatalog.h>
2021-07-02 13:36:02 +00:00
#include <Interpreters/JIT/CompiledExpressionCache.h>
2021-09-06 12:01:16 +00:00
#include <Storages/MergeTree/BackgroundJobsAssignee.h>
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <Interpreters/SynonymsExtensions.h>
#include <Interpreters/Lemmatizers.h>
2021-04-28 20:48:34 +00:00
#include <filesystem>
2021-04-28 20:48:34 +00:00
namespace fs = std::filesystem;
namespace ProfileEvents
{
extern const Event ContextLock;
extern const Event CompiledCacheSizeBytes;
}
namespace CurrentMetrics
{
extern const Metric ContextLockWait;
extern const Metric BackgroundMovePoolTask;
extern const Metric BackgroundSchedulePoolTask;
extern const Metric BackgroundBufferFlushSchedulePoolTask;
extern const Metric BackgroundDistributedSchedulePoolTask;
extern const Metric BackgroundMessageBrokerSchedulePoolTask;
2021-08-30 19:37:03 +00:00
extern const Metric DelayedInserts;
extern const Metric BackgroundMergesAndMutationsPoolTask;
2021-08-30 19:37:03 +00:00
extern const Metric BackgroundMovePoolTask;
extern const Metric BackgroundFetchesPoolTask;
extern const Metric BackgroundCommonPoolTask;
2021-08-30 19:37:03 +00:00
}
2012-08-02 17:33:31 +00:00
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int BAD_ARGUMENTS;
extern const int BAD_GET;
extern const int UNKNOWN_DATABASE;
extern const int UNKNOWN_TABLE;
extern const int TABLE_ALREADY_EXISTS;
extern const int THERE_IS_NO_SESSION;
extern const int THERE_IS_NO_QUERY;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT;
2019-09-03 19:53:59 +00:00
extern const int LOGICAL_ERROR;
2020-03-05 20:02:55 +00:00
extern const int NOT_IMPLEMENTED;
2021-04-19 19:16:34 +00:00
extern const int INVALID_SETTING_VALUE;
2021-09-06 15:59:46 +00:00
extern const int UNKNOWN_READ_METHOD;
}
2015-04-16 06:12:35 +00:00
/** Set of known objects (environment), that could be used in query.
* Shared (global) part. Order of members (especially, order of destruction) is very important.
2015-04-16 06:12:35 +00:00
*/
struct ContextSharedPart
2015-04-16 06:12:35 +00:00
{
2020-05-30 21:57:37 +00:00
Poco::Logger * log = &Poco::Logger::get("Context");
/// For access of most of shared objects. Recursive mutex.
mutable std::recursive_mutex mutex;
/// Separate mutex for access of dictionaries. Separate mutex to avoid locks when server doing request to itself.
mutable std::mutex embedded_dictionaries_mutex;
mutable std::mutex external_dictionaries_mutex;
mutable std::mutex external_user_defined_executable_functions_mutex;
mutable std::mutex external_models_mutex;
/// Separate mutex for storage policies. During server startup we may
2020-05-21 18:49:31 +00:00
/// initialize some important storages (system logs with MergeTree engine)
/// under context lock.
mutable std::mutex storage_policies_mutex;
2020-01-11 09:50:41 +00:00
/// Separate mutex for re-initialization of zookeeper session. This operation could take a long time and must not interfere with another operations.
mutable std::mutex zookeeper_mutex;
2017-04-02 17:37:49 +00:00
mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper.
ConfigurationPtr zookeeper_config; /// Stores zookeeper configs
2021-02-01 13:18:17 +00:00
#if USE_NURAFT
2021-10-18 15:27:51 +00:00
mutable std::mutex keeper_dispatcher_mutex;
mutable std::shared_ptr<KeeperDispatcher> keeper_dispatcher;
2021-02-01 13:18:17 +00:00
#endif
mutable std::mutex auxiliary_zookeepers_mutex;
mutable std::map<String, zkutil::ZooKeeperPtr> auxiliary_zookeepers; /// Map for auxiliary ZooKeeper clients.
ConfigurationPtr auxiliary_zookeepers_config; /// Stores auxiliary zookeepers configs
2017-04-02 17:37:49 +00:00
String interserver_io_host; /// The host name by which this server is available for other servers.
UInt16 interserver_io_port = 0; /// and port.
String interserver_scheme; /// http or https
2021-04-07 13:52:11 +00:00
MultiVersion<InterserverCredentials> interserver_io_credentials;
2017-04-02 17:37:49 +00:00
String path; /// Path to the data directory, with a slash at the end.
String flags_path; /// Path to the directory with some control flags for server maintenance.
String user_files_path; /// Path to the directory with user provided files, usable by 'file' table function.
String dictionaries_lib_path; /// Path to the directory with user provided binaries and libraries for external dictionaries.
2021-08-25 19:30:22 +00:00
String user_scripts_path; /// Path to the directory with user provided scripts.
ConfigurationPtr config; /// Global configuration settings.
String tmp_path; /// Path to the temporary files that occur when processing the request.
2020-07-08 14:25:23 +00:00
mutable VolumePtr tmp_volume; /// Volume for the the temporary files that occur when processing the request.
2021-08-18 22:19:14 +00:00
mutable VolumePtr backups_volume; /// Volume for all the backups.
2019-02-04 23:18:04 +00:00
mutable std::optional<EmbeddedDictionaries> embedded_dictionaries; /// Metrica's dictionaries. Have lazy initialization.
mutable std::optional<ExternalDictionariesLoader> external_dictionaries_loader;
mutable std::optional<ExternalUserDefinedExecutableFunctionsLoader> external_user_defined_executable_functions_loader;
mutable std::optional<ExternalModelsLoader> external_models_loader;
ExternalLoaderXMLConfigRepository * external_models_config_repository = nullptr;
2021-06-15 19:55:21 +00:00
scope_guard models_repository_guard;
ExternalLoaderXMLConfigRepository * external_dictionaries_config_repository = nullptr;
2021-06-15 19:55:21 +00:00
scope_guard dictionaries_xmls;
ExternalLoaderXMLConfigRepository * user_defined_executable_functions_config_repository = nullptr;
2021-09-15 20:52:18 +00:00
scope_guard user_defined_executable_functions_xmls;
#if USE_NLP
mutable std::optional<SynonymsExtensions> synonyms_extensions;
2021-06-05 00:52:35 +00:00
mutable std::optional<Lemmatizers> lemmatizers;
#endif
String default_profile_name; /// Default profile name used for default values.
String system_profile_name; /// Profile used by system processes
String buffer_profile_name; /// Profile used by Buffer engine for flushing to the underlying
2021-08-18 07:23:32 +00:00
std::unique_ptr<AccessControlManager> access_control_manager;
2017-04-02 17:37:49 +00:00
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
mutable UncompressedCachePtr index_uncompressed_cache; /// The cache of decompressed blocks for MergeTree indices.
mutable MarkCachePtr index_mark_cache; /// Cache of marks in compressed files of MergeTree indices.
2021-03-28 19:24:28 +00:00
mutable MMappedFileCachePtr mmap_cache; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads.
2017-04-02 17:37:49 +00:00
ProcessList process_list; /// Executing queries at the moment.
MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree)
2020-10-26 16:38:35 +00:00
ReplicatedFetchList replicated_fetch_list;
2017-04-02 17:37:49 +00:00
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
2020-11-06 14:07:56 +00:00
mutable std::optional<BackgroundSchedulePool> buffer_flush_schedule_pool; /// A thread pool that can do background flush for Buffer tables.
mutable std::optional<BackgroundSchedulePool> schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables)
mutable std::optional<BackgroundSchedulePool> distributed_schedule_pool; /// A thread pool that can run different jobs in background (used for distributed sends)
mutable std::optional<BackgroundSchedulePool> message_broker_schedule_pool; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka)
2021-05-26 20:37:44 +00:00
mutable ThrottlerPtr replicated_fetches_throttler; /// A server-wide throttler for replicated fetches
mutable ThrottlerPtr replicated_sends_throttler; /// A server-wide throttler for replicated sends
2018-03-13 23:44:23 +00:00
MultiVersion<Macros> macros; /// Substitutions extracted from config.
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
2018-12-21 12:17:30 +00:00
mutable std::unique_ptr<CompressionCodecSelector> compression_codec_selector;
/// Storage disk chooser for MergeTree engines
mutable std::shared_ptr<const DiskSelector> merge_tree_disk_selector;
/// Storage policy chooser for MergeTree engines
mutable std::shared_ptr<const StoragePolicySelector> merge_tree_storage_policy_selector;
2019-09-04 12:44:12 +00:00
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
2020-08-10 11:23:58 +00:00
std::optional<MergeTreeSettings> replicated_merge_tree_settings; /// Settings of ReplicatedMergeTree* engines.
std::atomic_size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
std::atomic_size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
String format_schema_path; /// Path to a directory that contains schema files used by input formats.
ActionLocksManagerPtr action_locks_manager; /// Set of storages' action lockers
Fix one more lock-order-inversion TSan report [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314) Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368 Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f) 4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f) 5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f) 6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda) 7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9) 4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9) 5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b) 4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b) 5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b) 6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768) 7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b) 8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b) 9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea) 10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea) 11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea) 12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea) 13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8) 14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8) 15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8) 16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8) 17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff) 18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff) 19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff) 20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff) 21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655) 22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655) 23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) Mutex M2505 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f) 4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f) 5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) [1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1 v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor) v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_ v4: fix conflicts
2021-03-03 05:08:10 +00:00
std::unique_ptr<SystemLogs> system_logs; /// Used to log queries and operations on parts
std::optional<StorageS3Settings> storage_s3_settings; /// Settings of S3 storage
2021-07-13 00:59:52 +00:00
std::vector<String> warnings; /// Store warning messages about server configuration.
2021-09-02 10:39:27 +00:00
/// Background executors for *MergeTree tables
MergeMutateBackgroundExecutorPtr merge_mutate_executor;
OrdinaryBackgroundExecutorPtr moves_executor;
OrdinaryBackgroundExecutorPtr fetch_executor;
OrdinaryBackgroundExecutorPtr common_executor;
2021-09-02 10:39:27 +00:00
RemoteHostFilter remote_host_filter; /// Allowed URL from config.xml
2019-02-03 09:57:12 +00:00
std::optional<TraceCollector> trace_collector; /// Thread collecting traces from threads executing queries
/// Clusters for distributed tables
/// Initialized on demand (on distributed storages initialization) since Settings should be initialized
2021-06-26 14:15:57 +00:00
std::shared_ptr<Clusters> clusters;
2020-01-11 09:50:41 +00:00
ConfigurationPtr clusters_config; /// Stores updated configs
mutable std::mutex clusters_mutex; /// Guards clusters and clusters_config
2021-03-17 14:11:47 +00:00
std::shared_ptr<AsynchronousInsertQueue> async_insert_queue;
2021-08-20 04:11:47 +00:00
std::map<String, UInt16> server_ports;
2021-03-17 14:11:47 +00:00
bool shutdown_called = false;
Stopwatch uptime_watch;
Context::ApplicationType application_type = Context::ApplicationType::SERVER;
/// vector of xdbc-bridge commands, they will be killed when Context will be destroyed
std::vector<std::unique_ptr<ShellCommand>> bridge_commands;
Context::ConfigReloadCallback config_reload_callback;
ContextSharedPart()
2021-08-18 07:23:32 +00:00
: access_control_manager(std::make_unique<AccessControlManager>()), macros(std::make_unique<Macros>())
2017-06-19 20:35:53 +00:00
{
/// TODO: make it singleton (?)
2017-06-19 20:35:53 +00:00
static std::atomic<size_t> num_calls{0};
if (++num_calls > 1)
{
std::cerr << "Attempting to create multiple ContextShared instances. Stack trace:\n" << StackTrace().toString();
std::cerr.flush();
std::terminate();
}
}
~ContextSharedPart()
{
try
{
shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
2017-04-02 17:37:49 +00:00
/** Perform a complex job of destroying objects in advance.
*/
void shutdown()
{
if (shutdown_called)
return;
shutdown_called = true;
2021-08-19 14:09:44 +00:00
Session::shutdownNamedSessions();
2019-06-21 17:25:47 +00:00
/** After system_logs have been shut down it is guaranteed that no system table gets created or written to.
* Note that part changes at shutdown won't be logged to part log.
*/
2019-03-21 19:22:38 +00:00
if (system_logs)
system_logs->shutdown();
2020-04-08 23:59:39 +00:00
DatabaseCatalog::shutdown();
2019-02-04 23:18:04 +00:00
2021-09-02 10:39:27 +00:00
if (merge_mutate_executor)
merge_mutate_executor->wait();
if (fetch_executor)
fetch_executor->wait();
if (moves_executor)
moves_executor->wait();
if (common_executor)
common_executor->wait();
2021-09-02 10:39:27 +00:00
Fix one more lock-order-inversion TSan report [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314) Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368 Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f) 4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f) 5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f) 6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda) 7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9) 4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9) 5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b) 4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b) 5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b) 6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768) 7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b) 8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b) 9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea) 10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea) 11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea) 12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea) 13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8) 14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8) 15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8) 16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8) 17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff) 18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff) 19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff) 20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff) 21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655) 22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655) 23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) Mutex M2505 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f) 4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f) 5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) [1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1 v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor) v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_ v4: fix conflicts
2021-03-03 05:08:10 +00:00
std::unique_ptr<SystemLogs> delete_system_logs;
{
auto lock = std::lock_guard(mutex);
2021-06-29 22:46:57 +00:00
/** Compiled expressions stored in cache need to be destroyed before destruction of static objects.
* Because CHJIT instance can be static object.
*/
2021-06-29 20:40:15 +00:00
#if USE_EMBEDDED_COMPILER
if (auto * cache = CompiledExpressionCacheFactory::instance().tryGetCache())
cache->reset();
#endif
Fix one more lock-order-inversion TSan report [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314) Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368 Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f) 4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f) 5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f) 6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda) 7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9) 4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9) 5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b) 4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b) 5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b) 6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768) 7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b) 8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b) 9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea) 10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea) 11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea) 12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea) 13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8) 14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8) 15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8) 16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8) 17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff) 18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff) 19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff) 20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff) 21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655) 22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655) 23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) Mutex M2505 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f) 4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f) 5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) [1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1 v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor) v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_ v4: fix conflicts
2021-03-03 05:08:10 +00:00
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
/// TODO: Get rid of this.
/// Dictionaries may be required:
/// - for storage shutdown (during final flush of the Buffer engine)
/// - before storage startup (because of some streaming of, i.e. Kafka, to
/// the table with materialized column that has dictGet)
///
/// So they should be created before any storages and preserved until storages will be terminated.
///
/// But they cannot be created before storages since they may required table as a source,
/// but at least they can be preserved for storage termination.
dictionaries_xmls.reset();
2021-09-15 20:52:18 +00:00
user_defined_executable_functions_xmls.reset();
Fix one more lock-order-inversion TSan report [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314) Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368 Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f) 4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f) 5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f) 6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda) 7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9) 4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9) 5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b) 4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b) 5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b) 6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768) 7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b) 8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b) 9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea) 10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea) 11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea) 12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea) 13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8) 14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8) 15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8) 16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8) 17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff) 18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff) 19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff) 20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff) 21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655) 22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655) 23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) Mutex M2505 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f) 4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f) 5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) [1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1 v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor) v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_ v4: fix conflicts
2021-03-03 05:08:10 +00:00
delete_system_logs = std::move(system_logs);
embedded_dictionaries.reset();
external_dictionaries_loader.reset();
external_user_defined_executable_functions_loader.reset();
Fix one more lock-order-inversion TSan report [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314) Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368 Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f) 4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f) 5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f) 6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda) 7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9) 4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9) 5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b) 4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b) 5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b) 6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768) 7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b) 8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b) 9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea) 10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea) 11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea) 12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea) 13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8) 14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8) 15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8) 16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8) 17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff) 18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff) 19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff) 20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff) 21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655) 22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655) 23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) Mutex M2505 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f) 4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f) 5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) [1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1 v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor) v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_ v4: fix conflicts
2021-03-03 05:08:10 +00:00
models_repository_guard.reset();
external_models_loader.reset();
buffer_flush_schedule_pool.reset();
schedule_pool.reset();
distributed_schedule_pool.reset();
message_broker_schedule_pool.reset();
ddl_worker.reset();
2021-08-18 07:23:32 +00:00
access_control_manager.reset();
Fix one more lock-order-inversion TSan report [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314) Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368 Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f) 4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f) 5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f) 6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda) 7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9) 4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9) 5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b) 4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b) 5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b) 6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768) 7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b) 8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b) 9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea) 10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea) 11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea) 12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea) 13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8) 14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8) 15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8) 16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8) 17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff) 18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff) 19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff) 20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff) 21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655) 22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655) 23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) Mutex M2505 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f) 4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f) 5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) [1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1 v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor) v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_ v4: fix conflicts
2021-03-03 05:08:10 +00:00
/// Stop trace collector if any
trace_collector.reset();
/// Stop zookeeper connection
zookeeper.reset();
}
/// Can be removed w/o context lock
delete_system_logs.reset();
}
bool hasTraceCollector() const
2019-07-06 20:42:03 +00:00
{
return trace_collector.has_value();
}
2019-05-19 20:22:44 +00:00
void initializeTraceCollector(std::shared_ptr<TraceLog> trace_log)
2019-02-09 22:40:47 +00:00
{
if (!trace_log)
return;
if (hasTraceCollector())
return;
trace_collector.emplace(std::move(trace_log));
}
2021-07-09 09:27:51 +00:00
2021-07-13 00:59:52 +00:00
void addWarningMessage(const String & message)
2021-07-09 09:27:51 +00:00
{
2021-07-13 00:59:52 +00:00
/// A warning goes both: into server's log; stored to be placed in `system.warnings` table.
2021-07-09 09:27:51 +00:00
log->warning(message);
2021-07-13 00:59:52 +00:00
warnings.push_back(message);
2021-07-09 09:27:51 +00:00
}
2015-04-16 06:12:35 +00:00
};
Context::Context() = default;
Context::Context(const Context &) = default;
Context & Context::operator=(const Context &) = default;
SharedContextHolder::SharedContextHolder(SharedContextHolder &&) noexcept = default;
2020-04-19 15:46:12 +00:00
SharedContextHolder & SharedContextHolder::operator=(SharedContextHolder &&) = default;
2020-04-19 15:41:35 +00:00
SharedContextHolder::SharedContextHolder() = default;
2020-04-16 14:51:33 +00:00
SharedContextHolder::~SharedContextHolder() = default;
SharedContextHolder::SharedContextHolder(std::unique_ptr<ContextSharedPart> shared_context)
: shared(std::move(shared_context)) {}
2020-04-17 12:58:52 +00:00
void SharedContextHolder::reset() { shared.reset(); }
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context::createGlobal(ContextSharedPart * shared)
2015-04-16 06:12:35 +00:00
{
auto res = std::shared_ptr<Context>(new Context);
res->shared = shared;
return res;
2015-04-16 06:12:35 +00:00
}
void Context::initGlobal()
{
DatabaseCatalog::init(shared_from_this());
}
2020-04-16 14:51:33 +00:00
SharedContextHolder Context::createShared()
2020-04-16 14:37:38 +00:00
{
return SharedContextHolder(std::make_unique<ContextSharedPart>());
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context::createCopy(const ContextPtr & other)
{
return std::shared_ptr<Context>(new Context(*other));
}
2021-06-01 12:20:52 +00:00
ContextMutablePtr Context::createCopy(const ContextWeakPtr & other)
{
auto ptr = other.lock();
if (!ptr) throw Exception("Can't copy an expired context", ErrorCodes::LOGICAL_ERROR);
return createCopy(ptr);
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context::createCopy(const ContextMutablePtr & other)
{
return createCopy(std::const_pointer_cast<const Context>(other));
}
Context::~Context() = default;
2015-04-16 06:12:35 +00:00
InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; }
std::unique_lock<std::recursive_mutex> Context::getLock() const
{
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
return std::unique_lock(shared->mutex);
}
ProcessList & Context::getProcessList() { return shared->process_list; }
const ProcessList & Context::getProcessList() const { return shared->process_list; }
MergeList & Context::getMergeList() { return shared->merge_list; }
const MergeList & Context::getMergeList() const { return shared->merge_list; }
2020-10-26 16:38:35 +00:00
ReplicatedFetchList & Context::getReplicatedFetchList() { return shared->replicated_fetch_list; }
const ReplicatedFetchList & Context::getReplicatedFetchList() const { return shared->replicated_fetch_list; }
2015-04-16 06:12:35 +00:00
2020-02-03 12:54:36 +00:00
String Context::resolveDatabase(const String & database_name) const
{
2020-02-03 12:54:36 +00:00
String res = database_name.empty() ? getCurrentDatabase() : database_name;
if (res.empty())
throw Exception("Default database is not selected", ErrorCodes::UNKNOWN_DATABASE);
return res;
}
2012-08-02 17:33:31 +00:00
String Context::getPath() const
{
auto lock = getLock();
return shared->path;
2012-08-02 17:33:31 +00:00
}
String Context::getFlagsPath() const
{
auto lock = getLock();
return shared->flags_path;
}
String Context::getUserFilesPath() const
{
auto lock = getLock();
return shared->user_files_path;
}
2012-08-02 17:33:31 +00:00
String Context::getDictionariesLibPath() const
{
auto lock = getLock();
return shared->dictionaries_lib_path;
}
2021-08-25 19:30:22 +00:00
String Context::getUserScriptsPath() const
{
auto lock = getLock();
return shared->user_scripts_path;
}
2021-07-09 09:27:51 +00:00
std::vector<String> Context::getWarnings() const
{
auto lock = getLock();
2021-07-13 00:59:52 +00:00
return shared->warnings;
2021-07-09 09:27:51 +00:00
}
2020-07-08 14:25:23 +00:00
VolumePtr Context::getTemporaryVolume() const
{
auto lock = getLock();
return shared->tmp_volume;
}
2012-08-02 17:33:31 +00:00
void Context::setPath(const String & path)
{
auto lock = getLock();
shared->path = path;
if (shared->tmp_path.empty() && !shared->tmp_volume)
shared->tmp_path = shared->path + "tmp/";
if (shared->flags_path.empty())
shared->flags_path = shared->path + "flags/";
if (shared->user_files_path.empty())
shared->user_files_path = shared->path + "user_files/";
if (shared->dictionaries_lib_path.empty())
shared->dictionaries_lib_path = shared->path + "dictionaries_lib/";
2021-08-25 19:30:22 +00:00
if (shared->user_scripts_path.empty())
shared->user_scripts_path = shared->path + "user_scripts/";
2012-08-02 17:33:31 +00:00
}
2020-07-08 14:25:23 +00:00
VolumePtr Context::setTemporaryStorage(const String & path, const String & policy_name)
{
std::lock_guard lock(shared->storage_policies_mutex);
if (policy_name.empty())
{
shared->tmp_path = path;
if (!shared->tmp_path.ends_with('/'))
shared->tmp_path += '/';
auto disk = std::make_shared<DiskLocal>("_tmp_default", shared->tmp_path, 0);
shared->tmp_volume = std::make_shared<SingleDiskVolume>("_tmp_default", disk, 0);
}
else
{
StoragePolicyPtr tmp_policy = getStoragePolicySelector(lock)->get(policy_name);
if (tmp_policy->getVolumes().size() != 1)
throw Exception("Policy " + policy_name + " is used temporary files, such policy should have exactly one volume",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
shared->tmp_volume = tmp_policy->getVolume(0);
}
if (shared->tmp_volume->getDisks().empty())
throw Exception("No disks volume for temporary files", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
return shared->tmp_volume;
}
2021-08-18 22:19:14 +00:00
void Context::setBackupsVolume(const String & path, const String & policy_name)
{
std::lock_guard lock(shared->storage_policies_mutex);
if (policy_name.empty())
{
String path_with_separator = path;
if (!path_with_separator.ends_with('/'))
path_with_separator += '/';
auto disk = std::make_shared<DiskLocal>("_backups_default", path_with_separator, 0);
shared->backups_volume = std::make_shared<SingleDiskVolume>("_backups_default", disk, 0);
}
else
{
StoragePolicyPtr policy = getStoragePolicySelector(lock)->get(policy_name);
if (policy->getVolumes().size() != 1)
throw Exception("Policy " + policy_name + " is used for backups, such policy should have exactly one volume",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
shared->backups_volume = policy->getVolume(0);
}
BackupFactory::instance().setBackupsVolume(shared->backups_volume);
}
VolumePtr Context::getBackupsVolume() const
{
std::lock_guard lock(shared->storage_policies_mutex);
return shared->backups_volume;
}
void Context::setFlagsPath(const String & path)
{
auto lock = getLock();
shared->flags_path = path;
}
void Context::setUserFilesPath(const String & path)
{
auto lock = getLock();
shared->user_files_path = path;
}
void Context::setDictionariesLibPath(const String & path)
{
auto lock = getLock();
shared->dictionaries_lib_path = path;
}
2021-08-25 19:30:22 +00:00
void Context::setUserScriptsPath(const String & path)
{
auto lock = getLock();
shared->user_scripts_path = path;
}
2021-07-09 14:40:32 +00:00
void Context::addWarningMessage(const String & msg)
{
auto lock = getLock();
shared->addWarningMessage(msg);
}
void Context::setConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
shared->config = config;
2021-08-18 07:23:32 +00:00
shared->access_control_manager->setExternalAuthenticatorsConfig(*shared->config);
}
const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
{
auto lock = getLock();
return shared->config ? *shared->config : Poco::Util::Application::instance().config();
}
2012-08-02 17:33:31 +00:00
2020-02-12 03:03:33 +00:00
AccessControlManager & Context::getAccessControlManager()
{
2021-08-18 07:23:32 +00:00
return *shared->access_control_manager;
}
const AccessControlManager & Context::getAccessControlManager() const
{
2021-08-18 07:23:32 +00:00
return *shared->access_control_manager;
}
void Context::setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config)
{
auto lock = getLock();
2021-08-18 07:23:32 +00:00
shared->access_control_manager->setExternalAuthenticatorsConfig(config);
}
2019-11-29 17:22:56 +00:00
std::unique_ptr<GSSAcceptorContext> Context::makeGSSAcceptorContext() const
{
auto lock = getLock();
2021-08-18 07:23:32 +00:00
return std::make_unique<GSSAcceptorContext>(shared->access_control_manager->getExternalAuthenticators().getKerberosParams());
}
void Context::setUsersConfig(const ConfigurationPtr & config)
{
auto lock = getLock();
shared->users_config = config;
2021-08-18 07:23:32 +00:00
shared->access_control_manager->setUsersConfig(*shared->users_config);
}
ConfigurationPtr Context::getUsersConfig()
{
auto lock = getLock();
return shared->users_config;
}
2021-08-01 14:12:34 +00:00
void Context::setUser(const UUID & user_id_)
{
auto lock = getLock();
2021-08-01 14:12:34 +00:00
user_id = user_id_;
2021-08-01 14:12:34 +00:00
access = getAccessControlManager().getContextAccess(
user_id_, /* current_roles = */ {}, /* use_default_roles = */ true, settings, current_database, client_info);
auto user = access->getUser();
current_roles = std::make_shared<std::vector<UUID>>(user->granted_roles.findGranted(user->default_roles));
2019-11-04 19:17:27 +00:00
auto default_profile_info = access->getDefaultProfileInfo();
settings_constraints_and_current_profiles = default_profile_info->getConstraintsAndProfileIDs();
applySettingsChanges(default_profile_info->settings);
2021-08-01 14:12:34 +00:00
if (!user->default_database.empty())
setCurrentDatabase(user->default_database);
}
2020-02-12 03:03:33 +00:00
std::shared_ptr<const User> Context::getUser() const
{
return getAccess()->getUser();
}
2020-02-12 03:03:33 +00:00
String Context::getUserName() const
{
return getAccess()->getUserName();
}
std::optional<UUID> Context::getUserID() const
{
auto lock = getLock();
return user_id;
}
2021-08-01 14:12:34 +00:00
void Context::setQuotaKey(String quota_key_)
{
auto lock = getLock();
client_info.quota_key = std::move(quota_key_);
}
void Context::setCurrentRoles(const std::vector<UUID> & current_roles_)
{
auto lock = getLock();
if (current_roles ? (*current_roles == current_roles_) : current_roles_.empty())
return;
current_roles = std::make_shared<std::vector<UUID>>(current_roles_);
2020-02-21 00:17:07 +00:00
calculateAccessRights();
}
2020-02-21 00:17:07 +00:00
void Context::setCurrentRolesDefault()
{
auto user = getUser();
setCurrentRoles(user->granted_roles.findGranted(user->default_roles));
}
boost::container::flat_set<UUID> Context::getCurrentRoles() const
{
return getRolesInfo()->current_roles;
}
boost::container::flat_set<UUID> Context::getEnabledRoles() const
2015-10-01 15:10:41 +00:00
{
return getRolesInfo()->enabled_roles;
2015-10-01 15:10:41 +00:00
}
std::shared_ptr<const EnabledRolesInfo> Context::getRolesInfo() const
{
return getAccess()->getRolesInfo();
}
2020-02-12 03:03:33 +00:00
void Context::calculateAccessRights()
{
auto lock = getLock();
2020-02-12 03:03:33 +00:00
if (user_id)
access = getAccessControlManager().getContextAccess(
*user_id,
current_roles ? *current_roles : std::vector<UUID>{},
/* use_default_roles = */ false,
settings,
current_database,
client_info);
}
2020-01-12 21:00:55 +00:00
template <typename... Args>
void Context::checkAccessImpl(const Args &... args) const
{
return getAccess()->checkAccess(args...);
2020-01-12 21:00:55 +00:00
}
void Context::checkAccess(const AccessFlags & flags) const { return checkAccessImpl(flags); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database) const { return checkAccessImpl(flags, database); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table) const { return checkAccessImpl(flags, database, table); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::string_view & column) const { return checkAccessImpl(flags, database, table, column); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const std::vector<std::string_view> & columns) const { return checkAccessImpl(flags, database, table, columns); }
void Context::checkAccess(const AccessFlags & flags, const std::string_view & database, const std::string_view & table, const Strings & columns) const { return checkAccessImpl(flags, database, table, columns); }
void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName()); }
void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::string_view & column) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), column); }
void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const std::vector<std::string_view> & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); }
void Context::checkAccess(const AccessFlags & flags, const StorageID & table_id, const Strings & columns) const { checkAccessImpl(flags, table_id.getDatabaseName(), table_id.getTableName(), columns); }
void Context::checkAccess(const AccessRightsElement & element) const { return checkAccessImpl(element); }
void Context::checkAccess(const AccessRightsElements & elements) const { return checkAccessImpl(elements); }
std::shared_ptr<const ContextAccess> Context::getAccess() const
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
return access ? access : ContextAccess::getFullAccess();
}
2012-08-02 17:33:31 +00:00
ASTPtr Context::getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType type) const
2020-02-12 03:03:33 +00:00
{
auto lock = getLock();
auto initial_condition = initial_row_policy ? initial_row_policy->getCondition(database, table_name, type) : nullptr;
return getAccess()->getRowPolicyCondition(database, table_name, type, initial_condition);
2012-08-02 17:33:31 +00:00
}
void Context::setInitialRowPolicy()
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
initial_row_policy = nullptr;
2021-08-01 14:12:34 +00:00
if (client_info.initial_user == client_info.current_user)
return;
auto initial_user_id = getAccessControlManager().find<User>(client_info.initial_user);
if (!initial_user_id)
return;
initial_row_policy = getAccessControlManager().getEnabledRowPolicies(*initial_user_id, {});
}
2020-02-12 03:03:33 +00:00
std::shared_ptr<const EnabledQuota> Context::getQuota() const
{
return getAccess()->getQuota();
2012-08-02 17:33:31 +00:00
}
std::optional<QuotaUsage> Context::getQuotaUsage() const
{
return getAccess()->getQuotaUsage();
}
void Context::setCurrentProfile(const String & profile_name)
2012-08-02 17:33:31 +00:00
{
auto lock = getLock();
try
{
UUID profile_id = getAccessControlManager().getID<SettingsProfile>(profile_name);
setCurrentProfile(profile_id);
}
catch (Exception & e)
{
e.addMessage(", while trying to set settings profile {}", profile_name);
throw;
}
}
void Context::setCurrentProfile(const UUID & profile_id)
{
auto lock = getLock();
auto profile_info = getAccessControlManager().getSettingsProfileInfo(profile_id);
checkSettingsConstraints(profile_info->settings);
applySettingsChanges(profile_info->settings);
settings_constraints_and_current_profiles = profile_info->getConstraintsAndProfileIDs(settings_constraints_and_current_profiles);
}
std::vector<UUID> Context::getCurrentProfiles() const
{
auto lock = getLock();
return settings_constraints_and_current_profiles->current_profiles;
}
std::vector<UUID> Context::getEnabledProfiles() const
{
auto lock = getLock();
return settings_constraints_and_current_profiles->enabled_profiles;
2012-08-02 17:33:31 +00:00
}
2019-10-19 20:36:35 +00:00
const Scalars & Context::getScalars() const
{
return scalars;
}
const Block & Context::getScalar(const String & name) const
{
auto it = scalars.find(name);
if (scalars.end() == it)
2020-06-06 01:23:21 +00:00
{
// This should be a logical error, but it fails the sql_fuzz test too
// often, so 'bad arguments' for now.
throw Exception("Scalar " + backQuoteIfNeed(name) + " doesn't exist (internal bug)", ErrorCodes::BAD_ARGUMENTS);
}
2019-10-19 20:36:35 +00:00
return it->second;
}
const Block * Context::tryGetLocalScalar(const String & name) const
{
auto it = local_scalars.find(name);
if (local_scalars.end() == it)
return nullptr;
return &it->second;
}
2019-10-19 20:36:35 +00:00
Tables Context::getExternalTables() const
{
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
auto lock = getLock();
Tables res;
2020-04-22 06:01:33 +00:00
for (const auto & table : external_tables_mapping)
res[table.first] = table.second->getTable();
auto query_context_ptr = query_context.lock();
auto session_context_ptr = session_context.lock();
if (query_context_ptr && query_context_ptr.get() != this)
{
Tables buf = query_context_ptr->getExternalTables();
res.insert(buf.begin(), buf.end());
}
else if (session_context_ptr && session_context_ptr.get() != this)
{
Tables buf = session_context_ptr->getExternalTables();
res.insert(buf.begin(), buf.end());
}
return res;
}
2020-03-10 19:36:17 +00:00
void Context::addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table)
2019-12-10 19:48:16 +00:00
{
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
2020-02-13 21:00:03 +00:00
auto lock = getLock();
if (external_tables_mapping.end() != external_tables_mapping.find(table_name))
throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
2020-03-10 19:36:17 +00:00
external_tables_mapping.emplace(table_name, std::make_shared<TemporaryTableHolder>(std::move(temporary_table)));
}
2020-03-13 15:41:36 +00:00
std::shared_ptr<TemporaryTableHolder> Context::removeExternalTable(const String & table_name)
2012-08-02 17:33:31 +00:00
{
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
2020-02-17 13:52:59 +00:00
std::shared_ptr<TemporaryTableHolder> holder;
{
auto lock = getLock();
2020-02-17 13:52:59 +00:00
auto iter = external_tables_mapping.find(table_name);
if (iter == external_tables_mapping.end())
return {};
2020-02-17 13:52:59 +00:00
holder = iter->second;
external_tables_mapping.erase(iter);
}
2020-03-13 15:41:36 +00:00
return holder;
}
2019-10-19 20:36:35 +00:00
void Context::addScalar(const String & name, const Block & block)
{
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
2019-10-19 20:36:35 +00:00
scalars[name] = block;
}
void Context::addLocalScalar(const String & name, const Block & block)
{
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
local_scalars[name] = block;
}
2019-10-19 20:36:35 +00:00
bool Context::hasScalar(const String & name) const
{
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
2019-10-19 20:36:35 +00:00
return scalars.count(name);
}
void Context::addQueryAccessInfo(
const String & quoted_database_name,
const String & full_quoted_table_name,
const Names & column_names,
const String & projection_name,
const String & view_name)
{
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
2021-01-20 12:30:41 +00:00
std::lock_guard<std::mutex> lock(query_access_info.mutex);
2020-12-14 03:30:39 +00:00
query_access_info.databases.emplace(quoted_database_name);
query_access_info.tables.emplace(full_quoted_table_name);
for (const auto & column_name : column_names)
2020-12-14 03:30:39 +00:00
query_access_info.columns.emplace(full_quoted_table_name + "." + backQuoteIfNeed(column_name));
if (!projection_name.empty())
query_access_info.projections.emplace(full_quoted_table_name + "." + backQuoteIfNeed(projection_name));
if (!view_name.empty())
query_access_info.views.emplace(view_name);
}
2021-01-21 21:15:11 +00:00
void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const
{
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
auto lock = getLock();
2021-01-21 21:15:11 +00:00
switch (factory_type)
{
case QueryLogFactories::AggregateFunction:
query_factories_info.aggregate_functions.emplace(created_object);
break;
case QueryLogFactories::AggregateFunctionCombinator:
query_factories_info.aggregate_function_combinators.emplace(created_object);
break;
2021-01-21 21:15:11 +00:00
case QueryLogFactories::Database:
query_factories_info.database_engines.emplace(created_object);
2021-01-21 21:15:11 +00:00
break;
case QueryLogFactories::DataType:
query_factories_info.data_type_families.emplace(created_object);
2021-01-21 21:15:11 +00:00
break;
case QueryLogFactories::Dictionary:
query_factories_info.dictionaries.emplace(created_object);
break;
case QueryLogFactories::Format:
query_factories_info.formats.emplace(created_object);
break;
case QueryLogFactories::Function:
query_factories_info.functions.emplace(created_object);
break;
case QueryLogFactories::Storage:
query_factories_info.storages.emplace(created_object);
break;
case QueryLogFactories::TableFunction:
query_factories_info.table_functions.emplace(created_object);
}
}
StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
{
/// Slightly suboptimal.
auto hash = table_expression->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
StoragePtr & res = table_function_results[key];
if (!res)
{
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, shared_from_this());
/// Run it and remember the result
res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName());
}
return res;
}
void Context::addViewSource(const StoragePtr & storage)
{
if (view_source)
throw Exception(
"Temporary view source storage " + backQuoteIfNeed(view_source->getName()) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
view_source = storage;
}
2021-05-31 14:49:02 +00:00
StoragePtr Context::getViewSource() const
{
return view_source;
}
2012-08-02 17:33:31 +00:00
Settings Context::getSettings() const
{
2020-09-24 17:08:15 +00:00
auto lock = getLock();
return settings;
2012-08-02 17:33:31 +00:00
}
void Context::setSettings(const Settings & settings_)
{
2020-01-12 21:00:55 +00:00
auto lock = getLock();
auto old_readonly = settings.readonly;
auto old_allow_ddl = settings.allow_ddl;
auto old_allow_introspection_functions = settings.allow_introspection_functions;
2020-01-12 21:00:55 +00:00
settings = settings_;
2020-01-12 21:00:55 +00:00
if ((settings.readonly != old_readonly) || (settings.allow_ddl != old_allow_ddl) || (settings.allow_introspection_functions != old_allow_introspection_functions))
calculateAccessRights();
2012-08-02 17:33:31 +00:00
}
void Context::setSetting(const StringRef & name, const String & value)
2012-08-02 19:03:32 +00:00
{
auto lock = getLock();
if (name == "profile")
{
setCurrentProfile(value);
return;
}
settings.set(std::string_view{name}, value);
2020-01-12 21:00:55 +00:00
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
}
void Context::setSetting(const StringRef & name, const Field & value)
{
auto lock = getLock();
if (name == "profile")
{
setCurrentProfile(value.safeGet<String>());
return;
}
settings.set(std::string_view{name}, value);
2020-01-12 21:00:55 +00:00
if (name == "readonly" || name == "allow_ddl" || name == "allow_introspection_functions")
calculateAccessRights();
}
void Context::applySettingChange(const SettingChange & change)
{
2020-06-14 01:07:47 +00:00
try
{
setSetting(change.name, change.value);
}
catch (Exception & e)
{
e.addMessage(fmt::format("in attempt to set the value of setting '{}' to {}",
change.name, applyVisitor(FieldVisitorToString(), change.value)));
throw;
}
}
void Context::applySettingsChanges(const SettingsChanges & changes)
{
auto lock = getLock();
for (const SettingChange & change : changes)
applySettingChange(change);
applySettingsQuirks(settings);
}
void Context::checkSettingsConstraints(const SettingChange & change) const
{
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, change);
}
void Context::checkSettingsConstraints(const SettingsChanges & changes) const
{
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes);
2012-08-02 19:03:32 +00:00
}
void Context::checkSettingsConstraints(SettingsChanges & changes) const
{
getSettingsConstraintsAndCurrentProfiles()->constraints.check(settings, changes);
}
void Context::clampToSettingsConstraints(SettingsChanges & changes) const
{
getSettingsConstraintsAndCurrentProfiles()->constraints.clamp(settings, changes);
}
std::shared_ptr<const SettingsConstraintsAndProfileIDs> Context::getSettingsConstraintsAndCurrentProfiles() const
{
auto lock = getLock();
if (settings_constraints_and_current_profiles)
return settings_constraints_and_current_profiles;
static auto no_constraints_or_profiles = std::make_shared<SettingsConstraintsAndProfileIDs>(getAccessControlManager());
return no_constraints_or_profiles;
}
2012-08-02 17:33:31 +00:00
String Context::getCurrentDatabase() const
{
2020-02-03 12:54:36 +00:00
auto lock = getLock();
return current_database;
2012-08-02 17:33:31 +00:00
}
String Context::getInitialQueryId() const
{
return client_info.initial_query_id;
}
2020-09-25 11:27:00 +00:00
void Context::setCurrentDatabaseNameInGlobalContext(const String & name)
{
if (!isGlobalContext())
throw Exception("Cannot set current database for non global context, this method should be used during server initialization",
ErrorCodes::LOGICAL_ERROR);
2020-09-25 11:27:00 +00:00
auto lock = getLock();
if (!current_database.empty())
throw Exception("Default database name cannot be changed in global context without server restart",
ErrorCodes::LOGICAL_ERROR);
2020-09-25 11:27:00 +00:00
current_database = name;
}
2012-08-02 17:33:31 +00:00
void Context::setCurrentDatabase(const String & name)
{
DatabaseCatalog::instance().assertDatabaseExists(name);
auto lock = getLock();
current_database = name;
calculateAccessRights();
2012-08-02 17:33:31 +00:00
}
void Context::setCurrentQueryId(const String & query_id)
{
2020-08-20 20:59:40 +00:00
/// Generate random UUID, but using lower quality RNG,
/// because Poco::UUIDGenerator::generateRandom method is using /dev/random, that is very expensive.
/// NOTE: Actually we don't need to use UUIDs for query identifiers.
/// We could use any suitable string instead.
union
{
2020-08-20 20:59:40 +00:00
char bytes[16];
struct
{
2020-08-20 20:59:40 +00:00
UInt64 a;
UInt64 b;
} words;
2021-05-04 22:42:14 +00:00
UUID uuid{};
2020-08-20 20:59:40 +00:00
} random;
2020-08-20 20:59:40 +00:00
random.words.a = thread_local_rng(); //-V656
random.words.b = thread_local_rng(); //-V656
2020-08-28 19:02:50 +00:00
2021-05-04 22:42:14 +00:00
if (client_info.client_trace_context.trace_id != UUID())
2020-11-18 17:43:18 +00:00
{
// Use the OpenTelemetry trace context we received from the client, and
// create a new span for the query.
query_trace_context = client_info.client_trace_context;
query_trace_context.span_id = thread_local_rng();
}
else if (client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
{
2020-09-08 13:19:27 +00:00
// If this is an initial query without any parent OpenTelemetry trace, we
// might start the trace ourselves, with some configurable probability.
std::bernoulli_distribution should_start_trace{
settings.opentelemetry_start_trace_probability};
2020-09-08 13:19:27 +00:00
if (should_start_trace(thread_local_rng))
{
2020-09-08 13:19:27 +00:00
// Use the randomly generated default query id as the new trace id.
2020-11-18 17:43:18 +00:00
query_trace_context.trace_id = random.uuid;
query_trace_context.span_id = thread_local_rng();
2020-09-08 13:19:27 +00:00
// Mark this trace as sampled in the flags.
2020-11-18 17:43:18 +00:00
query_trace_context.trace_flags = 1;
2020-09-08 13:19:27 +00:00
}
2020-08-28 01:21:08 +00:00
}
2020-08-20 20:59:40 +00:00
String query_id_to_set = query_id;
if (query_id_to_set.empty()) /// If the user did not submit his query_id, then we generate it ourselves.
{
/// Use protected constructor.
struct QueryUUID : Poco::UUID
{
QueryUUID(const char * bytes, Poco::UUID::Version version)
: Poco::UUID(bytes, version) {}
};
query_id_to_set = QueryUUID(random.bytes, Poco::UUID::UUID_RANDOM).toString();
}
client_info.current_query_id = query_id_to_set;
2021-08-01 14:12:34 +00:00
if (client_info.query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
client_info.initial_query_id = client_info.current_query_id;
}
void Context::killCurrentQuery()
{
if (process_list_elem)
{
process_list_elem->cancelQuery(true);
}
};
String Context::getDefaultFormat() const
{
return default_format.empty() ? "TabSeparated" : default_format;
}
void Context::setDefaultFormat(const String & name)
{
default_format = name;
}
2018-03-13 23:44:23 +00:00
MultiVersion<Macros>::Version Context::getMacros() const
2014-08-11 15:59:01 +00:00
{
2018-03-13 23:44:23 +00:00
return shared->macros.get();
2014-08-11 15:59:01 +00:00
}
2018-03-13 23:44:23 +00:00
void Context::setMacros(std::unique_ptr<Macros> && macros)
2014-08-11 15:59:01 +00:00
{
2018-03-13 23:44:23 +00:00
shared->macros.set(std::move(macros));
2014-08-11 15:59:01 +00:00
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context::getQueryContext() const
{
auto ptr = query_context.lock();
if (!ptr) throw Exception("There is no query or query context has expired", ErrorCodes::THERE_IS_NO_QUERY);
return ptr;
}
bool Context::isInternalSubquery() const
{
auto ptr = query_context.lock();
return ptr && ptr.get() != this;
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context::getSessionContext() const
2016-11-16 11:29:51 +00:00
{
auto ptr = session_context.lock();
if (!ptr) throw Exception("There is no session or session context has expired", ErrorCodes::THERE_IS_NO_SESSION);
return ptr;
2016-11-16 11:29:51 +00:00
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context::getGlobalContext() const
2012-08-02 17:33:31 +00:00
{
auto ptr = global_context.lock();
if (!ptr) throw Exception("There is no global context or global context has expired", ErrorCodes::LOGICAL_ERROR);
return ptr;
2012-08-02 17:33:31 +00:00
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context::getBufferContext() const
{
if (!buffer_context) throw Exception("There is no buffer context", ErrorCodes::LOGICAL_ERROR);
return buffer_context;
}
2012-12-19 20:15:15 +00:00
const EmbeddedDictionaries & Context::getEmbeddedDictionaries() const
2012-12-19 20:15:15 +00:00
{
return getEmbeddedDictionariesImpl(false);
}
EmbeddedDictionaries & Context::getEmbeddedDictionaries()
{
return getEmbeddedDictionariesImpl(false);
}
const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const
{
return const_cast<Context *>(this)->getExternalDictionariesLoader();
}
ExternalDictionariesLoader & Context::getExternalDictionariesLoader()
{
std::lock_guard lock(shared->external_dictionaries_mutex);
return getExternalDictionariesLoaderUnlocked();
}
ExternalDictionariesLoader & Context::getExternalDictionariesLoaderUnlocked()
{
if (!shared->external_dictionaries_loader)
shared->external_dictionaries_loader.emplace(getGlobalContext());
return *shared->external_dictionaries_loader;
}
const ExternalUserDefinedExecutableFunctionsLoader & Context::getExternalUserDefinedExecutableFunctionsLoader() const
{
return const_cast<Context *>(this)->getExternalUserDefinedExecutableFunctionsLoader();
}
ExternalUserDefinedExecutableFunctionsLoader & Context::getExternalUserDefinedExecutableFunctionsLoader()
{
std::lock_guard lock(shared->external_user_defined_executable_functions_mutex);
return getExternalUserDefinedExecutableFunctionsLoaderUnlocked();
}
ExternalUserDefinedExecutableFunctionsLoader & Context::getExternalUserDefinedExecutableFunctionsLoaderUnlocked()
{
if (!shared->external_user_defined_executable_functions_loader)
shared->external_user_defined_executable_functions_loader.emplace(getGlobalContext());
return *shared->external_user_defined_executable_functions_loader;
}
const ExternalModelsLoader & Context::getExternalModelsLoader() const
{
return const_cast<Context *>(this)->getExternalModelsLoader();
}
ExternalModelsLoader & Context::getExternalModelsLoader()
{
std::lock_guard lock(shared->external_models_mutex);
2021-04-12 09:28:46 +00:00
return getExternalModelsLoaderUnlocked();
}
ExternalModelsLoader & Context::getExternalModelsLoaderUnlocked()
{
if (!shared->external_models_loader)
shared->external_models_loader.emplace(getGlobalContext());
2021-04-12 09:28:46 +00:00
return *shared->external_models_loader;
}
void Context::loadOrReloadModels(const Poco::Util::AbstractConfiguration & config)
{
auto patterns_values = getMultipleValuesFromConfig(config, "", "models_config");
std::unordered_set<std::string> patterns(patterns_values.begin(), patterns_values.end());
std::lock_guard lock(shared->external_models_mutex);
auto & external_models_loader = getExternalModelsLoaderUnlocked();
if (shared->external_models_config_repository)
{
shared->external_models_config_repository->updatePatterns(patterns);
external_models_loader.reloadConfig(shared->external_models_config_repository->getName());
return;
}
auto app_path = getPath();
auto config_path = getConfigRef().getString("config-file", "config.xml");
auto repository = std::make_unique<ExternalLoaderXMLConfigRepository>(app_path, config_path, patterns);
shared->external_models_config_repository = repository.get();
shared->models_repository_guard = external_models_loader.addConfigRepository(std::move(repository));
}
EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const
{
std::lock_guard lock(shared->embedded_dictionaries_mutex);
if (!shared->embedded_dictionaries)
{
2019-09-26 16:12:15 +00:00
auto geo_dictionaries_loader = std::make_unique<GeoDictionariesLoader>();
shared->embedded_dictionaries.emplace(
std::move(geo_dictionaries_loader),
getGlobalContext(),
throw_on_error);
}
return *shared->embedded_dictionaries;
2012-12-19 20:15:15 +00:00
}
void Context::tryCreateEmbeddedDictionaries(const Poco::Util::AbstractConfiguration & config) const
{
if (!config.getBool("dictionaries_lazy_load", true))
static_cast<void>(getEmbeddedDictionariesImpl(true));
}
void Context::loadOrReloadDictionaries(const Poco::Util::AbstractConfiguration & config)
{
bool dictionaries_lazy_load = config.getBool("dictionaries_lazy_load", true);
auto patterns_values = getMultipleValuesFromConfig(config, "", "dictionaries_config");
std::unordered_set<std::string> patterns(patterns_values.begin(), patterns_values.end());
std::lock_guard lock(shared->external_dictionaries_mutex);
auto & external_dictionaries_loader = getExternalDictionariesLoaderUnlocked();
external_dictionaries_loader.enableAlwaysLoadEverything(!dictionaries_lazy_load);
if (shared->external_dictionaries_config_repository)
{
shared->external_dictionaries_config_repository->updatePatterns(patterns);
external_dictionaries_loader.reloadConfig(shared->external_dictionaries_config_repository->getName());
return;
}
auto app_path = getPath();
auto config_path = getConfigRef().getString("config-file", "config.xml");
auto repository = std::make_unique<ExternalLoaderXMLConfigRepository>(app_path, config_path, patterns);
shared->external_dictionaries_config_repository = repository.get();
shared->dictionaries_xmls = external_dictionaries_loader.addConfigRepository(std::move(repository));
}
void Context::loadOrReloadUserDefinedExecutableFunctions(const Poco::Util::AbstractConfiguration & config)
{
auto patterns_values = getMultipleValuesFromConfig(config, "", "user_defined_executable_functions_config");
std::unordered_set<std::string> patterns(patterns_values.begin(), patterns_values.end());
std::lock_guard lock(shared->external_user_defined_executable_functions_mutex);
auto & external_user_defined_executable_functions_loader = getExternalUserDefinedExecutableFunctionsLoaderUnlocked();
if (shared->user_defined_executable_functions_config_repository)
{
shared->user_defined_executable_functions_config_repository->updatePatterns(patterns);
external_user_defined_executable_functions_loader.reloadConfig(shared->user_defined_executable_functions_config_repository->getName());
return;
}
auto app_path = getPath();
auto config_path = getConfigRef().getString("config-file", "config.xml");
auto repository = std::make_unique<ExternalLoaderXMLConfigRepository>(app_path, config_path, patterns);
shared->user_defined_executable_functions_config_repository = repository.get();
shared->user_defined_executable_functions_xmls = external_user_defined_executable_functions_loader.addConfigRepository(std::move(repository));
}
#if USE_NLP
2021-06-04 00:33:41 +00:00
SynonymsExtensions & Context::getSynonymsExtensions() const
{
auto lock = getLock();
if (!shared->synonyms_extensions)
shared->synonyms_extensions.emplace(getConfigRef());
return *shared->synonyms_extensions;
}
2021-06-05 00:52:35 +00:00
Lemmatizers & Context::getLemmatizers() const
{
auto lock = getLock();
if (!shared->lemmatizers)
2021-06-05 00:52:35 +00:00
shared->lemmatizers.emplace(getConfigRef());
return *shared->lemmatizers;
2021-06-05 00:52:35 +00:00
}
#endif
2021-06-05 00:52:35 +00:00
void Context::setProgressCallback(ProgressCallback callback)
{
2017-04-02 17:37:49 +00:00
/// Callback is set to a session or to a query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
progress_callback = callback;
}
ProgressCallback Context::getProgressCallback() const
{
return progress_callback;
}
void Context::setProcessListElement(ProcessList::Element * elem)
{
2017-04-02 17:37:49 +00:00
/// Set to a session or query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
process_list_elem = elem;
}
ProcessList::Element * Context::getProcessListElement() const
{
return process_list_elem;
}
2014-03-28 14:36:24 +00:00
void Context::setUncompressedCache(size_t max_size_in_bytes)
{
auto lock = getLock();
if (shared->uncompressed_cache)
throw Exception("Uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->uncompressed_cache = std::make_shared<UncompressedCache>(max_size_in_bytes);
}
UncompressedCachePtr Context::getUncompressedCache() const
{
auto lock = getLock();
return shared->uncompressed_cache;
}
void Context::dropUncompressedCache() const
{
auto lock = getLock();
if (shared->uncompressed_cache)
shared->uncompressed_cache->reset();
}
2014-02-11 13:30:42 +00:00
void Context::setMarkCache(size_t cache_size_in_bytes)
{
auto lock = getLock();
2014-02-11 13:30:42 +00:00
if (shared->mark_cache)
throw Exception("Mark cache has been already created.", ErrorCodes::LOGICAL_ERROR);
2014-02-11 13:30:42 +00:00
shared->mark_cache = std::make_shared<MarkCache>(cache_size_in_bytes);
2014-02-11 13:30:42 +00:00
}
MarkCachePtr Context::getMarkCache() const
{
auto lock = getLock();
return shared->mark_cache;
2014-02-11 13:30:42 +00:00
}
void Context::dropMarkCache() const
{
auto lock = getLock();
if (shared->mark_cache)
shared->mark_cache->reset();
}
void Context::setIndexUncompressedCache(size_t max_size_in_bytes)
{
auto lock = getLock();
if (shared->index_uncompressed_cache)
throw Exception("Index uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->index_uncompressed_cache = std::make_shared<UncompressedCache>(max_size_in_bytes);
}
UncompressedCachePtr Context::getIndexUncompressedCache() const
{
auto lock = getLock();
return shared->index_uncompressed_cache;
}
void Context::dropIndexUncompressedCache() const
{
auto lock = getLock();
if (shared->index_uncompressed_cache)
shared->index_uncompressed_cache->reset();
}
void Context::setIndexMarkCache(size_t cache_size_in_bytes)
{
auto lock = getLock();
if (shared->index_mark_cache)
throw Exception("Index mark cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->index_mark_cache = std::make_shared<MarkCache>(cache_size_in_bytes);
}
MarkCachePtr Context::getIndexMarkCache() const
{
auto lock = getLock();
return shared->index_mark_cache;
}
void Context::dropIndexMarkCache() const
{
auto lock = getLock();
if (shared->index_mark_cache)
shared->index_mark_cache->reset();
}
2021-03-28 19:24:28 +00:00
void Context::setMMappedFileCache(size_t cache_size_in_num_entries)
{
auto lock = getLock();
if (shared->mmap_cache)
throw Exception("Mapped file cache has been already created.", ErrorCodes::LOGICAL_ERROR);
2021-03-28 19:24:28 +00:00
shared->mmap_cache = std::make_shared<MMappedFileCache>(cache_size_in_num_entries);
}
2021-03-28 19:24:28 +00:00
MMappedFileCachePtr Context::getMMappedFileCache() const
{
auto lock = getLock();
return shared->mmap_cache;
}
2021-03-28 19:24:28 +00:00
void Context::dropMMappedFileCache() const
{
auto lock = getLock();
if (shared->mmap_cache)
shared->mmap_cache->reset();
}
void Context::dropCaches() const
{
auto lock = getLock();
if (shared->uncompressed_cache)
shared->uncompressed_cache->reset();
if (shared->mark_cache)
shared->mark_cache->reset();
if (shared->index_uncompressed_cache)
shared->index_uncompressed_cache->reset();
if (shared->index_mark_cache)
shared->index_mark_cache->reset();
if (shared->mmap_cache)
shared->mmap_cache->reset();
}
2020-11-06 14:07:56 +00:00
BackgroundSchedulePool & Context::getBufferFlushSchedulePool() const
{
auto lock = getLock();
if (!shared->buffer_flush_schedule_pool)
shared->buffer_flush_schedule_pool.emplace(
settings.background_buffer_flush_schedule_pool_size,
CurrentMetrics::BackgroundBufferFlushSchedulePoolTask,
"BgBufSchPool");
return *shared->buffer_flush_schedule_pool;
}
2020-10-16 10:12:31 +00:00
BackgroundTaskSchedulingSettings Context::getBackgroundProcessingTaskSchedulingSettings() const
2020-10-14 16:09:18 +00:00
{
2020-10-16 10:12:31 +00:00
BackgroundTaskSchedulingSettings task_settings;
2020-10-14 16:09:18 +00:00
const auto & config = getConfigRef();
task_settings.thread_sleep_seconds = config.getDouble("background_processing_pool_thread_sleep_seconds", 10);
task_settings.thread_sleep_seconds_random_part = config.getDouble("background_processing_pool_thread_sleep_seconds_random_part", 1.0);
task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1);
2020-10-16 08:00:15 +00:00
task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_min", 10);
task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_max", 600);
task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1);
2020-10-14 16:09:18 +00:00
task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0);
return task_settings;
}
2020-10-16 10:12:31 +00:00
BackgroundTaskSchedulingSettings Context::getBackgroundMoveTaskSchedulingSettings() const
2020-10-14 16:09:18 +00:00
{
2020-10-16 10:12:31 +00:00
BackgroundTaskSchedulingSettings task_settings;
2020-10-14 16:09:18 +00:00
const auto & config = getConfigRef();
task_settings.thread_sleep_seconds = config.getDouble("background_move_processing_pool_thread_sleep_seconds", 10);
task_settings.thread_sleep_seconds_random_part = config.getDouble("background_move_processing_pool_thread_sleep_seconds_random_part", 1.0);
task_settings.thread_sleep_seconds_if_nothing_to_do = config.getDouble("background_move_processing_pool_thread_sleep_seconds_if_nothing_to_do", 0.1);
2020-10-16 08:00:15 +00:00
task_settings.task_sleep_seconds_when_no_work_min = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_min", 10);
task_settings.task_sleep_seconds_when_no_work_max = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_max", 600);
task_settings.task_sleep_seconds_when_no_work_multiplier = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_multiplier", 1.1);
2020-10-14 16:09:18 +00:00
task_settings.task_sleep_seconds_when_no_work_random_part = config.getDouble("background_move_processing_pool_task_sleep_seconds_when_no_work_random_part", 1.0);
return task_settings;
}
2020-11-06 14:07:56 +00:00
BackgroundSchedulePool & Context::getSchedulePool() const
{
auto lock = getLock();
if (!shared->schedule_pool)
shared->schedule_pool.emplace(
settings.background_schedule_pool_size,
CurrentMetrics::BackgroundSchedulePoolTask,
"BgSchPool");
return *shared->schedule_pool;
}
2020-11-06 14:07:56 +00:00
BackgroundSchedulePool & Context::getDistributedSchedulePool() const
{
auto lock = getLock();
if (!shared->distributed_schedule_pool)
shared->distributed_schedule_pool.emplace(
settings.background_distributed_schedule_pool_size,
CurrentMetrics::BackgroundDistributedSchedulePoolTask,
"BgDistSchPool");
return *shared->distributed_schedule_pool;
}
BackgroundSchedulePool & Context::getMessageBrokerSchedulePool() const
{
auto lock = getLock();
if (!shared->message_broker_schedule_pool)
shared->message_broker_schedule_pool.emplace(
settings.background_message_broker_schedule_pool_size,
CurrentMetrics::BackgroundMessageBrokerSchedulePoolTask,
"BgMBSchPool");
return *shared->message_broker_schedule_pool;
}
2021-05-26 20:37:44 +00:00
ThrottlerPtr Context::getReplicatedFetchesThrottler() const
{
auto lock = getLock();
if (!shared->replicated_fetches_throttler)
shared->replicated_fetches_throttler = std::make_shared<Throttler>(
settings.max_replicated_fetches_network_bandwidth_for_server);
2021-05-27 12:54:47 +00:00
return shared->replicated_fetches_throttler;
2021-05-26 20:37:44 +00:00
}
ThrottlerPtr Context::getReplicatedSendsThrottler() const
{
auto lock = getLock();
if (!shared->replicated_sends_throttler)
shared->replicated_sends_throttler = std::make_shared<Throttler>(
settings.max_replicated_sends_network_bandwidth_for_server);
2021-05-27 12:54:47 +00:00
return shared->replicated_sends_throttler;
2021-05-26 20:37:44 +00:00
}
2020-11-12 20:11:04 +00:00
bool Context::hasDistributedDDL() const
2020-11-06 18:28:59 +00:00
{
return getConfigRef().has("distributed_ddl");
}
void Context::setDDLWorker(std::unique_ptr<DDLWorker> ddl_worker)
{
auto lock = getLock();
if (shared->ddl_worker)
2020-11-06 18:28:59 +00:00
throw Exception("DDL background thread has already been initialized", ErrorCodes::LOGICAL_ERROR);
2020-12-01 17:20:42 +00:00
ddl_worker->startup();
shared->ddl_worker = std::move(ddl_worker);
}
DDLWorker & Context::getDDLWorker() const
{
auto lock = getLock();
if (!shared->ddl_worker)
2020-11-06 18:28:59 +00:00
{
if (!hasZooKeeper())
throw Exception("There is no Zookeeper configuration in server config", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
if (!hasDistributedDDL())
throw Exception("There is no DistributedDDL configuration in server config", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
throw Exception("DDL background thread is not initialized", ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
return *shared->ddl_worker;
}
2014-05-13 10:10:26 +00:00
zkutil::ZooKeeperPtr Context::getZooKeeper() const
2014-03-21 19:17:59 +00:00
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->zookeeper_mutex);
2014-04-25 13:55:15 +00:00
2020-11-03 09:57:24 +00:00
const auto & config = shared->zookeeper_config ? *shared->zookeeper_config : getConfigRef();
if (!shared->zookeeper)
2021-07-09 14:05:35 +00:00
shared->zookeeper = std::make_shared<zkutil::ZooKeeper>(config, "zookeeper", getZooKeeperLog());
else if (shared->zookeeper->expired())
shared->zookeeper = shared->zookeeper->startNewSession();
2014-04-25 13:55:15 +00:00
return shared->zookeeper;
2014-03-21 19:17:59 +00:00
}
2021-10-18 09:13:24 +00:00
namespace
{
2021-10-18 09:13:24 +00:00
bool checkZooKeeperConfigIsLocal(const Poco::Util::AbstractConfiguration & config, const std::string & config_name)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_name, keys);
for (const auto & key : keys)
{
if (startsWith(key, "node"))
{
String host = config.getString(config_name + "." + key + ".host");
if (isLocalAddress(DNSResolver::instance().resolveHost(host)))
return true;
}
}
return false;
}
}
bool Context::tryCheckClientConnectionToMyKeeperCluster() const
{
try
{
2021-10-18 09:13:24 +00:00
/// If our server is part of main Keeper cluster
if (checkZooKeeperConfigIsLocal(getConfigRef(), "zookeeper"))
{
LOG_DEBUG(shared->log, "Keeper server is participant of the main zookeeper cluster, will try to connect to it");
getZooKeeper();
/// Connected, return true
return true;
}
else
{
Poco::Util::AbstractConfiguration::Keys keys;
getConfigRef().keys("auxiliary_zookeepers", keys);
/// If our server is part of some auxiliary_zookeeper
for (const auto & aux_zk_name : keys)
{
if (checkZooKeeperConfigIsLocal(getConfigRef(), "auxiliary_zookeepers." + aux_zk_name))
{
LOG_DEBUG(shared->log, "Our Keeper server is participant of the auxiliary zookeeper cluster ({}), will try to connect to it", aux_zk_name);
getAuxiliaryZooKeeper(aux_zk_name);
/// Connected, return true
return true;
}
}
}
/// Our server doesn't depend on our Keeper cluster
return true;
}
catch (...)
{
return false;
}
}
2021-09-13 12:19:37 +00:00
UInt32 Context::getZooKeeperSessionUptime() const
{
std::lock_guard lock(shared->zookeeper_mutex);
if (!shared->zookeeper || shared->zookeeper->expired())
return 0;
return shared->zookeeper->getSessionUptime();
}
void Context::setSystemZooKeeperLogAfterInitializationIfNeeded()
{
/// It can be nearly impossible to understand in which order global objects are initialized on server startup.
/// If getZooKeeper() is called before initializeSystemLogs(), then zkutil::ZooKeeper gets nullptr
/// instead of pointer to system table and it logs nothing.
/// This method explicitly sets correct pointer to system log after its initialization.
/// TODO get rid of this if possible
std::lock_guard lock(shared->zookeeper_mutex);
if (!shared->system_logs || !shared->system_logs->zookeeper_log)
return;
if (shared->zookeeper)
shared->zookeeper->setZooKeeperLog(shared->system_logs->zookeeper_log);
for (auto & zk : shared->auxiliary_zookeepers)
zk.second->setZooKeeperLog(shared->system_logs->zookeeper_log);
}
2021-02-01 13:18:17 +00:00
2021-10-19 15:35:59 +00:00
void Context::initializeKeeperDispatcher([[maybe_unused]] bool start_async) const
2021-01-25 12:29:12 +00:00
{
2021-02-01 13:18:17 +00:00
#if USE_NURAFT
2021-10-18 15:27:51 +00:00
std::lock_guard lock(shared->keeper_dispatcher_mutex);
2021-01-25 12:29:12 +00:00
2021-10-18 15:27:51 +00:00
if (shared->keeper_dispatcher)
2021-03-29 08:24:56 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to initialize Keeper multiple times");
2021-01-25 12:29:12 +00:00
2021-01-26 08:17:19 +00:00
const auto & config = getConfigRef();
2021-03-29 08:24:56 +00:00
if (config.has("keeper_server"))
2021-01-25 12:29:12 +00:00
{
bool is_standalone_app = getApplicationType() == ApplicationType::KEEPER;
2021-10-18 09:13:24 +00:00
if (start_async)
{
assert(!is_standalone_app);
LOG_INFO(shared->log, "Connected to ZooKeeper (or Keeper) before internal Keeper start or we don't depend on our Keeper cluster"
", will wait for Keeper asynchronously");
}
else
{
LOG_INFO(shared->log, "Cannot connect to ZooKeeper (or Keeper) before internal Keeper start,"
"will wait for Keeper synchronously");
}
2021-10-18 15:27:51 +00:00
shared->keeper_dispatcher = std::make_shared<KeeperDispatcher>();
shared->keeper_dispatcher->initialize(config, is_standalone_app, start_async);
2021-01-25 12:29:12 +00:00
}
2021-02-01 13:18:17 +00:00
#endif
2021-01-25 12:29:12 +00:00
}
2021-02-01 13:18:17 +00:00
#if USE_NURAFT
std::shared_ptr<KeeperDispatcher> & Context::getKeeperDispatcher() const
2020-10-30 14:16:47 +00:00
{
2021-10-18 15:27:51 +00:00
std::lock_guard lock(shared->keeper_dispatcher_mutex);
if (!shared->keeper_dispatcher)
2021-03-29 08:24:56 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Keeper must be initialized before requests");
2020-10-30 14:16:47 +00:00
2021-10-18 15:27:51 +00:00
return shared->keeper_dispatcher;
2020-10-30 14:16:47 +00:00
}
2021-02-01 13:18:17 +00:00
#endif
2020-10-30 14:16:47 +00:00
void Context::shutdownKeeperDispatcher() const
2021-01-27 17:54:25 +00:00
{
2021-02-01 13:18:17 +00:00
#if USE_NURAFT
2021-10-18 15:27:51 +00:00
std::lock_guard lock(shared->keeper_dispatcher_mutex);
if (shared->keeper_dispatcher)
2021-01-27 17:54:25 +00:00
{
2021-10-18 15:27:51 +00:00
shared->keeper_dispatcher->shutdown();
shared->keeper_dispatcher.reset();
2021-01-27 17:54:25 +00:00
}
2021-02-01 13:18:17 +00:00
#endif
2020-10-30 14:16:47 +00:00
}
2021-02-01 13:18:17 +00:00
2021-10-18 15:27:51 +00:00
void Context::updateKeeperConfiguration(const Poco::Util::AbstractConfiguration & config)
{
#if USE_NURAFT
std::lock_guard lock(shared->keeper_dispatcher_mutex);
if (!shared->keeper_dispatcher)
return;
shared->keeper_dispatcher->updateConfiguration(config);
#endif
}
zkutil::ZooKeeperPtr Context::getAuxiliaryZooKeeper(const String & name) const
{
std::lock_guard lock(shared->auxiliary_zookeepers_mutex);
auto zookeeper = shared->auxiliary_zookeepers.find(name);
if (zookeeper == shared->auxiliary_zookeepers.end())
2020-08-28 11:12:51 +00:00
{
const auto & config = shared->auxiliary_zookeepers_config ? *shared->auxiliary_zookeepers_config : getConfigRef();
if (!config.has("auxiliary_zookeepers." + name))
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Unknown auxiliary ZooKeeper name '{}'. If it's required it can be added to the section <auxiliary_zookeepers> in "
"config.xml",
name);
2021-07-09 14:05:35 +00:00
zookeeper = shared->auxiliary_zookeepers.emplace(name,
std::make_shared<zkutil::ZooKeeper>(config, "auxiliary_zookeepers." + name, getZooKeeperLog())).first;
2020-08-28 11:12:51 +00:00
}
else if (zookeeper->second->expired())
zookeeper->second = zookeeper->second->startNewSession();
return zookeeper->second;
}
void Context::resetZooKeeper() const
{
std::lock_guard lock(shared->zookeeper_mutex);
shared->zookeeper.reset();
}
2021-07-09 14:05:35 +00:00
static void reloadZooKeeperIfChangedImpl(const ConfigurationPtr & config, const std::string & config_name, zkutil::ZooKeeperPtr & zk,
std::shared_ptr<ZooKeeperLog> zk_log)
{
if (!zk || zk->configChanged(*config, config_name))
{
if (zk)
zk->finalize("Config changed");
2021-07-09 14:05:35 +00:00
zk = std::make_shared<zkutil::ZooKeeper>(*config, config_name, std::move(zk_log));
}
}
void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const
{
std::lock_guard lock(shared->zookeeper_mutex);
2020-11-03 09:57:24 +00:00
shared->zookeeper_config = config;
2021-07-09 14:05:35 +00:00
reloadZooKeeperIfChangedImpl(config, "zookeeper", shared->zookeeper, getZooKeeperLog());
}
void Context::reloadAuxiliaryZooKeepersConfigIfChanged(const ConfigurationPtr & config)
{
std::lock_guard lock(shared->auxiliary_zookeepers_mutex);
shared->auxiliary_zookeepers_config = config;
2020-11-03 09:57:24 +00:00
for (auto it = shared->auxiliary_zookeepers.begin(); it != shared->auxiliary_zookeepers.end();)
{
if (!config->has("auxiliary_zookeepers." + it->first))
it = shared->auxiliary_zookeepers.erase(it);
else
{
2021-07-09 14:05:35 +00:00
reloadZooKeeperIfChangedImpl(config, "auxiliary_zookeepers." + it->first, it->second, getZooKeeperLog());
++it;
}
}
}
bool Context::hasZooKeeper() const
{
return getConfigRef().has("zookeeper");
}
2020-11-16 08:27:33 +00:00
bool Context::hasAuxiliaryZooKeeper(const String & name) const
{
return getConfigRef().has("auxiliary_zookeepers." + name);
}
2014-03-21 19:17:59 +00:00
2021-04-07 13:52:11 +00:00
InterserverCredentialsPtr Context::getInterserverCredentials()
{
2021-04-07 13:52:11 +00:00
return shared->interserver_io_credentials.get();
}
void Context::updateInterserverCredentials(const Poco::Util::AbstractConfiguration & config)
{
2021-04-07 13:52:11 +00:00
auto credentials = InterserverCredentials::make(config, "interserver_http_credentials");
shared->interserver_io_credentials.set(std::move(credentials));
}
void Context::setInterserverIOAddress(const String & host, UInt16 port)
{
shared->interserver_io_host = host;
shared->interserver_io_port = port;
}
std::pair<String, UInt16> Context::getInterserverIOAddress() const
{
if (shared->interserver_io_host.empty() || shared->interserver_io_port == 0)
throw Exception("Parameter 'interserver_http(s)_port' required for replication is not specified in configuration file.",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
return { shared->interserver_io_host, shared->interserver_io_port };
}
void Context::setInterserverScheme(const String & scheme)
{
shared->interserver_scheme = scheme;
}
String Context::getInterserverScheme() const
{
return shared->interserver_scheme;
}
void Context::setRemoteHostFilter(const Poco::Util::AbstractConfiguration & config)
{
shared->remote_host_filter.setValuesFromConfig(config);
}
2019-10-21 14:36:24 +00:00
const RemoteHostFilter & Context::getRemoteHostFilter() const
{
return shared->remote_host_filter;
}
UInt16 Context::getTCPPort() const
{
auto lock = getLock();
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef();
return config.getInt("tcp_port", DBMS_DEFAULT_PORT);
}
std::optional<UInt16> Context::getTCPPortSecure() const
{
auto lock = getLock();
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef();
if (config.has("tcp_port_secure"))
return config.getInt("tcp_port_secure");
return {};
}
2021-08-20 04:11:47 +00:00
void Context::registerServerPort(String port_name, UInt16 port)
{
shared->server_ports.emplace(std::move(port_name), port);
}
UInt16 Context::getServerPort(const String & port_name) const
{
auto it = shared->server_ports.find(port_name);
if (it == shared->server_ports.end())
throw Exception(ErrorCodes::BAD_GET, "There is no port named {}", port_name);
else
return it->second;
}
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
2013-12-07 16:51:29 +00:00
{
2021-06-26 14:15:57 +00:00
auto res = getClusters()->getCluster(cluster_name);
if (res)
return res;
if (!cluster_name.empty())
res = tryGetReplicatedDatabaseCluster(cluster_name);
if (res)
return res;
throw Exception("Requested cluster '" + cluster_name + "' not found", ErrorCodes::BAD_GET);
2013-12-07 16:51:29 +00:00
}
std::shared_ptr<Cluster> Context::tryGetCluster(const std::string & cluster_name) const
{
2021-06-26 14:15:57 +00:00
return getClusters()->getCluster(cluster_name);
2013-12-07 16:51:29 +00:00
}
2021-05-31 14:49:02 +00:00
void Context::reloadClusterConfig() const
{
while (true)
{
ConfigurationPtr cluster_config;
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
cluster_config = shared->clusters_config;
}
2020-04-22 06:01:33 +00:00
const auto & config = cluster_config ? *cluster_config : getConfigRef();
2021-06-26 14:15:57 +00:00
auto new_clusters = std::make_shared<Clusters>(config, settings);
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
if (shared->clusters_config.get() == cluster_config.get())
{
shared->clusters = std::move(new_clusters);
return;
}
2020-07-23 20:44:30 +00:00
// Clusters config has been suddenly changed, recompute clusters
}
}
}
2021-06-26 14:15:57 +00:00
std::shared_ptr<Clusters> Context::getClusters() const
2015-04-30 12:43:16 +00:00
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
if (!shared->clusters)
{
2020-04-22 06:01:33 +00:00
const auto & config = shared->clusters_config ? *shared->clusters_config : getConfigRef();
2021-06-26 14:15:57 +00:00
shared->clusters = std::make_shared<Clusters>(config, settings);
}
2016-03-10 03:08:09 +00:00
2021-06-26 14:15:57 +00:00
return shared->clusters;
2015-04-30 12:43:16 +00:00
}
/// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters
void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name)
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
/// Do not update clusters if this part of config wasn't changed.
2021-03-13 20:16:24 +00:00
if (shared->clusters && isSameConfiguration(*config, *shared->clusters_config, config_name))
return;
auto old_clusters_config = shared->clusters_config;
shared->clusters_config = config;
if (!shared->clusters)
shared->clusters = std::make_unique<Clusters>(*shared->clusters_config, settings, config_name);
else
shared->clusters->updateClusters(*shared->clusters_config, settings, config_name, old_clusters_config);
2015-04-30 12:43:16 +00:00
}
void Context::setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster)
{
2019-01-02 06:44:36 +00:00
std::lock_guard lock(shared->clusters_mutex);
if (!shared->clusters)
throw Exception("Clusters are not set", ErrorCodes::LOGICAL_ERROR);
shared->clusters->setCluster(cluster_name, cluster);
2015-04-30 12:43:16 +00:00
}
2019-08-04 15:51:04 +00:00
void Context::initializeSystemLogs()
{
auto lock = getLock();
Fix one more lock-order-inversion TSan report [1]: WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314) Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368 Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f) 4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f) 5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f) 6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda) 7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9) 4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9) 5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e) 6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba) 7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba) 8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba) 9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba) 10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba) 11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba) 12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0) 13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0) 14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0) 15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600) 16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600) 17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600) 18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600) 19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600) 20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561) 21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561) 22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561) 23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561) 24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205) 25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205) 26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205) 27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8) 28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8) 29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8) 30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8) Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9) 2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9) 3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b) 4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b) 5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b) 6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768) 7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b) 8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b) 9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea) 10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea) 11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea) 12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea) 13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8) 14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8) 15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8) 16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8) 17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff) 18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff) 19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff) 20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff) 21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655) 22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655) 23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) Mutex M2505 previously acquired by the same thread here: 0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6) 1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689) 2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689) 3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f) 4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f) 5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37) 6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2) 7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2) 8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2) 9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba) 10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b) 11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be) 12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093) 13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e) 14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9) [1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1 v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor) v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_ v4: fix conflicts
2021-03-03 05:08:10 +00:00
shared->system_logs = std::make_unique<SystemLogs>(getGlobalContext(), getConfigRef());
}
2019-02-03 21:30:45 +00:00
void Context::initializeTraceCollector()
{
2019-02-03 21:30:45 +00:00
shared->initializeTraceCollector(getTraceLog());
}
bool Context::hasTraceCollector() const
{
return shared->hasTraceCollector();
}
2021-05-31 14:49:02 +00:00
std::shared_ptr<QueryLog> Context::getQueryLog() const
{
auto lock = getLock();
if (!shared->system_logs)
2019-03-21 19:22:38 +00:00
return {};
2019-03-21 19:22:38 +00:00
return shared->system_logs->query_log;
}
2021-06-18 13:44:08 +00:00
std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog() const
{
auto lock = getLock();
if (!shared->system_logs)
return {};
2021-06-18 13:44:08 +00:00
return shared->system_logs->query_thread_log;
}
2021-06-18 13:44:08 +00:00
std::shared_ptr<QueryViewsLog> Context::getQueryViewsLog() const
{
auto lock = getLock();
if (!shared->system_logs)
2019-03-21 19:22:38 +00:00
return {};
2021-06-18 13:44:08 +00:00
return shared->system_logs->query_views_log;
}
2021-05-31 14:49:02 +00:00
std::shared_ptr<PartLog> Context::getPartLog(const String & part_database) const
{
auto lock = getLock();
/// No part log or system logs are shutting down.
if (!shared->system_logs)
2019-03-21 19:22:38 +00:00
return {};
/// Will not log operations on system tables (including part_log itself).
/// It doesn't make sense and not allow to destruct PartLog correctly due to infinite logging and flushing,
/// and also make troubles on startup.
if (part_database == DatabaseCatalog::SYSTEM_DATABASE)
2019-03-21 19:22:38 +00:00
return {};
2019-03-21 19:22:38 +00:00
return shared->system_logs->part_log;
}
2021-05-31 14:49:02 +00:00
std::shared_ptr<TraceLog> Context::getTraceLog() const
2019-02-03 21:30:45 +00:00
{
auto lock = getLock();
if (!shared->system_logs)
2019-07-23 14:50:38 +00:00
return {};
2019-02-03 21:30:45 +00:00
2019-05-19 20:22:44 +00:00
return shared->system_logs->trace_log;
2019-02-03 21:30:45 +00:00
}
2019-08-13 14:31:46 +00:00
2021-05-31 14:49:02 +00:00
std::shared_ptr<TextLog> Context::getTextLog() const
{
auto lock = getLock();
if (!shared->system_logs)
2019-08-04 15:51:04 +00:00
return {};
2019-07-31 14:18:59 +00:00
2019-08-04 15:51:04 +00:00
return shared->system_logs->text_log;
}
2021-05-31 14:49:02 +00:00
std::shared_ptr<MetricLog> Context::getMetricLog() const
2019-08-13 14:31:46 +00:00
{
auto lock = getLock();
if (!shared->system_logs)
2019-08-13 14:31:46 +00:00
return {};
return shared->system_logs->metric_log;
}
std::shared_ptr<AsynchronousMetricLog> Context::getAsynchronousMetricLog() const
2020-06-10 19:17:30 +00:00
{
auto lock = getLock();
if (!shared->system_logs)
return {};
return shared->system_logs->asynchronous_metric_log;
}
2021-05-31 14:49:02 +00:00
std::shared_ptr<OpenTelemetrySpanLog> Context::getOpenTelemetrySpanLog() const
2020-08-20 20:59:40 +00:00
{
auto lock = getLock();
if (!shared->system_logs)
return {};
2020-10-22 16:47:20 +00:00
return shared->system_logs->opentelemetry_span_log;
2020-08-20 20:59:40 +00:00
}
std::shared_ptr<SessionLog> Context::getSessionLog() const
{
auto lock = getLock();
if (!shared->system_logs)
return {};
return shared->system_logs->session_log;
}
2020-08-20 20:59:40 +00:00
2021-07-09 14:05:35 +00:00
std::shared_ptr<ZooKeeperLog> Context::getZooKeeperLog() const
{
auto lock = getLock();
if (!shared->system_logs)
return {};
return shared->system_logs->zookeeper_log;
}
2018-12-21 12:17:30 +00:00
CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const
{
auto lock = getLock();
2018-12-21 12:17:30 +00:00
if (!shared->compression_codec_selector)
{
constexpr auto config_name = "compression";
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef();
if (config.has(config_name))
2018-12-21 12:17:30 +00:00
shared->compression_codec_selector = std::make_unique<CompressionCodecSelector>(config, "compression");
else
2018-12-21 12:17:30 +00:00
shared->compression_codec_selector = std::make_unique<CompressionCodecSelector>();
}
2018-12-21 12:17:30 +00:00
return shared->compression_codec_selector->choose(part_size, part_size_ratio);
}
DiskPtr Context::getDisk(const String & name) const
{
std::lock_guard lock(shared->storage_policies_mutex);
auto disk_selector = getDiskSelector(lock);
return disk_selector->get(name);
}
StoragePolicyPtr Context::getStoragePolicy(const String & name) const
{
std::lock_guard lock(shared->storage_policies_mutex);
auto policy_selector = getStoragePolicySelector(lock);
return policy_selector->get(name);
}
2020-05-22 10:33:57 +00:00
DisksMap Context::getDisksMap() const
{
std::lock_guard lock(shared->storage_policies_mutex);
return getDiskSelector(lock)->getDisksMap();
}
2020-05-22 10:33:57 +00:00
StoragePoliciesMap Context::getPoliciesMap() const
{
std::lock_guard lock(shared->storage_policies_mutex);
return getStoragePolicySelector(lock)->getPoliciesMap();
}
DiskSelectorPtr Context::getDiskSelector(std::lock_guard<std::mutex> & /* lock */) const
{
if (!shared->merge_tree_disk_selector)
{
constexpr auto config_name = "storage_configuration.disks";
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef();
shared->merge_tree_disk_selector = std::make_shared<DiskSelector>(config, config_name, shared_from_this());
}
return shared->merge_tree_disk_selector;
}
StoragePolicySelectorPtr Context::getStoragePolicySelector(std::lock_guard<std::mutex> & lock) const
{
if (!shared->merge_tree_storage_policy_selector)
{
constexpr auto config_name = "storage_configuration.policies";
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef();
shared->merge_tree_storage_policy_selector = std::make_shared<StoragePolicySelector>(config, config_name, getDiskSelector(lock));
}
return shared->merge_tree_storage_policy_selector;
}
void Context::updateStorageConfiguration(const Poco::Util::AbstractConfiguration & config)
{
std::lock_guard lock(shared->storage_policies_mutex);
if (shared->merge_tree_disk_selector)
shared->merge_tree_disk_selector
= shared->merge_tree_disk_selector->updateFromConfig(config, "storage_configuration.disks", shared_from_this());
if (shared->merge_tree_storage_policy_selector)
{
try
{
shared->merge_tree_storage_policy_selector = shared->merge_tree_storage_policy_selector->updateFromConfig(
config, "storage_configuration.policies", shared->merge_tree_disk_selector);
}
catch (Exception & e)
{
LOG_ERROR(
shared->log, "An error has occurred while reloading storage policies, storage policies were not applied: {}", e.message());
}
}
2020-08-03 01:44:58 +00:00
#if !defined(ARCADIA_BUILD)
if (shared->storage_s3_settings)
{
shared->storage_s3_settings->loadFromConfig("s3", config);
}
2020-08-03 01:44:58 +00:00
#endif
}
const MergeTreeSettings & Context::getMergeTreeSettings() const
2015-07-16 21:32:51 +00:00
{
auto lock = getLock();
2015-07-16 21:32:51 +00:00
if (!shared->merge_tree_settings)
{
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef();
2019-08-26 14:24:29 +00:00
MergeTreeSettings mt_settings;
mt_settings.loadFromConfig("merge_tree", config);
shared->merge_tree_settings.emplace(mt_settings);
}
2015-07-16 21:32:51 +00:00
return *shared->merge_tree_settings;
2015-07-16 21:32:51 +00:00
}
2020-08-10 11:23:58 +00:00
const MergeTreeSettings & Context::getReplicatedMergeTreeSettings() const
{
auto lock = getLock();
if (!shared->replicated_merge_tree_settings)
{
const auto & config = getConfigRef();
MergeTreeSettings mt_settings;
mt_settings.loadFromConfig("merge_tree", config);
mt_settings.loadFromConfig("replicated_merge_tree", config);
shared->replicated_merge_tree_settings.emplace(mt_settings);
}
return *shared->replicated_merge_tree_settings;
}
const StorageS3Settings & Context::getStorageS3Settings() const
{
2020-08-03 00:14:59 +00:00
#if !defined(ARCADIA_BUILD)
auto lock = getLock();
if (!shared->storage_s3_settings)
{
const auto & config = getConfigRef();
shared->storage_s3_settings.emplace().loadFromConfig("s3", config);
}
return *shared->storage_s3_settings;
2020-08-03 00:14:59 +00:00
#else
throw Exception("S3 is unavailable in Arcadia", ErrorCodes::NOT_IMPLEMENTED);
#endif
}
2015-07-16 21:32:51 +00:00
void Context::checkCanBeDropped(const String & database, const String & table, const size_t & size, const size_t & max_size_to_drop) const
{
if (!max_size_to_drop || size <= max_size_to_drop)
return;
2021-04-28 20:48:34 +00:00
fs::path force_file(getFlagsPath() + "force_drop_table");
bool force_file_exists = fs::exists(force_file);
if (force_file_exists)
{
try
{
2021-04-28 20:48:34 +00:00
fs::remove(force_file);
return;
}
catch (...)
{
/// User should recreate force file on each drop, it shouldn't be protected
tryLogCurrentException("Drop table check", "Can't remove force file to enable table or partition drop");
}
}
String size_str = formatReadableSizeWithDecimalSuffix(size);
String max_size_to_drop_str = formatReadableSizeWithDecimalSuffix(max_size_to_drop);
2020-11-10 18:22:26 +00:00
throw Exception(ErrorCodes::TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT,
"Table or Partition in {}.{} was not dropped.\nReason:\n"
"1. Size ({}) is greater than max_[table/partition]_size_to_drop ({})\n"
"2. File '{}' intended to force DROP {}\n"
"How to fix this:\n"
2020-12-03 16:51:27 +00:00
"1. Either increase (or set to zero) max_[table/partition]_size_to_drop in server config\n"
2020-11-10 18:22:26 +00:00
"2. Either create forcing file {} and make sure that ClickHouse has write permission for it.\n"
"Example:\nsudo touch '{}' && sudo chmod 666 '{}'",
backQuoteIfNeed(database), backQuoteIfNeed(table),
size_str, max_size_to_drop_str,
2021-04-28 20:48:34 +00:00
force_file.string(), force_file_exists ? "exists but not writeable (could not be removed)" : "doesn't exist",
force_file.string(),
force_file.string(), force_file.string());
}
void Context::setMaxTableSizeToDrop(size_t max_size)
{
2019-11-22 13:56:16 +00:00
// Is initialized at server startup and updated at config reload
shared->max_table_size_to_drop.store(max_size, std::memory_order_relaxed);
}
void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const
{
size_t max_table_size_to_drop = shared->max_table_size_to_drop.load(std::memory_order_relaxed);
2018-08-06 08:43:34 +00:00
checkCanBeDropped(database, table, table_size, max_table_size_to_drop);
}
void Context::setMaxPartitionSizeToDrop(size_t max_size)
{
2019-11-22 13:56:16 +00:00
// Is initialized at server startup and updated at config reload
shared->max_partition_size_to_drop.store(max_size, std::memory_order_relaxed);
}
void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const
{
size_t max_partition_size_to_drop = shared->max_partition_size_to_drop.load(std::memory_order_relaxed);
2018-08-06 08:43:34 +00:00
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
}
2021-10-11 16:11:50 +00:00
InputFormatPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size, const std::optional<FormatSettings> & format_settings) const
{
2021-10-11 16:11:50 +00:00
return FormatFactory::instance().getInput(name, buf, sample, shared_from_this(), max_block_size, format_settings);
}
2021-10-11 16:11:50 +00:00
OutputFormatPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const
{
2021-10-11 16:11:50 +00:00
return FormatFactory::instance().getOutputFormat(name, buf, sample, shared_from_this());
}
2020-12-30 03:07:30 +00:00
OutputFormatPtr Context::getOutputFormatParallelIfPossible(const String & name, WriteBuffer & buf, const Block & sample) const
{
return FormatFactory::instance().getOutputFormatParallelIfPossible(name, buf, sample, shared_from_this());
}
2016-04-09 07:47:08 +00:00
time_t Context::getUptimeSeconds() const
{
auto lock = getLock();
return shared->uptime_watch.elapsedSeconds();
2016-04-09 07:47:08 +00:00
}
void Context::setConfigReloadCallback(ConfigReloadCallback && callback)
{
/// Is initialized at server startup, so lock isn't required. Otherwise use mutex.
shared->config_reload_callback = std::move(callback);
}
void Context::reloadConfig() const
{
/// Use mutex if callback may be changed after startup.
if (!shared->config_reload_callback)
2020-06-27 19:05:00 +00:00
throw Exception("Can't reload config because config_reload_callback is not set.", ErrorCodes::LOGICAL_ERROR);
shared->config_reload_callback();
}
2015-04-16 06:12:35 +00:00
void Context::shutdown()
{
2020-10-26 16:09:53 +00:00
// Disk selector might not be initialized if there was some error during
// its initialization. Don't try to initialize it again on shutdown.
if (shared->merge_tree_disk_selector)
{
for (auto & [disk_name, disk] : getDisksMap())
{
LOG_INFO(shared->log, "Shutdown disk {}", disk_name);
disk->shutdown();
}
}
shared->shutdown();
2015-04-16 06:12:35 +00:00
}
Context::ApplicationType Context::getApplicationType() const
{
return shared->application_type;
}
void Context::setApplicationType(ApplicationType type)
{
/// Lock isn't required, you should set it at start
shared->application_type = type;
}
void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config)
{
shared->default_profile_name = config.getString("default_profile", "default");
getAccessControlManager().setDefaultProfileName(shared->default_profile_name);
shared->system_profile_name = config.getString("system_profile", shared->default_profile_name);
setCurrentProfile(shared->system_profile_name);
applySettingsQuirks(settings, &Poco::Logger::get("SettingsQuirks"));
shared->buffer_profile_name = config.getString("buffer_profile", shared->system_profile_name);
buffer_context = Context::createCopy(shared_from_this());
buffer_context->setCurrentProfile(shared->buffer_profile_name);
}
String Context::getDefaultProfileName() const
{
return shared->default_profile_name;
}
String Context::getSystemProfileName() const
{
return shared->system_profile_name;
}
String Context::getFormatSchemaPath() const
{
return shared->format_schema_path;
}
void Context::setFormatSchemaPath(const String & path)
{
shared->format_schema_path = path;
}
2018-07-05 23:36:09 +00:00
Context::SampleBlockCache & Context::getSampleBlockCache() const
{
assert(hasQueryContext());
return getQueryContext()->sample_block_cache;
}
2019-05-25 13:43:52 +00:00
bool Context::hasQueryParameters() const
2019-05-18 21:07:23 +00:00
{
2019-06-15 17:52:53 +00:00
return !query_parameters.empty();
2019-05-18 21:07:23 +00:00
}
2019-06-15 17:52:53 +00:00
const NameToNameMap & Context::getQueryParameters() const
2019-05-18 21:07:23 +00:00
{
2019-06-15 17:52:53 +00:00
return query_parameters;
2019-05-18 21:07:23 +00:00
}
2019-06-15 17:52:53 +00:00
void Context::setQueryParameter(const String & name, const String & value)
2019-05-18 21:07:23 +00:00
{
2019-06-15 17:52:53 +00:00
if (!query_parameters.emplace(name, value).second)
throw Exception("Duplicate name " + backQuote(name) + " of query parameter", ErrorCodes::BAD_ARGUMENTS);
2019-05-18 21:07:23 +00:00
}
2021-03-17 08:21:04 +00:00
void Context::addBridgeCommand(std::unique_ptr<ShellCommand> cmd) const
{
auto lock = getLock();
shared->bridge_commands.emplace_back(std::move(cmd));
}
IHostContextPtr & Context::getHostContext()
{
return host_context;
}
const IHostContextPtr & Context::getHostContext() const
{
return host_context;
}
std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()
{
auto lock = getLock();
if (!shared->action_locks_manager)
shared->action_locks_manager = std::make_shared<ActionLocksManager>(shared_from_this());
return shared->action_locks_manager;
}
void Context::setExternalTablesInitializer(ExternalTablesInitializer && initializer)
{
if (external_tables_initializer_callback)
throw Exception("External tables initializer is already set", ErrorCodes::LOGICAL_ERROR);
external_tables_initializer_callback = std::move(initializer);
}
void Context::initializeExternalTablesIfSet()
{
if (external_tables_initializer_callback)
{
external_tables_initializer_callback(shared_from_this());
/// Reset callback
external_tables_initializer_callback = {};
}
}
2019-05-28 18:30:10 +00:00
void Context::setInputInitializer(InputInitializer && initializer)
{
if (input_initializer_callback)
throw Exception("Input initializer is already set", ErrorCodes::LOGICAL_ERROR);
input_initializer_callback = std::move(initializer);
}
void Context::initializeInput(const StoragePtr & input_storage)
{
if (!input_initializer_callback)
throw Exception("Input initializer is not set", ErrorCodes::LOGICAL_ERROR);
input_initializer_callback(shared_from_this(), input_storage);
2019-05-28 18:30:10 +00:00
/// Reset callback
input_initializer_callback = {};
}
void Context::setInputBlocksReaderCallback(InputBlocksReader && reader)
{
if (input_blocks_reader)
throw Exception("Input blocks reader is already set", ErrorCodes::LOGICAL_ERROR);
input_blocks_reader = std::move(reader);
}
InputBlocksReader Context::getInputBlocksReaderCallback() const
{
return input_blocks_reader;
}
void Context::resetInputCallbacks()
{
if (input_initializer_callback)
input_initializer_callback = {};
if (input_blocks_reader)
input_blocks_reader = {};
}
2020-02-10 18:19:35 +00:00
StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where) const
2020-02-03 12:54:36 +00:00
{
2020-03-13 19:46:16 +00:00
if (storage_id.uuid != UUIDHelpers::Nil)
return storage_id;
2020-04-03 23:37:58 +00:00
StorageID resolved = StorageID::createEmpty();
2020-02-12 18:14:12 +00:00
std::optional<Exception> exc;
2020-04-03 23:37:58 +00:00
{
auto lock = getLock();
resolved = resolveStorageIDImpl(std::move(storage_id), where, &exc);
}
2020-02-12 18:14:12 +00:00
if (exc)
2020-03-13 15:41:36 +00:00
throw Exception(*exc);
2020-04-13 14:09:56 +00:00
if (!resolved.hasUUID() && resolved.database_name != DatabaseCatalog::TEMPORARY_DATABASE)
resolved.uuid = DatabaseCatalog::instance().getDatabase(resolved.database_name)->tryGetTableUUID(resolved.table_name);
return resolved;
2020-02-12 18:14:12 +00:00
}
StorageID Context::tryResolveStorageID(StorageID storage_id, StorageNamespace where) const
{
2020-03-13 19:46:16 +00:00
if (storage_id.uuid != UUIDHelpers::Nil)
return storage_id;
2020-04-03 23:37:58 +00:00
StorageID resolved = StorageID::createEmpty();
{
auto lock = getLock();
resolved = resolveStorageIDImpl(std::move(storage_id), where, nullptr);
}
2020-04-13 14:09:56 +00:00
if (resolved && !resolved.hasUUID() && resolved.database_name != DatabaseCatalog::TEMPORARY_DATABASE)
{
auto db = DatabaseCatalog::instance().tryGetDatabase(resolved.database_name);
if (db)
resolved.uuid = db->tryGetTableUUID(resolved.table_name);
}
return resolved;
2020-02-03 12:54:36 +00:00
}
2020-02-12 18:14:12 +00:00
StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace where, std::optional<Exception> * exception) const
{
if (storage_id.uuid != UUIDHelpers::Nil)
return storage_id;
2020-02-10 18:19:35 +00:00
2020-03-13 15:41:36 +00:00
if (!storage_id)
{
2020-02-12 18:14:12 +00:00
if (exception)
exception->emplace("Both table name and UUID are empty", ErrorCodes::UNKNOWN_TABLE);
return storage_id;
}
2020-02-12 18:14:12 +00:00
bool look_for_external_table = where & StorageNamespace::ResolveExternal;
bool in_current_database = where & StorageNamespace::ResolveCurrentDatabase;
bool in_specified_database = where & StorageNamespace::ResolveGlobal;
2020-02-10 18:19:35 +00:00
if (!storage_id.database_name.empty())
{
2020-02-10 18:19:35 +00:00
if (in_specified_database)
2020-03-13 15:41:36 +00:00
return storage_id; /// NOTE There is no guarantees that table actually exists in database.
2020-02-12 18:14:12 +00:00
if (exception)
exception->emplace("External and temporary tables have no database, but " +
2020-02-10 18:19:35 +00:00
storage_id.database_name + " is specified", ErrorCodes::UNKNOWN_TABLE);
2020-02-12 18:14:12 +00:00
return StorageID::createEmpty();
}
2020-03-13 15:41:36 +00:00
/// Database name is not specified. It's temporary table or table in current database.
2019-05-28 18:30:10 +00:00
2020-02-10 18:19:35 +00:00
if (look_for_external_table)
{
2020-03-03 19:53:18 +00:00
/// Global context should not contain temporary tables
assert(!isGlobalContext() || getApplicationType() == ApplicationType::LOCAL);
2020-03-03 19:53:18 +00:00
2020-03-11 19:10:55 +00:00
auto resolved_id = StorageID::createEmpty();
2021-06-01 12:20:52 +00:00
auto try_resolve = [&](ContextPtr context) -> bool
{
const auto & tables = context->external_tables_mapping;
2020-03-11 19:10:55 +00:00
auto it = tables.find(storage_id.getTableName());
if (it == tables.end())
return false;
resolved_id = it->second->getGlobalTableID();
return true;
};
2020-03-11 19:10:55 +00:00
/// Firstly look for temporary table in current context
if (try_resolve(shared_from_this()))
2020-03-11 19:10:55 +00:00
return resolved_id;
2020-03-11 19:10:55 +00:00
/// If not found and current context was created from some query context, look for temporary table in query context
auto query_context_ptr = query_context.lock();
bool is_local_context = query_context_ptr && query_context_ptr.get() != this;
if (is_local_context && try_resolve(query_context_ptr))
2020-03-11 19:10:55 +00:00
return resolved_id;
2020-03-11 19:10:55 +00:00
/// If not found and current context was created from some session context, look for temporary table in session context
auto session_context_ptr = session_context.lock();
bool is_local_or_query_context = session_context_ptr && session_context_ptr.get() != this;
if (is_local_or_query_context && try_resolve(session_context_ptr))
2020-03-11 19:10:55 +00:00
return resolved_id;
2020-02-10 18:19:35 +00:00
}
2020-03-13 15:41:36 +00:00
/// Temporary table not found. It's table in current database.
2020-02-10 18:19:35 +00:00
if (in_current_database)
{
if (current_database.empty())
2020-02-12 18:14:12 +00:00
{
if (exception)
exception->emplace("Default database is not selected", ErrorCodes::UNKNOWN_DATABASE);
return StorageID::createEmpty();
}
storage_id.database_name = current_database;
2020-03-13 15:41:36 +00:00
/// NOTE There is no guarantees that table actually exists in database.
2020-02-10 18:19:35 +00:00
return storage_id;
}
2020-02-12 18:14:12 +00:00
if (exception)
exception->emplace("Cannot resolve database name for table " + storage_id.getNameForLogs(), ErrorCodes::UNKNOWN_TABLE);
return StorageID::createEmpty();
}
2021-02-19 23:41:58 +00:00
void Context::initZooKeeperMetadataTransaction(ZooKeeperMetadataTransactionPtr txn, [[maybe_unused]] bool attach_existing)
2020-11-20 16:06:27 +00:00
{
assert(!metadata_transaction);
assert(attach_existing || query_context.lock().get() == this);
2020-11-20 16:06:27 +00:00
metadata_transaction = std::move(txn);
}
2021-02-19 23:41:58 +00:00
ZooKeeperMetadataTransactionPtr Context::getZooKeeperMetadataTransaction() const
2020-11-20 16:06:27 +00:00
{
2021-01-26 17:51:25 +00:00
assert(!metadata_transaction || hasQueryContext());
2020-11-20 16:06:27 +00:00
return metadata_transaction;
}
2021-07-02 16:39:55 +00:00
void Context::resetZooKeeperMetadataTransaction()
{
assert(metadata_transaction);
assert(hasQueryContext());
metadata_transaction = nullptr;
}
2021-05-31 14:49:02 +00:00
PartUUIDsPtr Context::getPartUUIDs() const
{
auto lock = getLock();
if (!part_uuids)
2021-05-31 14:49:02 +00:00
/// For context itself, only this initialization is not const.
/// We could have done in constructor.
/// TODO: probably, remove this from Context.
const_cast<PartUUIDsPtr &>(part_uuids) = std::make_shared<PartUUIDs>();
return part_uuids;
}
2021-04-08 19:00:39 +00:00
ReadTaskCallback Context::getReadTaskCallback() const
{
if (!next_task_callback.has_value())
throw Exception(fmt::format("Next task callback is not set for query {}", getInitialQueryId()), ErrorCodes::LOGICAL_ERROR);
return next_task_callback.value();
}
2021-04-08 19:00:39 +00:00
void Context::setReadTaskCallback(ReadTaskCallback && callback)
{
next_task_callback = callback;
}
2021-05-31 14:49:02 +00:00
PartUUIDsPtr Context::getIgnoredPartUUIDs() const
{
auto lock = getLock();
if (!ignored_part_uuids)
2021-05-31 14:49:02 +00:00
const_cast<PartUUIDsPtr &>(ignored_part_uuids) = std::make_shared<PartUUIDs>();
return ignored_part_uuids;
}
AsynchronousInsertQueue * Context::getAsynchronousInsertQueue() const
2021-03-17 14:11:47 +00:00
{
return shared->async_insert_queue.get();
2021-03-17 14:11:47 +00:00
}
2021-04-19 19:16:34 +00:00
void Context::setAsynchronousInsertQueue(const std::shared_ptr<AsynchronousInsertQueue> & ptr)
{
using namespace std::chrono;
2021-09-16 17:18:34 +00:00
if (std::chrono::milliseconds(settings.async_insert_busy_timeout_ms) == 0ms)
throw Exception("Setting async_insert_busy_timeout_ms can't be zero", ErrorCodes::INVALID_SETTING_VALUE);
2021-04-19 19:16:34 +00:00
shared->async_insert_queue = ptr;
}
2021-10-19 08:19:43 +00:00
void Context::initializeBackgroundExecutorsIfNeeded()
2021-08-30 19:37:03 +00:00
{
2021-10-19 08:19:43 +00:00
auto lock = getLock();
if (is_background_executors_initialized)
return;
const size_t max_merges_and_mutations = getSettingsRef().background_pool_size * getSettingsRef().background_merges_mutations_concurrency_ratio;
2021-09-08 00:21:21 +00:00
/// With this executor we can execute more tasks than threads we have
shared->merge_mutate_executor = MergeMutateBackgroundExecutor::create
2021-09-02 17:40:29 +00:00
(
"MergeMutate",
/*max_threads_count*/getSettingsRef().background_pool_size,
/*max_tasks_count*/max_merges_and_mutations,
CurrentMetrics::BackgroundMergesAndMutationsPoolTask
2021-09-02 17:40:29 +00:00
);
LOG_INFO(shared->log, "Initialized background executor for merges and mutations with num_threads={}, num_tasks={}",
getSettingsRef().background_pool_size, max_merges_and_mutations);
shared->moves_executor = OrdinaryBackgroundExecutor::create
2021-09-02 17:40:29 +00:00
(
"Move",
2021-09-08 12:42:03 +00:00
getSettingsRef().background_move_pool_size,
getSettingsRef().background_move_pool_size,
2021-09-02 17:40:29 +00:00
CurrentMetrics::BackgroundMovePoolTask
);
LOG_INFO(shared->log, "Initialized background executor for move operations with num_threads={}, num_tasks={}",
getSettingsRef().background_move_pool_size, getSettingsRef().background_move_pool_size);
2021-09-02 17:40:29 +00:00
shared->fetch_executor = OrdinaryBackgroundExecutor::create
2021-09-02 17:40:29 +00:00
(
"Fetch",
2021-09-08 12:42:03 +00:00
getSettingsRef().background_fetches_pool_size,
getSettingsRef().background_fetches_pool_size,
2021-09-02 17:40:29 +00:00
CurrentMetrics::BackgroundFetchesPoolTask
);
LOG_INFO(shared->log, "Initialized background executor for fetches with num_threads={}, num_tasks={}",
getSettingsRef().background_fetches_pool_size, getSettingsRef().background_fetches_pool_size);
shared->common_executor = OrdinaryBackgroundExecutor::create
(
"Common",
getSettingsRef().background_common_pool_size,
getSettingsRef().background_common_pool_size,
CurrentMetrics::BackgroundCommonPoolTask
);
LOG_INFO(shared->log, "Initialized background executor for common operations (e.g. clearing old parts) with num_threads={}, num_tasks={}",
getSettingsRef().background_common_pool_size, getSettingsRef().background_common_pool_size);
is_background_executors_initialized = true;
2021-08-30 19:37:03 +00:00
}
MergeMutateBackgroundExecutorPtr Context::getMergeMutateExecutor() const
2021-08-30 19:37:03 +00:00
{
2021-09-02 10:39:27 +00:00
return shared->merge_mutate_executor;
2021-08-30 19:37:03 +00:00
}
OrdinaryBackgroundExecutorPtr Context::getMovesExecutor() const
2021-08-30 19:37:03 +00:00
{
2021-09-02 10:39:27 +00:00
return shared->moves_executor;
2021-08-30 19:37:03 +00:00
}
OrdinaryBackgroundExecutorPtr Context::getFetchesExecutor() const
2021-08-30 19:37:03 +00:00
{
2021-09-02 10:39:27 +00:00
return shared->fetch_executor;
2021-08-30 19:37:03 +00:00
}
OrdinaryBackgroundExecutorPtr Context::getCommonExecutor() const
{
return shared->common_executor;
}
2021-08-30 19:37:03 +00:00
2021-08-24 21:45:58 +00:00
ReadSettings Context::getReadSettings() const
{
ReadSettings res;
2021-09-06 15:59:46 +00:00
std::string_view read_method_str = settings.local_filesystem_read_method.value;
if (auto opt_method = magic_enum::enum_cast<ReadMethod>(read_method_str))
res.local_fs_method = *opt_method;
else
throw Exception(ErrorCodes::UNKNOWN_READ_METHOD, "Unknown read method '{}'", read_method_str);
2021-08-24 21:45:58 +00:00
res.local_fs_prefetch = settings.local_filesystem_read_prefetch;
2021-08-25 22:14:42 +00:00
res.remote_fs_prefetch = settings.remote_filesystem_read_prefetch;
2021-09-21 06:40:59 +00:00
res.remote_fs_backoff_threshold = settings.remote_fs_read_backoff_threshold;
res.remote_fs_backoff_max_tries = settings.remote_fs_read_backoff_max_tries;
2021-08-24 21:45:58 +00:00
res.local_fs_buffer_size = settings.max_read_buffer_size;
res.direct_io_threshold = settings.min_bytes_to_use_direct_io;
res.mmap_threshold = settings.min_bytes_to_use_mmap_io;
res.priority = settings.read_priority;
res.mmap_cache = getMMappedFileCache().get();
return res;
}
2012-08-02 17:33:31 +00:00
}