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>
2017-05-24 19:31:50 +00:00
# include <Poco/UUID.h>
2017-01-21 04:24:28 +00:00
# include <Poco/Net/IPAddress.h>
2019-11-04 19:17:27 +00:00
# include <Poco/Util/Application.h>
2017-04-01 09:19:00 +00:00
# include <Common/Macros.h>
# include <Common/escapeForFileName.h>
2022-09-23 12:40:04 +00:00
# include <Common/EventNotifier.h>
2017-06-02 18:48:33 +00:00
# include <Common/setThreadName.h>
2017-04-01 09:19:00 +00:00
# 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>
2021-09-29 12:52:58 +00:00
# include <Common/getMultipleKeysFromConfig.h>
2021-08-24 12:30:31 +00:00
# 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>
2018-06-10 19:22:49 +00:00
# include <Formats/FormatFactory.h>
2017-10-13 01:02:16 +00:00
# include <Databases/IDatabase.h>
2023-07-26 17:58:41 +00:00
# include <Server/ServerType.h>
2017-04-01 09:19:00 +00:00
# include <Storages/IStorage.h>
# include <Storages/MarkCache.h>
# include <Storages/MergeTree/MergeList.h>
2022-10-25 13:56:46 +00:00
# include <Storages/MergeTree/MovesList.h>
2020-10-27 12:47:42 +00:00
# include <Storages/MergeTree/ReplicatedFetchList.h>
2020-10-26 12:40:55 +00:00
# include <Storages/MergeTree/MergeTreeData.h>
2017-04-01 09:19:00 +00:00
# include <Storages/MergeTree/MergeTreeSettings.h>
2018-12-21 12:17:30 +00:00
# include <Storages/CompressionCodecSelector.h>
2020-06-01 17:16:09 +00:00
# include <Storages/StorageS3Settings.h>
2020-01-19 14:26:28 +00:00
# include <Disks/DiskLocal.h>
2022-12-06 10:04:15 +00:00
# include <Disks/ObjectStorages/DiskObjectStorage.h>
2022-05-22 11:51:48 +00:00
# include <Disks/ObjectStorages/IObjectStorage.h>
2022-09-07 15:44:29 +00:00
# include <Disks/IO/ThreadPoolRemoteFSReader.h>
2022-09-23 17:35:16 +00:00
# include <Disks/IO/ThreadPoolReader.h>
2022-09-27 20:34:31 +00:00
# include <Disks/StoragePolicy.h>
2022-09-23 17:35:16 +00:00
# include <IO/SynchronousReader.h>
2018-03-01 01:49:36 +00:00
# include <TableFunctions/TableFunctionFactory.h>
2018-05-21 13:49:54 +00:00
# include <Interpreters/ActionLocksManager.h>
2021-04-01 10:19:28 +00:00
# include <Interpreters/ExternalLoaderXMLConfigRepository.h>
2022-08-31 17:17:31 +00:00
# include <Interpreters/TemporaryDataOnDisk.h>
2023-01-26 22:23:10 +00:00
# include <Interpreters/Cache/QueryCache.h>
2023-03-27 17:06:49 +00:00
# include <Interpreters/Cache/FileCacheFactory.h>
# include <Interpreters/Cache/FileCache.h>
Refactor reading the pool setting & from server config. (#48055)
After #36425 there was a lot of confusions/problems with configuring pools - when the message was confusing, and settings need to be ajusted in several places.
See some examples in #44251, #43351, #47900, #46515.
The commit includes the following changes:
1) Introduced a unified mechanism for reading pool sizes from the configuration file(s). Previously, pool sizes were read from the Context.cpp with fallbacks to profiles, whereas main_config_reloader in Server.cpp read them directly without fallbacks.
2) Corrected the data type for background_merges_mutations_concurrency_ratio. It should be float instead of int.
3) Refactored the default values for settings. Previously, they were defined in multiple places throughout the codebase, but they are now defined in one place (or two, to be exact: Settings.h and ServerSettings.h).
4) Improved documentation, including the correct message in system.settings.
Additionally make the code more conform with #46550.
2023-03-30 14:44:11 +00:00
# include <Core/ServerSettings.h>
2023-03-02 19:19:58 +00:00
# include <Interpreters/PreparedSets.h>
2019-03-22 12:08:30 +00:00
# include <Core/Settings.h>
2021-03-25 06:34:28 +00:00
# include <Core/SettingsQuirks.h>
2021-11-02 11:06:20 +00:00
# include <Access/AccessControl.h>
2020-03-07 17:37:38 +00:00
# include <Access/ContextAccess.h>
2020-04-29 19:35:56 +00:00
# include <Access/EnabledRolesInfo.h>
2020-03-07 17:37:38 +00:00
# include <Access/EnabledRowPolicies.h>
2020-05-08 12:50:45 +00:00
# include <Access/QuotaUsage.h>
2020-01-26 09:49:53 +00:00
# include <Access/User.h>
2021-03-11 20:41:10 +00:00
# include <Access/Credentials.h>
2020-03-04 22:27:03 +00:00
# include <Access/SettingsProfile.h>
2021-07-22 16:07:03 +00:00
# include <Access/SettingsProfilesInfo.h>
# include <Access/SettingsConstraintsAndProfileIDs.h>
2021-03-11 20:41:10 +00:00
# include <Access/ExternalAuthenticators.h>
# include <Access/GSSAcceptor.h>
2022-09-27 13:26:41 +00:00
# include <IO/ResourceManagerFactory.h>
2022-04-25 14:09:46 +00:00
# include <Backups/BackupsWorker.h>
2019-09-26 16:12:15 +00:00
# include <Dictionaries/Embedded/GeoDictionariesLoader.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/EmbeddedDictionaries.h>
2019-09-26 10:41:33 +00:00
# include <Interpreters/ExternalDictionariesLoader.h>
2022-09-24 21:24:39 +00:00
# include <Functions/UserDefined/ExternalUserDefinedExecutableFunctionsLoader.h>
# include <Functions/UserDefined/IUserDefinedSQLObjectsLoader.h>
# include <Functions/UserDefined/createUserDefinedSQLObjectsLoader.h>
2018-08-30 16:31:20 +00:00
# include <Interpreters/ExpressionActions.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/ProcessList.h>
2020-08-26 08:36:58 +00:00
# include <Interpreters/InterserverCredentials.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/Cluster.h>
# include <Interpreters/InterserverIOHandler.h>
2017-06-05 13:59:38 +00:00
# include <Interpreters/SystemLog.h>
2021-03-05 14:57:16 +00:00
# include <Interpreters/SessionLog.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/Context.h>
2019-02-21 16:41:27 +00:00
# include <Interpreters/DDLWorker.h>
2021-03-09 17:05:24 +00:00
# include <Interpreters/DDLTask.h>
2021-08-19 14:09:44 +00:00
# include <Interpreters/Session.h>
2022-01-10 19:35:42 +00:00
# include <Interpreters/TraceCollector.h>
2017-04-01 09:19:00 +00:00
# include <IO/ReadBufferFromFile.h>
# include <IO/UncompressedCache.h>
2021-03-28 19:24:28 +00:00
# include <IO/MMappedFileCache.h>
2022-03-21 08:52:48 +00:00
# include <IO/WriteSettings.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTCreateQuery.h>
# include <Parsers/ParserCreateQuery.h>
# include <Parsers/parseQuery.h>
2022-10-14 15:09:35 +00:00
# include <Parsers/ASTAsterisk.h>
# include <Parsers/ASTIdentifier.h>
2019-07-29 22:26:44 +00:00
# include <Common/StackTrace.h>
2022-08-09 17:53:32 +00:00
# include <Common/Config/ConfigHelper.h>
2018-02-28 20:34:25 +00:00
# include <Common/Config/ConfigProcessor.h>
2021-03-12 21:17:19 +00:00
# include <Common/Config/AbstractConfigurationComparison.h>
2017-06-19 20:06:35 +00:00
# include <Common/ZooKeeper/ZooKeeper.h>
2018-11-22 15:59:00 +00:00
# include <Common/ShellCommand.h>
2022-04-27 15:05:45 +00:00
# include <Common/logger_useful.h>
2021-10-02 07:13:14 +00:00
# include <base/EnumReflection.h>
2019-10-10 12:58:06 +00:00
# include <Common/RemoteHostFilter.h>
2023-06-15 13:49:49 +00:00
# include <Common/HTTPHeaderFilter.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>
2021-09-30 21:26:24 +00:00
# include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
2020-11-20 17:23:53 +00:00
# include <Storages/MergeTree/MergeTreeDataPartUUID.h>
2021-12-29 04:31:54 +00:00
# include <Storages/MergeTree/MergeTreeMetadataCache.h>
2021-06-19 17:57:46 +00:00
# include <Interpreters/SynonymsExtensions.h>
# include <Interpreters/Lemmatizers.h>
2021-11-11 09:03:53 +00:00
# include <Interpreters/ClusterDiscovery.h>
2022-01-19 18:29:31 +00:00
# include <Interpreters/TransactionLog.h>
2021-04-28 20:48:34 +00:00
# include <filesystem>
2022-08-23 15:46:11 +00:00
# include <re2/re2.h>
2022-09-23 11:35:22 +00:00
# include <Storages/StorageView.h>
# include <Parsers/ASTFunction.h>
2022-10-04 15:32:48 +00:00
# include <base/find_symbols.h>
2020-01-30 19:00:51 +00:00
2021-12-29 04:31:54 +00:00
# if USE_ROCKSDB
# include <rocksdb/table.h>
# endif
2014-08-22 01:01:28 +00:00
2021-04-28 20:48:34 +00:00
namespace fs = std : : filesystem ;
2016-12-07 22:49:42 +00:00
namespace ProfileEvents
{
extern const Event ContextLock ;
}
namespace CurrentMetrics
{
extern const Metric ContextLockWait ;
2019-12-12 22:08:54 +00:00
extern const Metric BackgroundMovePoolTask ;
2023-03-16 17:31:41 +00:00
extern const Metric BackgroundMovePoolSize ;
2020-04-22 23:00:24 +00:00
extern const Metric BackgroundSchedulePoolTask ;
2023-03-16 17:31:41 +00:00
extern const Metric BackgroundSchedulePoolSize ;
2020-04-22 23:00:24 +00:00
extern const Metric BackgroundBufferFlushSchedulePoolTask ;
2023-03-16 17:31:41 +00:00
extern const Metric BackgroundBufferFlushSchedulePoolSize ;
2020-04-22 23:00:24 +00:00
extern const Metric BackgroundDistributedSchedulePoolTask ;
2023-03-16 17:31:41 +00:00
extern const Metric BackgroundDistributedSchedulePoolSize ;
2020-08-31 11:06:35 +00:00
extern const Metric BackgroundMessageBrokerSchedulePoolTask ;
2023-03-16 17:31:41 +00:00
extern const Metric BackgroundMessageBrokerSchedulePoolSize ;
2021-09-30 21:26:24 +00:00
extern const Metric BackgroundMergesAndMutationsPoolTask ;
2023-03-16 17:31:41 +00:00
extern const Metric BackgroundMergesAndMutationsPoolSize ;
2021-08-30 19:37:03 +00:00
extern const Metric BackgroundFetchesPoolTask ;
2023-03-16 17:31:41 +00:00
extern const Metric BackgroundFetchesPoolSize ;
2021-09-30 21:26:24 +00:00
extern const Metric BackgroundCommonPoolTask ;
2023-03-16 17:31:41 +00:00
extern const Metric BackgroundCommonPoolSize ;
2023-03-30 18:08:38 +00:00
extern const Metric MarksLoaderThreads ;
extern const Metric MarksLoaderThreadsActive ;
extern const Metric IOPrefetchThreads ;
extern const Metric IOPrefetchThreadsActive ;
extern const Metric IOWriterThreads ;
extern const Metric IOWriterThreadsActive ;
2016-12-07 22:49:42 +00:00
}
2012-08-02 17:33:31 +00:00
namespace DB
{
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int BAD_ARGUMENTS ;
extern const int BAD_GET ;
2016-01-11 21:46:36 +00:00
extern const int UNKNOWN_DATABASE ;
extern const int UNKNOWN_TABLE ;
extern const int TABLE_ALREADY_EXISTS ;
extern const int THERE_IS_NO_SESSION ;
2018-03-02 05:44:17 +00:00
extern const int THERE_IS_NO_QUERY ;
2016-01-11 21:46:36 +00:00
extern const int NO_ELEMENTS_IN_CONFIG ;
2017-02-01 03:10:12 +00:00
extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT ;
2019-09-03 19:53:59 +00:00
extern const int LOGICAL_ERROR ;
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 ;
2022-07-09 05:37:13 +00:00
extern const int NOT_IMPLEMENTED ;
2022-11-16 11:27:08 +00:00
extern const int UNKNOWN_FUNCTION ;
2023-03-24 21:58:23 +00:00
extern const int ILLEGAL_COLUMN ;
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH ;
2016-01-11 21:46:36 +00:00
}
2015-04-16 06:12:35 +00:00
2023-07-07 14:40:47 +00:00
# define SHUTDOWN(log, desc, ptr, method) do \
{ \
if ( ptr ) \
{ \
LOG_DEBUG ( log , " Shutting down " desc ) ; \
2023-07-08 04:51:17 +00:00
( ptr ) - > method ; \
2023-07-07 14:40:47 +00:00
} \
} while ( false ) \
2015-04-16 06:12:35 +00:00
2016-11-23 20:41:39 +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
*/
2022-09-05 21:10:03 +00:00
struct ContextSharedPart : boost : : noncopyable
2015-04-16 06:12:35 +00:00
{
2020-05-30 21:57:37 +00:00
Poco : : Logger * log = & Poco : : Logger : : get ( " Context " ) ;
2017-04-01 07:20:54 +00:00
2016-11-23 20:41:39 +00:00
/// For access of most of shared objects. Recursive mutex.
2018-03-01 01:49:36 +00:00
mutable std : : recursive_mutex mutex ;
2017-01-21 04:24:28 +00:00
/// Separate mutex for access of dictionaries. Separate mutex to avoid locks when server doing request to itself.
mutable std : : mutex embedded_dictionaries_mutex ;
2019-06-02 12:11:01 +00:00
mutable std : : mutex external_dictionaries_mutex ;
2021-09-07 23:55:17 +00:00
mutable std : : mutex external_user_defined_executable_functions_mutex ;
2020-05-21 14:11:56 +00:00
/// 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)
2020-05-21 14:11:56 +00:00
/// 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.
2016-01-17 07:53:58 +00:00
mutable std : : mutex zookeeper_mutex ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
mutable zkutil : : ZooKeeperPtr zookeeper ; /// Client for ZooKeeper.
2020-11-03 07:52:31 +00:00
ConfigurationPtr zookeeper_config ; /// Stores zookeeper configs
2017-04-01 07:20:54 +00:00
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
2020-08-27 14:19:18 +00:00
mutable std : : mutex auxiliary_zookeepers_mutex ;
mutable std : : map < String , zkutil : : ZooKeeperPtr > auxiliary_zookeepers ; /// Map for auxiliary ZooKeeper clients.
2020-11-03 07:52:31 +00:00
ConfigurationPtr auxiliary_zookeepers_config ; /// Stores auxiliary zookeepers configs
2020-08-27 14:19:18 +00:00
2017-04-02 17:37:49 +00:00
String interserver_io_host ; /// The host name by which this server is available for other servers.
2017-09-07 21:04:48 +00:00
UInt16 interserver_io_port = 0 ; /// and port.
2018-07-30 18:32:21 +00:00
String interserver_scheme ; /// http or https
2021-04-07 13:52:11 +00:00
MultiVersion < InterserverCredentials > interserver_io_credentials ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
String path ; /// Path to the data directory, with a slash at the end.
2017-06-02 21:01:17 +00:00
String flags_path ; /// Path to the directory with some control flags for server maintenance.
2018-04-19 05:32:09 +00:00
String user_files_path ; /// Path to the directory with user provided files, usable by 'file' table function.
2019-12-10 17:27:29 +00:00
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.
2017-08-24 14:51:13 +00:00
ConfigurationPtr config ; /// Global configuration settings.
2020-01-19 14:26:28 +00:00
String tmp_path ; /// Path to the temporary files that occur when processing the request.
2023-04-05 10:07:44 +00:00
/// All temporary files that occur when processing the requests accounted here.
/// Child scopes for more fine-grained accounting are created per user/query/etc.
/// Initialized once during server startup.
TemporaryDataOnDiskScopePtr root_temp_data_on_disk ;
2020-01-19 14:26:28 +00:00
2022-03-06 10:23:50 +00:00
mutable std : : unique_ptr < EmbeddedDictionaries > embedded_dictionaries ; /// Metrica's dictionaries. Have lazy initialization.
mutable std : : unique_ptr < ExternalDictionariesLoader > external_dictionaries_loader ;
2021-09-29 12:52:58 +00:00
2021-06-15 19:55:21 +00:00
scope_guard models_repository_guard ;
2021-04-01 10:19:28 +00:00
2021-09-29 12:52:58 +00:00
ExternalLoaderXMLConfigRepository * external_dictionaries_config_repository = nullptr ;
2021-06-15 19:55:21 +00:00
scope_guard dictionaries_xmls ;
2021-09-29 12:52:58 +00:00
2022-09-24 21:24:39 +00:00
mutable std : : unique_ptr < ExternalUserDefinedExecutableFunctionsLoader > external_user_defined_executable_functions_loader ;
2021-09-29 12:52:58 +00:00
ExternalLoaderXMLConfigRepository * user_defined_executable_functions_config_repository = nullptr ;
2021-09-15 20:52:18 +00:00
scope_guard user_defined_executable_functions_xmls ;
2021-05-12 18:16:01 +00:00
2022-09-24 21:24:39 +00:00
mutable std : : unique_ptr < IUserDefinedSQLObjectsLoader > user_defined_sql_objects_loader ;
2021-07-30 13:30:30 +00:00
# if USE_NLP
2021-06-03 02:20:42 +00:00
mutable std : : optional < SynonymsExtensions > synonyms_extensions ;
2021-06-05 00:52:35 +00:00
mutable std : : optional < Lemmatizers > lemmatizers ;
2021-07-30 13:30:30 +00:00
# endif
2021-06-03 02:20:42 +00:00
2022-05-02 17:09:51 +00:00
std : : optional < BackupsWorker > backups_worker ;
2017-03-23 14:14:56 +00:00
String default_profile_name ; /// Default profile name used for default values.
2018-02-01 13:52:29 +00:00
String system_profile_name ; /// Profile used by system processes
2021-01-27 18:05:18 +00:00
String buffer_profile_name ; /// Profile used by Buffer engine for flushing to the underlying
2021-11-02 11:06:20 +00:00
std : : unique_ptr < AccessControl > access_control ;
2022-09-27 13:26:41 +00:00
mutable ResourceManagerPtr resource_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.
2022-09-05 18:12:40 +00:00
mutable std : : unique_ptr < ThreadPool > load_marks_threadpool ; /// Threadpool for loading marks cache.
2023-02-07 17:50:31 +00:00
mutable std : : unique_ptr < ThreadPool > prefetch_threadpool ; /// Threadpool for loading marks cache.
2021-08-21 17:27:22 +00:00
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.
2023-01-26 22:23:10 +00:00
mutable QueryCachePtr query_cache ; /// Cache of query results.
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.
2021-10-22 12:56:09 +00:00
GlobalOvercommitTracker global_overcommit_tracker ;
2017-04-02 17:37:49 +00:00
MergeList merge_list ; /// The list of executable merge (for (Replicated)?MergeTree)
2022-10-25 13:56:46 +00:00
MovesList moves_list ; /// The list of executing moves (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-24 11:38:02 +00:00
2022-03-06 10:23:50 +00:00
mutable std : : unique_ptr < BackgroundSchedulePool > buffer_flush_schedule_pool ; /// A thread pool that can do background flush for Buffer tables.
mutable std : : unique_ptr < BackgroundSchedulePool > schedule_pool ; /// A thread pool that can run different jobs in background (used in replicated tables)
mutable std : : unique_ptr < BackgroundSchedulePool > distributed_schedule_pool ; /// A thread pool that can run different jobs in background (used for distributed sends)
mutable std : : unique_ptr < 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
2022-09-23 17:35:16 +00:00
mutable std : : unique_ptr < IAsynchronousReader > asynchronous_remote_fs_reader ;
mutable std : : unique_ptr < IAsynchronousReader > asynchronous_local_fs_reader ;
mutable std : : unique_ptr < IAsynchronousReader > synchronous_local_fs_reader ;
2022-09-07 15:44:29 +00:00
mutable std : : unique_ptr < ThreadPool > threadpool_writer ;
2022-07-11 12:59:39 +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
2023-03-29 14:18:56 +00:00
2022-07-11 12:59:39 +00:00
mutable ThrottlerPtr remote_read_throttler ; /// A server-wide throttler for remote IO reads
mutable ThrottlerPtr remote_write_throttler ; /// A server-wide throttler for remote IO writes
2021-05-26 20:37:44 +00:00
2023-03-30 17:02:28 +00:00
mutable ThrottlerPtr local_read_throttler ; /// A server-wide throttler for local IO reads
mutable ThrottlerPtr local_write_throttler ; /// A server-wide throttler for local IO writes
2023-03-30 17:12:00 +00:00
mutable ThrottlerPtr backups_server_throttler ; /// A server-wide throttler for BACKUPs
2023-03-30 17:06:49 +00:00
2018-03-13 23:44:23 +00:00
MultiVersion < Macros > macros ; /// Substitutions extracted from config.
2019-02-21 16:41:27 +00:00
std : : unique_ptr < DDLWorker > ddl_worker ; /// Process ddl commands from zk.
2017-10-13 01:02:16 +00:00
/// 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 ;
2020-01-19 14:26:28 +00:00
/// Storage disk chooser for MergeTree engines
2020-01-09 14:50:34 +00:00
mutable std : : shared_ptr < const DiskSelector > merge_tree_disk_selector ;
2020-01-19 14:26:28 +00:00
/// Storage policy chooser for MergeTree engines
2020-01-09 14:50:34 +00:00
mutable std : : shared_ptr < const StoragePolicySelector > merge_tree_storage_policy_selector ;
2019-09-04 12:44:12 +00:00
2023-03-29 14:18:56 +00:00
ServerSettings server_settings ;
2019-11-20 16:40:27 +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.
2019-11-20 16:40:27 +00:00
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)
2017-11-10 06:48:28 +00:00
String format_schema_path ; /// Path to a directory that contains schema files used by input formats.
2018-05-21 13:49:54 +00:00
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
2020-06-01 17:16:09 +00:00
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.
2017-06-07 12:54:35 +00:00
2021-09-02 10:39:27 +00:00
/// Background executors for *MergeTree tables
2021-09-30 21:26:24 +00:00
MergeMutateBackgroundExecutorPtr merge_mutate_executor ;
OrdinaryBackgroundExecutorPtr moves_executor ;
OrdinaryBackgroundExecutorPtr fetch_executor ;
OrdinaryBackgroundExecutorPtr common_executor ;
2021-09-02 10:39:27 +00:00
2023-06-15 13:49:49 +00:00
RemoteHostFilter remote_host_filter ; /// Allowed URL from config.xml
HTTPHeaderFilter http_header_filter ; /// Forbidden HTTP headers from config.xml
2019-02-03 09:57:12 +00:00
2023-06-15 13:49:49 +00:00
std : : optional < TraceCollector > trace_collector ; /// Thread collecting traces from threads executing queries
2017-04-01 07:20:54 +00:00
2016-10-10 08:44:52 +00:00
/// Clusters for distributed tables
2016-10-14 15:06:46 +00:00
/// 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
2021-11-11 09:03:53 +00:00
std : : unique_ptr < ClusterDiscovery > cluster_discovery ;
2023-04-18 12:01:45 +00:00
mutable std : : mutex clusters_mutex ; /// Guards clusters, clusters_config and cluster_discovery
2017-04-01 07:20:54 +00:00
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
2015-04-16 06:12:35 +00:00
bool shutdown_called = false ;
2017-04-01 07:20:54 +00:00
2021-10-20 17:15:07 +00:00
/// Has background executors for MergeTree tables been initialized?
2022-04-19 15:01:41 +00:00
bool are_background_executors_initialized = false ;
2021-10-20 17:15:07 +00:00
2016-04-09 07:47:08 +00:00
Stopwatch uptime_watch ;
2017-04-01 07:20:54 +00:00
2016-11-11 17:01:02 +00:00
Context : : ApplicationType application_type = Context : : ApplicationType : : SERVER ;
2017-04-01 07:20:54 +00:00
2018-11-22 15:59:00 +00:00
/// vector of xdbc-bridge commands, they will be killed when Context will be destroyed
std : : vector < std : : unique_ptr < ShellCommand > > bridge_commands ;
2018-03-13 10:41:47 +00:00
Context : : ConfigReloadCallback config_reload_callback ;
2023-07-26 17:58:41 +00:00
Context : : StartStopServersCallback start_servers_callback ;
Context : : StartStopServersCallback stop_servers_callback ;
2022-03-14 20:43:34 +00:00
bool is_server_completely_started = false ;
2021-12-27 03:50:59 +00:00
# if USE_ROCKSDB
2021-12-28 11:29:01 +00:00
/// Global merge tree metadata cache, stored in rocksdb.
2021-12-28 10:06:13 +00:00
MergeTreeMetadataCachePtr merge_tree_metadata_cache ;
2021-12-27 03:50:59 +00:00
# endif
2021-12-08 02:40:59 +00:00
2021-04-10 23:33:54 +00:00
ContextSharedPart ( )
2021-10-22 12:56:09 +00:00
: access_control ( std : : make_unique < AccessControl > ( ) )
, global_overcommit_tracker ( & process_list )
, macros ( std : : make_unique < Macros > ( ) )
2017-06-19 20:35:53 +00:00
{
2017-07-10 04:34:14 +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 ( ) ;
}
}
2017-04-01 07:20:54 +00:00
2021-04-10 23:33:54 +00:00
~ ContextSharedPart ( )
2015-04-16 06:12:35 +00:00
{
2022-09-07 15:44:29 +00:00
/// Wait for thread pool for background reads and writes,
2022-03-14 05:21:38 +00:00
/// since it may use per-user MemoryTracker which will be destroyed here.
2022-09-23 17:35:16 +00:00
if ( asynchronous_remote_fs_reader )
2022-03-14 05:21:38 +00:00
{
2022-09-23 17:35:16 +00:00
try
2022-09-05 21:10:03 +00:00
{
2023-06-09 16:23:22 +00:00
LOG_DEBUG ( log , " Destructing remote fs threadpool reader " ) ;
2022-09-23 17:35:16 +00:00
asynchronous_remote_fs_reader - > wait ( ) ;
asynchronous_remote_fs_reader . reset ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
2022-09-05 21:10:03 +00:00
}
2022-03-14 05:21:38 +00:00
}
2022-09-23 17:35:16 +00:00
if ( asynchronous_local_fs_reader )
2022-03-14 05:21:38 +00:00
{
2022-09-23 17:35:16 +00:00
try
{
2023-06-09 16:23:22 +00:00
LOG_DEBUG ( log , " Destructing local fs threadpool reader " ) ;
2022-09-23 17:35:16 +00:00
asynchronous_local_fs_reader - > wait ( ) ;
asynchronous_local_fs_reader . reset ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
if ( synchronous_local_fs_reader )
2022-03-14 05:21:38 +00:00
{
2022-09-07 15:44:29 +00:00
try
{
2023-06-09 16:23:22 +00:00
LOG_DEBUG ( log , " Destructing local fs threadpool reader " ) ;
2022-09-23 17:35:16 +00:00
synchronous_local_fs_reader - > wait ( ) ;
synchronous_local_fs_reader . reset ( ) ;
2022-09-07 15:44:29 +00:00
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
2022-03-14 05:21:38 +00:00
}
2022-09-07 15:44:29 +00:00
if ( threadpool_writer )
2022-03-14 05:21:38 +00:00
{
2022-09-07 15:44:29 +00:00
try
{
2023-06-09 16:23:22 +00:00
LOG_DEBUG ( log , " Destructing threadpool writer " ) ;
2022-09-07 15:44:29 +00:00
threadpool_writer - > wait ( ) ;
threadpool_writer . reset ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
2022-03-14 05:21:38 +00:00
}
2022-09-13 14:28:36 +00:00
if ( load_marks_threadpool )
{
try
{
2023-06-09 16:23:22 +00:00
LOG_DEBUG ( log , " Destructing marks loader " ) ;
2022-09-13 14:28:36 +00:00
load_marks_threadpool - > wait ( ) ;
load_marks_threadpool . reset ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
2022-03-14 05:21:38 +00:00
}
2023-02-07 17:50:31 +00:00
if ( prefetch_threadpool )
{
try
{
2023-06-09 16:23:22 +00:00
LOG_DEBUG ( log , " Destructing prefetch threadpool " ) ;
2023-02-07 17:50:31 +00:00
prefetch_threadpool - > wait ( ) ;
prefetch_threadpool . reset ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
2015-04-16 06:12:35 +00:00
try
{
shutdown ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/** Perform a complex job of destroying objects in advance.
2015-04-16 06:12:35 +00:00
*/
void shutdown ( )
{
if ( shutdown_called )
return ;
shutdown_called = true ;
2017-04-01 07:20:54 +00:00
2021-11-21 19:39:36 +00:00
/// Stop periodic reloading of the configuration files.
/// This must be done first because otherwise the reloading may pass a changed config
/// to some destroyed parts of ContextSharedPart.
2023-07-07 14:40:47 +00:00
SHUTDOWN ( log , " dictionaries loader " , external_dictionaries_loader , enablePeriodicUpdates ( false ) ) ;
SHUTDOWN ( log , " UDFs loader " , external_user_defined_executable_functions_loader , enablePeriodicUpdates ( false ) ) ;
SHUTDOWN ( log , " another UDFs loader " , user_defined_sql_objects_loader , stopWatching ( ) ) ;
LOG_TRACE ( log , " Shutting down named sessions " ) ;
2021-08-19 14:09:44 +00:00
Session : : shutdownNamedSessions ( ) ;
2022-04-25 14:09:46 +00:00
/// Waiting for current backups/restores to be finished. This must be done before `DatabaseCatalog::shutdown()`.
2023-07-07 14:40:47 +00:00
SHUTDOWN ( log , " backups worker " , backups_worker , shutdown ( ) ) ;
2022-04-25 14:09:46 +00:00
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 .
*/
2023-07-07 14:40:47 +00:00
SHUTDOWN ( log , " system logs " , system_logs , shutdown ( ) ) ;
2019-02-04 14:08:39 +00:00
2023-07-07 14:40:47 +00:00
LOG_TRACE ( log , " Shutting down database catalog " ) ;
2020-04-08 23:59:39 +00:00
DatabaseCatalog : : shutdown ( ) ;
2019-02-04 23:18:04 +00:00
2023-07-07 14:40:47 +00:00
SHUTDOWN ( log , " merges executor " , merge_mutate_executor , wait ( ) ) ;
SHUTDOWN ( log , " fetches executor " , fetch_executor , wait ( ) ) ;
SHUTDOWN ( log , " moves executor " , moves_executor , wait ( ) ) ;
SHUTDOWN ( log , " common executor " , common_executor , wait ( ) ) ;
2021-09-02 10:39:27 +00:00
2022-01-19 18:29:31 +00:00
TransactionLog : : shutdownIfAny ( ) ;
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 ;
2022-03-06 10:23:50 +00:00
std : : unique_ptr < EmbeddedDictionaries > delete_embedded_dictionaries ;
std : : unique_ptr < ExternalDictionariesLoader > delete_external_dictionaries_loader ;
std : : unique_ptr < ExternalUserDefinedExecutableFunctionsLoader > delete_external_user_defined_executable_functions_loader ;
2022-09-24 21:24:39 +00:00
std : : unique_ptr < IUserDefinedSQLObjectsLoader > delete_user_defined_sql_objects_loader ;
2022-03-06 10:23:50 +00:00
std : : unique_ptr < BackgroundSchedulePool > delete_buffer_flush_schedule_pool ;
std : : unique_ptr < BackgroundSchedulePool > delete_schedule_pool ;
std : : unique_ptr < BackgroundSchedulePool > delete_distributed_schedule_pool ;
std : : unique_ptr < BackgroundSchedulePool > delete_message_broker_schedule_pool ;
std : : unique_ptr < DDLWorker > delete_ddl_worker ;
std : : unique_ptr < AccessControl > delete_access_control ;
2022-10-24 15:09:35 +00:00
/// Delete DDLWorker before zookeeper.
2022-10-24 15:51:21 +00:00
/// Cause it can call Context::getZooKeeper and resurrect it.
2022-10-24 15:09:35 +00:00
{
auto lock = std : : lock_guard ( mutex ) ;
delete_ddl_worker = std : : move ( ddl_worker ) ;
}
/// DDLWorker should be deleted without lock, cause its internal thread can
/// take it as well, which will cause deadlock.
2023-07-07 14:40:47 +00:00
LOG_TRACE ( log , " Shutting down DDLWorker " ) ;
2022-10-24 15:09:35 +00:00
delete_ddl_worker . reset ( ) ;
2023-03-28 09:55:55 +00:00
/// Background operations in cache use background schedule pool.
/// Deactivate them before destructing it.
2023-07-07 14:40:47 +00:00
LOG_TRACE ( log , " Shutting down caches " ) ;
2023-03-28 09:55:55 +00:00
const auto & caches = FileCacheFactory : : instance ( ) . getAll ( ) ;
for ( const auto & [ _ , cache ] : caches )
cache - > cache - > deactivateBackgroundOperations ( ) ;
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
{
auto lock = std : : lock_guard ( mutex ) ;
2022-03-06 10:23:50 +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.
2021-05-12 18:16:01 +00:00
/// 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 ( ) ;
2022-03-06 10:23:50 +00:00
models_repository_guard . reset ( ) ;
2021-05-12 18:16:01 +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
delete_system_logs = std : : move ( system_logs ) ;
2022-03-06 10:23:50 +00:00
delete_embedded_dictionaries = std : : move ( embedded_dictionaries ) ;
delete_external_dictionaries_loader = std : : move ( external_dictionaries_loader ) ;
delete_external_user_defined_executable_functions_loader = std : : move ( external_user_defined_executable_functions_loader ) ;
2022-09-24 21:24:39 +00:00
delete_user_defined_sql_objects_loader = std : : move ( user_defined_sql_objects_loader ) ;
2022-03-06 10:23:50 +00:00
delete_buffer_flush_schedule_pool = std : : move ( buffer_flush_schedule_pool ) ;
delete_schedule_pool = std : : move ( schedule_pool ) ;
delete_distributed_schedule_pool = std : : move ( distributed_schedule_pool ) ;
delete_message_broker_schedule_pool = std : : move ( message_broker_schedule_pool ) ;
delete_access_control = std : : move ( access_control ) ;
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 ( ) ;
2021-12-08 02:40:59 +00:00
2021-12-27 03:50:59 +00:00
# if USE_ROCKSDB
2021-12-28 11:29:01 +00:00
/// Shutdown merge tree metadata cache
2021-12-28 10:06:13 +00:00
if ( merge_tree_metadata_cache )
2021-12-08 02:40:59 +00:00
{
2021-12-28 10:06:13 +00:00
merge_tree_metadata_cache - > shutdown ( ) ;
merge_tree_metadata_cache . reset ( ) ;
2021-12-08 02:40:59 +00:00
}
2021-12-27 03:50:59 +00:00
# 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
}
2022-04-17 23:02:49 +00:00
/// Can be removed without context lock
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 . reset ( ) ;
2022-03-06 10:23:50 +00:00
delete_embedded_dictionaries . reset ( ) ;
delete_external_dictionaries_loader . reset ( ) ;
delete_external_user_defined_executable_functions_loader . reset ( ) ;
2022-09-24 21:24:39 +00:00
delete_user_defined_sql_objects_loader . reset ( ) ;
2022-03-06 10:23:50 +00:00
delete_ddl_worker . reset ( ) ;
delete_buffer_flush_schedule_pool . reset ( ) ;
delete_schedule_pool . reset ( ) ;
delete_distributed_schedule_pool . reset ( ) ;
delete_message_broker_schedule_pool . reset ( ) ;
delete_access_control . reset ( ) ;
2022-05-02 22:45:13 +00:00
total_memory_tracker . resetOvercommitTracker ( ) ;
2015-04-16 06:12:35 +00:00
}
2017-11-27 21:31:13 +00:00
2020-03-03 00:24:44 +00:00
bool hasTraceCollector ( ) const
2019-07-06 20:42:03 +00:00
{
2020-03-03 00:24:44 +00:00
return trace_collector . has_value ( ) ;
2019-07-06 20:29:00 +00:00
}
2019-05-19 20:22:44 +00:00
void initializeTraceCollector ( std : : shared_ptr < TraceLog > trace_log )
2019-02-09 22:40:47 +00:00
{
2020-11-11 18:29:05 +00:00
if ( ! trace_log )
return ;
2020-03-03 00:24:44 +00:00
if ( hasTraceCollector ( ) )
2019-07-06 20:29:00 +00:00
return ;
2020-03-03 00:24:44 +00:00
trace_collector . emplace ( std : : move ( trace_log ) ) ;
2017-11-27 21:31:13 +00:00
}
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
}
2023-05-11 14:59:19 +00:00
void configureServerWideThrottling ( )
{
if ( auto bandwidth = server_settings . max_replicated_fetches_network_bandwidth_for_server )
replicated_fetches_throttler = std : : make_shared < Throttler > ( bandwidth ) ;
if ( auto bandwidth = server_settings . max_replicated_sends_network_bandwidth_for_server )
replicated_sends_throttler = std : : make_shared < Throttler > ( bandwidth ) ;
if ( auto bandwidth = server_settings . max_remote_read_network_bandwidth_for_server )
remote_read_throttler = std : : make_shared < Throttler > ( bandwidth ) ;
if ( auto bandwidth = server_settings . max_remote_write_network_bandwidth_for_server )
remote_write_throttler = std : : make_shared < Throttler > ( bandwidth ) ;
if ( auto bandwidth = server_settings . max_local_read_bandwidth_for_server )
local_read_throttler = std : : make_shared < Throttler > ( bandwidth ) ;
if ( auto bandwidth = server_settings . max_local_write_bandwidth_for_server )
local_write_throttler = std : : make_shared < Throttler > ( bandwidth ) ;
if ( auto bandwidth = server_settings . max_backup_bandwidth_for_server )
backups_server_throttler = std : : make_shared < Throttler > ( bandwidth ) ;
}
2015-04-16 06:12:35 +00:00
} ;
2017-06-19 20:31:23 +00:00
Context : : Context ( ) = default ;
2019-04-24 22:52:08 +00:00
Context : : Context ( const Context & ) = default ;
Context & Context : : operator = ( const Context & ) = default ;
2017-06-19 20:31:23 +00:00
2020-04-17 09:47:40 +00:00
SharedContextHolder : : SharedContextHolder ( SharedContextHolder & & ) noexcept = default ;
2022-02-17 21:18:12 +00:00
SharedContextHolder & SharedContextHolder : : operator = ( SharedContextHolder & & ) noexcept = default ;
2020-04-19 15:41:35 +00:00
SharedContextHolder : : SharedContextHolder ( ) = default ;
2020-04-16 14:51:33 +00:00
SharedContextHolder : : ~ SharedContextHolder ( ) = default ;
2021-04-10 23:33:54 +00:00
SharedContextHolder : : SharedContextHolder ( std : : unique_ptr < ContextSharedPart > shared_context )
2020-04-17 09:47:40 +00:00
: shared ( std : : move ( shared_context ) ) { }
2017-06-19 20:31:23 +00:00
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
{
2021-04-10 23:33:54 +00:00
auto res = std : : shared_ptr < Context > ( new Context ) ;
res - > shared = shared ;
2017-06-19 20:31:23 +00:00
return res ;
2015-04-16 06:12:35 +00:00
}
2020-09-07 23:08:17 +00:00
void Context : : initGlobal ( )
{
2021-12-28 11:23:35 +00:00
assert ( ! global_context_instance ) ;
global_context_instance = shared_from_this ( ) ;
2021-04-10 23:33:54 +00:00
DatabaseCatalog : : init ( shared_from_this ( ) ) ;
2022-09-23 12:40:04 +00:00
EventNotifier : : init ( ) ;
2020-09-07 23:08:17 +00:00
}
2020-04-16 14:51:33 +00:00
SharedContextHolder Context : : createShared ( )
2020-04-16 14:37:38 +00:00
{
2021-04-10 23:33:54 +00:00
return SharedContextHolder ( std : : make_unique < ContextSharedPart > ( ) ) ;
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context : : createCopy ( const ContextPtr & other )
2021-04-10 23:33:54 +00:00
{
2023-05-08 13:22:31 +00:00
auto lock = other - > getLock ( ) ;
2021-04-10 23:33:54 +00:00
return std : : shared_ptr < Context > ( new Context ( * other ) ) ;
}
2021-06-01 12:20:52 +00:00
ContextMutablePtr Context : : createCopy ( const ContextWeakPtr & other )
2021-04-10 23:33:54 +00:00
{
auto ptr = other . lock ( ) ;
2023-05-08 13:22:31 +00:00
if ( ! ptr )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Can't copy an expired context " ) ;
2021-04-10 23:33:54 +00:00
return createCopy ( ptr ) ;
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context : : createCopy ( const ContextMutablePtr & other )
2021-04-10 23:33:54 +00:00
{
return createCopy ( std : : const_pointer_cast < const Context > ( other ) ) ;
}
2019-02-04 14:08:39 +00:00
Context : : ~ Context ( ) = default ;
2015-04-16 06:12:35 +00:00
2018-03-01 01:49:36 +00:00
InterserverIOHandler & Context : : getInterserverIOHandler ( ) { return shared - > interserver_io_handler ; }
2022-05-03 08:20:50 +00:00
const InterserverIOHandler & Context : : getInterserverIOHandler ( ) const { return shared - > interserver_io_handler ; }
2016-12-07 22:49:42 +00:00
2018-03-01 01:49:36 +00:00
std : : unique_lock < std : : recursive_mutex > Context : : getLock ( ) const
2016-12-07 22:49:42 +00:00
{
ProfileEvents : : increment ( ProfileEvents : : ContextLock ) ;
CurrentMetrics : : Increment increment { CurrentMetrics : : ContextLockWait } ;
2018-03-01 01:49:36 +00:00
return std : : unique_lock ( shared - > mutex ) ;
2016-12-07 22:49:42 +00:00
}
2018-03-01 01:49:36 +00:00
ProcessList & Context : : getProcessList ( ) { return shared - > process_list ; }
const ProcessList & Context : : getProcessList ( ) const { return shared - > process_list ; }
2021-10-22 12:56:09 +00:00
OvercommitTracker * Context : : getGlobalOvercommitTracker ( ) const { return & shared - > global_overcommit_tracker ; }
2018-03-01 01:49:36 +00:00
MergeList & Context : : getMergeList ( ) { return shared - > merge_list ; }
const MergeList & Context : : getMergeList ( ) const { return shared - > merge_list ; }
2022-10-25 13:56:46 +00:00
MovesList & Context : : getMovesList ( ) { return shared - > moves_list ; }
const MovesList & Context : : getMovesList ( ) const { return shared - > moves_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
2017-06-02 18:48:33 +00:00
{
2020-02-03 12:54:36 +00:00
String res = database_name . empty ( ) ? getCurrentDatabase ( ) : database_name ;
2017-01-05 16:34:05 +00:00
if ( res . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE , " Default database is not selected " ) ;
2017-01-05 16:34:05 +00:00
return res ;
}
2012-08-02 17:33:31 +00:00
String Context : : getPath ( ) const
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2012-08-02 17:33:31 +00:00
return shared - > path ;
}
2016-10-24 14:01:24 +00:00
String Context : : getFlagsPath ( ) const
{
auto lock = getLock ( ) ;
return shared - > flags_path ;
}
2018-04-19 05:32:09 +00:00
String Context : : getUserFilesPath ( ) const
{
auto lock = getLock ( ) ;
return shared - > user_files_path ;
}
2012-08-02 17:33:31 +00:00
2019-12-10 17:27:29 +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-11-10 15:45:15 +00:00
Strings Context : : getWarnings ( ) const
2021-07-09 09:27:51 +00:00
{
2021-11-10 15:45:15 +00:00
Strings common_warnings ;
{
auto lock = getLock ( ) ;
common_warnings = shared - > warnings ;
}
2023-06-24 03:06:20 +00:00
/// Make setting's name ordered
std : : set < String > obsolete_settings ;
2021-11-10 15:45:15 +00:00
for ( const auto & setting : settings )
{
if ( setting . isValueChanged ( ) & & setting . isObsolete ( ) )
2023-06-24 03:06:20 +00:00
obsolete_settings . emplace ( setting . getName ( ) ) ;
2021-11-10 15:45:15 +00:00
}
2023-06-12 14:21:58 +00:00
2023-06-24 03:06:20 +00:00
if ( ! obsolete_settings . empty ( ) )
2023-06-12 14:21:58 +00:00
{
2023-06-24 03:06:20 +00:00
bool single_element = obsolete_settings . size ( ) = = 1 ;
String res = single_element ? " Obsolete setting [ " : " Obsolete settings [ " ;
bool first = true ;
for ( const auto & setting : obsolete_settings )
2021-11-10 15:45:15 +00:00
{
2023-06-24 03:06:20 +00:00
res + = first ? " " : " , " ;
res + = " ' " + setting + " ' " ;
first = false ;
2021-11-10 15:45:15 +00:00
}
2023-06-24 03:06:20 +00:00
res = res + " ] " + ( single_element ? " is " : " are " )
2023-06-12 14:21:58 +00:00
+ " changed. "
" Please check 'select * from system.settings where changed and is_obsolete' and read the changelog. " ;
common_warnings . emplace_back ( res ) ;
2021-11-10 15:45:15 +00:00
}
2023-06-12 14:21:58 +00:00
2021-11-10 15:45:15 +00:00
return common_warnings ;
2021-07-09 09:27:51 +00:00
}
2022-08-31 17:17:31 +00:00
/// TODO: remove, use `getTempDataOnDisk`
2023-04-05 13:42:13 +00:00
VolumePtr Context : : getGlobalTemporaryVolume ( ) const
2020-01-19 14:26:28 +00:00
{
auto lock = getLock ( ) ;
2023-04-05 10:07:44 +00:00
/// Calling this method we just bypass the `temp_data_on_disk` and write to the file on the volume directly.
/// Volume is the same for `root_temp_data_on_disk` (always set) and `temp_data_on_disk` (if it's set).
if ( shared - > root_temp_data_on_disk )
return shared - > root_temp_data_on_disk - > getVolume ( ) ;
2022-08-31 17:17:31 +00:00
return nullptr ;
}
2022-09-15 10:19:39 +00:00
TemporaryDataOnDiskScopePtr Context : : getTempDataOnDisk ( ) const
2022-08-31 17:17:31 +00:00
{
if ( this - > temp_data_on_disk )
return this - > temp_data_on_disk ;
2023-03-30 11:59:51 +00:00
auto lock = getLock ( ) ;
2023-04-05 10:07:44 +00:00
return shared - > root_temp_data_on_disk ;
2022-08-31 17:17:31 +00:00
}
2023-04-05 10:07:44 +00:00
TemporaryDataOnDiskScopePtr Context : : getSharedTempDataOnDisk ( ) const
2020-01-19 14:26:28 +00:00
{
auto lock = getLock ( ) ;
2023-04-05 10:07:44 +00:00
return shared - > root_temp_data_on_disk ;
2022-08-31 17:17:31 +00:00
}
2022-09-15 10:19:39 +00:00
void Context : : setTempDataOnDisk ( TemporaryDataOnDiskScopePtr temp_data_on_disk_ )
2020-01-19 14:26:28 +00:00
{
2023-03-30 11:59:51 +00:00
/// It's set from `ProcessList::insert` in `executeQueryImpl` before query execution
/// so no races with `getTempDataOnDisk` which is called from query execution.
2022-09-01 12:22:49 +00:00
this - > temp_data_on_disk = std : : move ( temp_data_on_disk_ ) ;
2020-01-19 14:26:28 +00:00
}
2012-08-02 17:33:31 +00:00
void Context : : setPath ( const String & path )
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2018-04-19 05:32:09 +00:00
2012-08-02 17:33:31 +00:00
shared - > path = path ;
2018-04-19 05:32:09 +00:00
2023-04-05 10:07:44 +00:00
if ( shared - > tmp_path . empty ( ) & & ! shared - > root_temp_data_on_disk )
2018-04-19 05:32:09 +00:00
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/ " ;
2019-12-10 17:27:29 +00:00
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
}
2022-12-06 10:04:15 +00:00
static void setupTmpPath ( Poco : : Logger * log , const std : : string & path )
try
2015-01-07 17:19:23 +00:00
{
2022-12-06 10:04:15 +00:00
LOG_DEBUG ( log , " Setting up {} to store temporary data in it " , path ) ;
2020-01-19 14:26:28 +00:00
2022-12-06 10:04:15 +00:00
fs : : create_directories ( path ) ;
2020-01-19 14:26:28 +00:00
2022-12-06 10:04:15 +00:00
/// Clearing old temporary files.
fs : : directory_iterator dir_end ;
for ( fs : : directory_iterator it ( path ) ; it ! = dir_end ; + + it )
{
2022-12-26 12:51:59 +00:00
if ( it - > is_regular_file ( ) )
2022-12-06 10:04:15 +00:00
{
2022-12-26 12:51:59 +00:00
if ( startsWith ( it - > path ( ) . filename ( ) , " tmp " ) )
{
LOG_DEBUG ( log , " Removing old temporary file {} " , it - > path ( ) . string ( ) ) ;
fs : : remove ( it - > path ( ) ) ;
}
else
LOG_DEBUG ( log , " Found unknown file in temporary path {} " , it - > path ( ) . string ( ) ) ;
2022-12-06 10:04:15 +00:00
}
2022-12-26 12:51:59 +00:00
/// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types.
2020-01-19 14:26:28 +00:00
}
2022-12-06 10:04:15 +00:00
}
catch ( . . . )
{
DB : : tryLogCurrentException ( log , fmt : : format (
" Caught exception while setup temporary path: {}. "
" It is ok to skip this exception as cleaning old temporary files is not necessary " , path ) ) ;
}
2023-06-19 17:41:17 +00:00
static VolumePtr createLocalSingleDiskVolume ( const std : : string & path , const Poco : : Util : : AbstractConfiguration & config_ )
2022-12-06 10:04:15 +00:00
{
2023-06-19 17:41:17 +00:00
auto disk = std : : make_shared < DiskLocal > ( " _tmp_default " , path , 0 , config_ , " storage_configuration.disks._tmp_default " ) ;
2022-12-06 10:04:15 +00:00
VolumePtr volume = std : : make_shared < SingleDiskVolume > ( " _tmp_default " , disk , 0 ) ;
return volume ;
}
void Context : : setTemporaryStoragePath ( const String & path , size_t max_size )
{
2023-04-05 10:07:44 +00:00
auto lock = getLock ( ) ;
if ( shared - > root_temp_data_on_disk )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Temporary storage is already set " ) ;
2022-12-06 10:04:15 +00:00
shared - > tmp_path = path ;
if ( ! shared - > tmp_path . ends_with ( ' / ' ) )
shared - > tmp_path + = ' / ' ;
2023-06-19 17:41:17 +00:00
VolumePtr volume = createLocalSingleDiskVolume ( shared - > tmp_path , getConfigRef ( ) ) ;
2022-12-06 10:04:15 +00:00
for ( const auto & disk : volume - > getDisks ( ) )
2020-01-19 14:26:28 +00:00
{
2022-12-06 10:04:15 +00:00
setupTmpPath ( shared - > log , disk - > getPath ( ) ) ;
2020-01-19 14:26:28 +00:00
}
2023-04-05 10:07:44 +00:00
shared - > root_temp_data_on_disk = std : : make_shared < TemporaryDataOnDiskScope > ( volume , max_size ) ;
2022-12-06 10:04:15 +00:00
}
void Context : : setTemporaryStoragePolicy ( const String & policy_name , size_t max_size )
{
2023-04-05 13:42:49 +00:00
StoragePolicyPtr tmp_policy ;
{
/// lock in required only for accessing `shared->merge_tree_storage_policy_selector`
/// StoragePolicy itself is immutable.
std : : lock_guard storage_policies_lock ( shared - > storage_policies_mutex ) ;
tmp_policy = getStoragePolicySelector ( storage_policies_lock ) - > get ( policy_name ) ;
}
2022-12-06 10:04:15 +00:00
if ( tmp_policy - > getVolumes ( ) . size ( ) ! = 1 )
2023-04-05 13:42:49 +00:00
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG ,
2022-12-06 10:04:15 +00:00
" Policy '{}' is used temporary files, such policy should have exactly one volume " , policy_name ) ;
2023-04-05 13:42:49 +00:00
2022-12-06 10:04:15 +00:00
VolumePtr volume = tmp_policy - > getVolume ( 0 ) ;
2022-08-31 17:17:31 +00:00
if ( volume - > getDisks ( ) . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG , " No disks volume for temporary files " ) ;
2020-01-19 14:26:28 +00:00
2022-08-31 17:17:31 +00:00
for ( const auto & disk : volume - > getDisks ( ) )
2022-08-30 15:01:02 +00:00
{
2022-09-26 11:10:08 +00:00
if ( ! disk )
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG , " Temporary disk is null " ) ;
/// Check that underlying disk is local (can be wrapped in decorator)
2022-09-28 10:22:16 +00:00
DiskPtr disk_ptr = disk ;
2022-09-26 11:10:08 +00:00
2022-09-28 10:22:16 +00:00
if ( dynamic_cast < const DiskLocal * > ( disk_ptr . get ( ) ) = = nullptr )
2022-09-26 11:10:08 +00:00
{
2022-09-28 10:22:16 +00:00
const auto * disk_raw_ptr = disk_ptr . get ( ) ;
2022-08-30 15:01:02 +00:00
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG ,
2022-09-26 11:10:08 +00:00
" Disk '{}' ({}) is not local and can't be used for temporary files " ,
2022-09-28 10:22:16 +00:00
disk_ptr - > getName ( ) , typeid ( * disk_raw_ptr ) . name ( ) ) ;
2022-09-26 11:10:08 +00:00
}
2022-12-06 10:04:15 +00:00
setupTmpPath ( shared - > log , disk - > getPath ( ) ) ;
2022-08-30 15:01:02 +00:00
}
2023-04-05 13:42:49 +00:00
auto lock = getLock ( ) ;
2022-12-06 10:04:15 +00:00
2023-04-05 13:42:49 +00:00
if ( shared - > root_temp_data_on_disk )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Temporary storage is already set " ) ;
2022-12-06 10:04:15 +00:00
2023-04-05 10:07:44 +00:00
shared - > root_temp_data_on_disk = std : : make_shared < TemporaryDataOnDiskScope > ( volume , max_size ) ;
2022-12-06 10:04:15 +00:00
}
void Context : : setTemporaryStorageInCache ( const String & cache_disk_name , size_t max_size )
{
auto disk_ptr = getDisk ( cache_disk_name ) ;
if ( ! disk_ptr )
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG , " Disk '{}' is not found " , cache_disk_name ) ;
2023-04-10 17:56:52 +00:00
auto lock = getLock ( ) ;
if ( shared - > root_temp_data_on_disk )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Temporary storage is already set " ) ;
2023-04-15 11:08:49 +00:00
auto file_cache = FileCacheFactory : : instance ( ) . getByName ( disk_ptr - > getCacheName ( ) ) . cache ;
2022-12-06 10:04:15 +00:00
if ( ! file_cache )
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG , " Cache '{}' is not found " , file_cache - > getBasePath ( ) ) ;
LOG_DEBUG ( shared - > log , " Using file cache ({}) for temporary files " , file_cache - > getBasePath ( ) ) ;
shared - > tmp_path = file_cache - > getBasePath ( ) ;
2023-06-19 17:41:17 +00:00
VolumePtr volume = createLocalSingleDiskVolume ( shared - > tmp_path , getConfigRef ( ) ) ;
2023-04-05 10:07:44 +00:00
shared - > root_temp_data_on_disk = std : : make_shared < TemporaryDataOnDiskScope > ( volume , file_cache . get ( ) , max_size ) ;
2015-01-07 17:19:23 +00:00
}
2016-10-24 14:01:24 +00:00
void Context : : setFlagsPath ( const String & path )
{
auto lock = getLock ( ) ;
shared - > flags_path = path ;
}
2018-04-19 05:32:09 +00:00
void Context : : setUserFilesPath ( const String & path )
{
auto lock = getLock ( ) ;
shared - > user_files_path = path ;
}
2019-12-10 17:27:29 +00:00
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 ;
}
2022-05-18 19:57:20 +00:00
void Context : : addWarningMessage ( const String & msg ) const
2021-07-09 14:40:32 +00:00
{
auto lock = getLock ( ) ;
2022-08-24 15:48:26 +00:00
auto suppress_re = getConfigRef ( ) . getString ( " warning_supress_regexp " , " " ) ;
2022-08-23 15:46:11 +00:00
bool is_supressed = ! suppress_re . empty ( ) & & re2 : : RE2 : : PartialMatch ( msg , suppress_re ) ;
if ( ! is_supressed )
shared - > addWarningMessage ( msg ) ;
2021-07-09 14:40:32 +00:00
}
2017-08-24 14:51:13 +00:00
void Context : : setConfig ( const ConfigurationPtr & config )
{
auto lock = getLock ( ) ;
shared - > config = config ;
2021-11-02 11:06:20 +00:00
shared - > access_control - > setExternalAuthenticatorsConfig ( * shared - > config ) ;
2017-08-24 14:51:13 +00:00
}
2018-07-08 04:54:37 +00:00
const Poco : : Util : : AbstractConfiguration & Context : : getConfigRef ( ) const
2017-08-24 14:51:13 +00:00
{
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
2021-11-02 11:06:20 +00:00
AccessControl & Context : : getAccessControl ( )
2019-11-09 15:33:07 +00:00
{
2021-11-02 11:06:20 +00:00
return * shared - > access_control ;
2019-11-09 15:33:07 +00:00
}
2021-11-02 11:06:20 +00:00
const AccessControl & Context : : getAccessControl ( ) const
2019-11-09 15:33:07 +00:00
{
2021-11-02 11:06:20 +00:00
return * shared - > access_control ;
2019-11-09 15:33:07 +00:00
}
2020-06-10 22:48:15 +00:00
void Context : : setExternalAuthenticatorsConfig ( const Poco : : Util : : AbstractConfiguration & config )
{
auto lock = getLock ( ) ;
2021-11-02 11:06:20 +00:00
shared - > access_control - > setExternalAuthenticatorsConfig ( config ) ;
2020-06-10 22:48:15 +00:00
}
2019-11-29 17:22:56 +00:00
2021-03-11 20:41:10 +00:00
std : : unique_ptr < GSSAcceptorContext > Context : : makeGSSAcceptorContext ( ) const
{
auto lock = getLock ( ) ;
2021-11-02 11:06:20 +00:00
return std : : make_unique < GSSAcceptorContext > ( shared - > access_control - > getExternalAuthenticators ( ) . getKerberosParams ( ) ) ;
2021-03-11 20:41:10 +00:00
}
2016-10-14 15:06:46 +00:00
void Context : : setUsersConfig ( const ConfigurationPtr & config )
2013-08-10 07:46:45 +00:00
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2014-02-13 07:17:22 +00:00
shared - > users_config = config ;
2021-11-02 11:06:20 +00:00
shared - > access_control - > setUsersConfig ( * shared - > users_config ) ;
2014-02-13 07:17:22 +00:00
}
ConfigurationPtr Context : : getUsersConfig ( )
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2014-02-13 07:17:22 +00:00
return shared - > users_config ;
2013-08-10 07:46:45 +00:00
}
2023-07-01 17:58:38 +00:00
void Context : : setUser ( const UUID & user_id_ , bool set_current_profiles_ , bool set_current_roles_ , bool set_current_database_ )
2013-08-10 07:46:45 +00:00
{
2023-07-01 17:58:38 +00:00
/// Prepare lists of user's profiles, constraints, settings, roles.
2013-08-10 07:46:45 +00:00
2023-07-01 17:58:38 +00:00
std : : shared_ptr < const User > user ;
std : : shared_ptr < const ContextAccess > temp_access ;
if ( set_current_profiles_ | | set_current_roles_ | | set_current_database_ )
{
std : : optional < ContextAccessParams > params ;
{
auto lock = getLock ( ) ;
params . emplace ( ContextAccessParams { user_id_ , /* full_access= */ false , /* use_default_roles = */ true , { } , settings , current_database , client_info } ) ;
}
/// `temp_access` is used here only to extract information about the user, not to actually check access.
/// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this.
temp_access = getAccessControl ( ) . getContextAccess ( * params ) ;
user = temp_access - > getUser ( ) ;
}
std : : shared_ptr < const SettingsProfilesInfo > profiles ;
if ( set_current_profiles_ )
profiles = temp_access - > getDefaultProfileInfo ( ) ;
2020-06-15 02:17:08 +00:00
2023-07-01 17:58:38 +00:00
std : : optional < std : : vector < UUID > > roles ;
if ( set_current_roles_ )
roles = user - > granted_roles . findGranted ( user - > default_roles ) ;
2023-07-01 15:39:50 +00:00
2023-07-01 17:58:38 +00:00
String database ;
if ( set_current_database_ )
database = user - > default_database ;
2019-11-04 19:17:27 +00:00
2023-07-01 17:58:38 +00:00
/// Apply user's profiles, constraints, settings, roles.
auto lock = getLock ( ) ;
setUserID ( user_id_ ) ;
2022-05-05 10:17:13 +00:00
2023-07-01 17:58:38 +00:00
if ( profiles )
{
/// A profile can specify a value and a readonly constraint for same setting at the same time,
/// so we shouldn't check constraints here.
setCurrentProfiles ( * profiles , /* check_constraints= */ false ) ;
}
2019-11-04 19:17:27 +00:00
2023-07-01 17:58:38 +00:00
if ( roles )
setCurrentRoles ( * roles ) ;
2019-04-19 00:45:15 +00:00
2023-07-01 17:58:38 +00:00
if ( ! database . empty ( ) )
setCurrentDatabase ( database ) ;
2020-09-14 21:55:43 +00:00
}
2020-02-12 03:03:33 +00:00
std : : shared_ptr < const User > Context : : getUser ( ) const
2019-04-19 00:45:15 +00:00
{
2020-05-08 12:50:45 +00:00
return getAccess ( ) - > getUser ( ) ;
2017-06-07 12:54:35 +00:00
}
2017-06-05 17:54:40 +00:00
2020-02-12 03:03:33 +00:00
String Context : : getUserName ( ) const
2019-07-28 13:12:26 +00:00
{
2020-05-08 12:50:45 +00:00
return getAccess ( ) - > getUserName ( ) ;
2019-07-28 13:12:26 +00:00
}
2017-06-07 12:54:35 +00:00
2023-07-01 17:58:38 +00:00
void Context : : setUserID ( const UUID & user_id_ )
{
auto lock = getLock ( ) ;
user_id = user_id_ ;
need_recalculate_access = true ;
}
2020-04-05 23:03:20 +00:00
std : : optional < UUID > Context : : getUserID ( ) const
2017-06-07 12:54:35 +00:00
{
auto lock = getLock ( ) ;
2020-04-05 23:03:20 +00:00
return user_id ;
2013-08-10 07:46:45 +00:00
}
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_ ) ;
}
2021-02-26 22:37:00 +00:00
void Context : : setCurrentRoles ( const std : : vector < UUID > & current_roles_ )
2018-04-17 19:33:58 +00:00
{
auto lock = getLock ( ) ;
2023-07-01 17:58:38 +00:00
if ( current_roles_ . empty ( ) )
current_roles = nullptr ;
else
current_roles = std : : make_shared < std : : vector < UUID > > ( current_roles_ ) ;
2023-07-01 15:39:50 +00:00
need_recalculate_access = true ;
2018-04-17 19:33:58 +00:00
}
2020-02-21 00:17:07 +00:00
void Context : : setCurrentRolesDefault ( )
2018-08-13 09:11:58 +00:00
{
2021-07-22 13:44:48 +00:00
auto user = getUser ( ) ;
setCurrentRoles ( user - > granted_roles . findGranted ( user - > default_roles ) ) ;
2018-08-13 09:11:58 +00:00
}
2020-04-29 19:35:56 +00:00
boost : : container : : flat_set < UUID > Context : : getCurrentRoles ( ) const
2019-09-13 08:22:34 +00:00
{
2020-04-29 19:35:56 +00:00
return getRolesInfo ( ) - > current_roles ;
2019-09-13 08:22:34 +00:00
}
2020-04-29 19:35:56 +00:00
boost : : container : : flat_set < UUID > Context : : getEnabledRoles ( ) const
2015-10-01 15:10:41 +00:00
{
2020-04-29 19:35:56 +00:00
return getRolesInfo ( ) - > enabled_roles ;
2015-10-01 15:10:41 +00:00
}
2020-04-29 19:35:56 +00:00
std : : shared_ptr < const EnabledRolesInfo > Context : : getRolesInfo ( ) const
2013-11-08 17:43:03 +00:00
{
2020-04-29 19:35:56 +00:00
return getAccess ( ) - > getRolesInfo ( ) ;
2019-05-09 06:12:02 +00:00
}
2013-11-08 17:43:03 +00:00
2020-01-12 21:00:55 +00:00
template < typename . . . Args >
void Context : : checkAccessImpl ( const Args & . . . args ) const
{
2020-03-07 17:37:38 +00:00
return getAccess ( ) - > checkAccess ( args . . . ) ;
2020-01-12 21:00:55 +00:00
}
2014-12-23 20:32:00 +00:00
2020-03-07 17:37:38 +00:00
void Context : : checkAccess ( const AccessFlags & flags ) const { return checkAccessImpl ( flags ) ; }
2022-07-14 16:11:35 +00:00
void Context : : checkAccess ( const AccessFlags & flags , std : : string_view database ) const { return checkAccessImpl ( flags , database ) ; }
void Context : : checkAccess ( const AccessFlags & flags , std : : string_view database , std : : string_view table ) const { return checkAccessImpl ( flags , database , table ) ; }
void Context : : checkAccess ( const AccessFlags & flags , std : : string_view database , std : : string_view table , std : : string_view column ) const { return checkAccessImpl ( flags , database , table , column ) ; }
void Context : : checkAccess ( const AccessFlags & flags , std : : string_view database , std : : string_view table , const std : : vector < std : : string_view > & columns ) const { return checkAccessImpl ( flags , database , table , columns ) ; }
void Context : : checkAccess ( const AccessFlags & flags , std : : string_view database , std : : string_view table , const Strings & columns ) const { return checkAccessImpl ( flags , database , table , columns ) ; }
2020-03-07 17:37:38 +00:00
void Context : : checkAccess ( const AccessFlags & flags , const StorageID & table_id ) const { checkAccessImpl ( flags , table_id . getDatabaseName ( ) , table_id . getTableName ( ) ) ; }
2022-07-14 16:11:35 +00:00
void Context : : checkAccess ( const AccessFlags & flags , const StorageID & table_id , std : : string_view column ) const { checkAccessImpl ( flags , table_id . getDatabaseName ( ) , table_id . getTableName ( ) , column ) ; }
2020-03-07 17:37:38 +00:00
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 ) ; }
2014-12-23 20:32:00 +00:00
2020-03-07 17:37:38 +00:00
std : : shared_ptr < const ContextAccess > Context : : getAccess ( ) const
2012-08-02 17:33:31 +00:00
{
2023-07-01 15:39:50 +00:00
/// A helper function to collect parameters for calculating access rights, called with Context::getLock() acquired.
auto get_params = [ this ] ( )
{
/// If setUserID() was never called then this must be the global context with the full access.
bool full_access = ! user_id ;
return ContextAccessParams { user_id , full_access , /* use_default_roles= */ false , current_roles , settings , current_database , client_info } ;
} ;
/// Check if the current access rights are still valid, otherwise get parameters for recalculating access rights.
std : : optional < ContextAccessParams > params ;
{
auto lock = getLock ( ) ;
if ( access & & ! need_recalculate_access )
return access ; /// No need to recalculate access rights.
params . emplace ( get_params ( ) ) ;
if ( access & & ( access - > getParams ( ) = = * params ) )
{
need_recalculate_access = false ;
return access ; /// No need to recalculate access rights.
}
}
/// Calculate new access rights according to the collected parameters.
/// NOTE: AccessControl::getContextAccess() may require some IO work, so Context::getLock() must be unlocked while we're doing this.
auto res = getAccessControl ( ) . getContextAccess ( * params ) ;
{
/// If the parameters of access rights were not changed while we were calculated them
/// then we store the new access rights in the Context to allow reusing it later.
auto lock = getLock ( ) ;
if ( get_params ( ) = = * params )
{
access = res ;
need_recalculate_access = false ;
}
}
return res ;
2020-02-07 19:31:09 +00:00
}
2012-08-02 17:33:31 +00:00
2022-10-24 07:58:14 +00:00
RowPolicyFilterPtr Context : : getRowPolicyFilter ( const String & database , const String & table_name , RowPolicyFilterType filter_type ) const
2020-02-12 03:03:33 +00:00
{
2023-07-01 17:58:38 +00:00
return getAccess ( ) - > getRowPolicyFilter ( database , table_name , filter_type ) ;
2019-10-11 13:21:52 +00:00
}
2014-06-26 00:58:14 +00:00
2020-02-12 03:03:33 +00:00
2020-03-07 17:37:38 +00:00
std : : shared_ptr < const EnabledQuota > Context : : getQuota ( ) const
2019-10-11 13:21:52 +00:00
{
2020-03-07 17:37:38 +00:00
return getAccess ( ) - > getQuota ( ) ;
2012-08-02 17:33:31 +00:00
}
2020-05-08 12:50:45 +00:00
std : : optional < QuotaUsage > Context : : getQuotaUsage ( ) const
{
return getAccess ( ) - > getQuotaUsage ( ) ;
}
2023-07-01 17:58:38 +00:00
void Context : : setCurrentProfile ( const String & profile_name , bool check_constraints )
2012-08-02 17:33:31 +00:00
{
2020-12-24 17:48:54 +00:00
try
{
2021-11-02 11:06:20 +00:00
UUID profile_id = getAccessControl ( ) . getID < SettingsProfile > ( profile_name ) ;
2023-07-01 17:58:38 +00:00
setCurrentProfile ( profile_id , check_constraints ) ;
2020-12-24 17:48:54 +00:00
}
catch ( Exception & e )
{
e . addMessage ( " , while trying to set settings profile {} " , profile_name ) ;
throw ;
}
2021-07-22 16:07:03 +00:00
}
2023-07-01 17:58:38 +00:00
void Context : : setCurrentProfile ( const UUID & profile_id , bool check_constraints )
2021-07-22 16:07:03 +00:00
{
2021-11-02 11:06:20 +00:00
auto profile_info = getAccessControl ( ) . getSettingsProfileInfo ( profile_id ) ;
2023-07-01 17:58:38 +00:00
setCurrentProfiles ( * profile_info , check_constraints ) ;
2021-07-22 16:07:03 +00:00
}
2023-07-01 17:58:38 +00:00
void Context : : setCurrentProfiles ( const SettingsProfilesInfo & profiles_info , bool check_constraints )
{
auto lock = getLock ( ) ;
if ( check_constraints )
checkSettingsConstraints ( profiles_info . settings ) ;
applySettingsChanges ( profiles_info . settings ) ;
settings_constraints_and_current_profiles = profiles_info . getConstraintsAndProfileIDs ( settings_constraints_and_current_profiles ) ;
}
2021-07-22 16:07:03 +00:00
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
}
2022-09-27 13:26:41 +00:00
ResourceManagerPtr Context : : getResourceManager ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > resource_manager )
2023-05-15 21:13:17 +00:00
shared - > resource_manager = ResourceManagerFactory : : instance ( ) . get ( getConfigRef ( ) . getString ( " resource_manager " , " dynamic " ) ) ;
2022-09-27 13:26:41 +00:00
return shared - > resource_manager ;
}
ClassifierPtr Context : : getClassifier ( ) const
{
auto lock = getLock ( ) ;
return getResourceManager ( ) - > acquire ( getSettingsRef ( ) . workload ) ;
}
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.
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " Scalar {} doesn't exist (internal bug) " , backQuoteIfNeed(name)) ;
2020-06-06 01:23:21 +00:00
}
2019-10-19 20:36:35 +00:00
return it - > second ;
}
2022-04-06 06:53:10 +00:00
const Block * Context : : tryGetSpecialScalar ( const String & name ) const
2021-07-31 07:45:26 +00:00
{
2022-04-06 06:53:10 +00:00
auto it = special_scalars . find ( name ) ;
if ( special_scalars . end ( ) = = it )
2021-07-31 07:45:26 +00:00
return nullptr ;
return & it - > second ;
}
2019-10-19 20:36:35 +00:00
2014-03-13 15:00:06 +00:00
Tables Context : : getExternalTables ( ) const
{
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have external tables " ) ;
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2014-03-14 15:42:30 +00:00
2018-02-14 04:00:37 +00:00
Tables res ;
2020-04-22 06:01:33 +00:00
for ( const auto & table : external_tables_mapping )
2020-01-30 19:00:51 +00:00
res [ table . first ] = table . second - > getTable ( ) ;
2018-02-14 04:00:37 +00:00
2021-04-10 23:33:54 +00:00
auto query_context_ptr = query_context . lock ( ) ;
auto session_context_ptr = session_context . lock ( ) ;
if ( query_context_ptr & & query_context_ptr . get ( ) ! = this )
2014-03-13 15:00:06 +00:00
{
2021-04-10 23:33:54 +00:00
Tables buf = query_context_ptr - > getExternalTables ( ) ;
2014-03-13 15:00:06 +00:00
res . insert ( buf . begin ( ) , buf . end ( ) ) ;
}
2021-04-10 23:33:54 +00:00
else if ( session_context_ptr & & session_context_ptr . get ( ) ! = this )
2014-03-13 15:00:06 +00:00
{
2021-04-10 23:33:54 +00:00
Tables buf = session_context_ptr - > getExternalTables ( ) ;
2014-03-13 15:00:06 +00:00
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
{
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have external tables " ) ;
2020-02-13 21:00:03 +00:00
auto lock = getLock ( ) ;
2020-01-30 19:00:51 +00:00
if ( external_tables_mapping . end ( ) ! = external_tables_mapping . find ( table_name ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : TABLE_ALREADY_EXISTS , " Temporary table {} already exists. " , backQuoteIfNeed ( table_name ) ) ;
2020-03-10 19:36:17 +00:00
external_tables_mapping . emplace ( table_name , std : : make_shared < TemporaryTableHolder > ( std : : move ( temporary_table ) ) ) ;
2015-08-19 21:15:27 +00:00
}
2023-05-25 19:18:11 +00:00
std : : shared_ptr < TemporaryTableHolder > Context : : findExternalTable ( const String & table_name ) const
{
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have external tables " ) ;
std : : shared_ptr < TemporaryTableHolder > holder ;
{
auto lock = getLock ( ) ;
auto iter = external_tables_mapping . find ( table_name ) ;
if ( iter = = external_tables_mapping . end ( ) )
return { } ;
holder = iter - > second ;
}
return holder ;
}
2015-08-19 21:15:27 +00:00
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
{
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have external tables " ) ;
2020-02-17 13:52:59 +00:00
std : : shared_ptr < TemporaryTableHolder > holder ;
2014-03-06 14:02:20 +00:00
{
2019-06-02 12:11:01 +00:00
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 ( ) )
2019-06-02 12:11:01 +00:00
return { } ;
2020-02-17 13:52:59 +00:00
holder = iter - > second ;
external_tables_mapping . erase ( iter ) ;
2015-08-19 21:15:27 +00:00
}
2020-03-13 15:41:36 +00:00
return holder ;
2014-03-04 15:31:56 +00:00
}
2019-10-19 20:36:35 +00:00
void Context : : addScalar ( const String & name , const Block & block )
{
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have scalars " ) ;
2019-10-19 20:36:35 +00:00
scalars [ name ] = block ;
}
2022-04-06 06:53:10 +00:00
void Context : : addSpecialScalar ( const String & name , const Block & block )
2021-07-31 07:45:26 +00:00
{
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have local scalars " ) ;
2022-04-06 06:53:10 +00:00
special_scalars [ name ] = block ;
2021-07-31 07:45:26 +00:00
}
2019-10-19 20:36:35 +00:00
bool Context : : hasScalar ( const String & name ) const
{
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have scalars " ) ;
2022-04-18 10:18:43 +00:00
return scalars . contains ( name ) ;
2019-10-19 20:36:35 +00:00
}
2021-02-10 14:12:49 +00:00
void Context : : addQueryAccessInfo (
2021-06-24 16:54:04 +00:00
const String & quoted_database_name ,
const String & full_quoted_table_name ,
const Names & column_names ,
const String & projection_name ,
const String & view_name )
2020-12-02 12:08:03 +00:00
{
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have query access info " ) ;
2023-03-02 13:36:47 +00:00
std : : lock_guard 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 ) ;
2020-12-02 12:08:03 +00:00
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 ) ) ;
2021-02-10 14:12:49 +00:00
if ( ! projection_name . empty ( ) )
query_access_info . projections . emplace ( full_quoted_table_name + " . " + backQuoteIfNeed ( projection_name ) ) ;
2021-06-24 16:54:04 +00:00
if ( ! view_name . empty ( ) )
query_access_info . views . emplace ( view_name ) ;
2020-12-02 12:08:03 +00:00
}
2023-05-08 10:01:24 +00:00
void Context : : addQueryAccessInfo ( const Names & partition_names )
{
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have query access info " ) ;
std : : lock_guard < std : : mutex > lock ( query_access_info . mutex ) ;
for ( const auto & partition_name : partition_names )
query_access_info . partitions . emplace ( partition_name ) ;
2023-07-19 13:28:17 +00:00
}
2023-07-20 03:12:22 +00:00
void Context : : addQueryAccessInfo ( const QualifiedProjectionName & qualified_projection_name )
2023-07-19 13:28:17 +00:00
{
2023-07-20 03:12:22 +00:00
if ( ! qualified_projection_name )
return ;
2023-07-19 13:28:17 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have query access info " ) ;
std : : lock_guard < std : : mutex > lock ( query_access_info . mutex ) ;
2023-07-20 03:12:22 +00:00
query_access_info . projections . emplace ( fmt : : format (
" {}.{} " , qualified_projection_name . storage_id . getFullTableName ( ) , backQuoteIfNeed ( qualified_projection_name . projection_name ) ) ) ;
2023-05-08 10:01:24 +00:00
}
2021-01-21 21:15:11 +00:00
void Context : : addQueryFactoriesInfo ( QueryLogFactories factory_type , const String & created_object ) const
2021-01-20 10:54:11 +00:00
{
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Global context cannot have query factories info " ) ;
2022-05-25 15:15:23 +00:00
std : : lock_guard lock ( query_factories_info . mutex ) ;
2021-01-20 10:54:11 +00:00
2021-01-21 21:15:11 +00:00
switch ( factory_type )
{
case QueryLogFactories : : AggregateFunction :
query_factories_info . aggregate_functions . emplace ( created_object ) ;
break ;
2021-01-23 12:07:37 +00:00
case QueryLogFactories : : AggregateFunctionCombinator :
query_factories_info . aggregate_function_combinators . emplace ( created_object ) ;
break ;
2021-01-21 21:15:11 +00:00
case QueryLogFactories : : Database :
2021-01-23 12:07:37 +00:00
query_factories_info . database_engines . emplace ( created_object ) ;
2021-01-21 21:15:11 +00:00
break ;
case QueryLogFactories : : DataType :
2021-01-23 12:07:37 +00:00
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 ) ;
}
2021-01-20 10:54:11 +00:00
}
2023-03-31 21:37:18 +00:00
static bool findIdentifier ( const ASTFunction * function )
{
if ( ! function | | ! function - > arguments )
return false ;
if ( const auto * arguments = function - > arguments - > as < ASTExpressionList > ( ) )
{
for ( const auto & argument : arguments - > children )
{
if ( argument - > as < ASTIdentifier > ( ) )
return true ;
if ( const auto * f = argument - > as < ASTFunction > ( ) ; f & & findIdentifier ( f ) )
return true ;
}
}
return false ;
}
2021-01-20 10:54:11 +00:00
2022-10-14 15:09:35 +00:00
StoragePtr Context : : executeTableFunction ( const ASTPtr & table_expression , const ASTSelectQuery * select_query_hint )
2018-03-01 01:49:36 +00:00
{
2022-11-07 14:22:45 +00:00
ASTFunction * function = assert_cast < ASTFunction * > ( table_expression . get ( ) ) ;
2022-10-17 16:11:22 +00:00
String database_name = getCurrentDatabase ( ) ;
String table_name = function - > name ;
2022-11-16 11:27:08 +00:00
if ( function - > is_compound_name )
2018-03-01 01:49:36 +00:00
{
2022-10-17 16:11:22 +00:00
std : : vector < std : : string > parts ;
splitInto < ' . ' > ( parts , function - > name ) ;
2022-10-04 15:32:48 +00:00
2022-10-17 16:11:22 +00:00
if ( parts . size ( ) = = 2 )
2022-10-04 15:32:48 +00:00
{
2022-10-17 16:11:22 +00:00
database_name = parts [ 0 ] ;
table_name = parts [ 1 ] ;
2022-10-04 15:32:48 +00:00
}
2022-10-17 16:11:22 +00:00
}
2022-10-04 15:32:48 +00:00
2022-10-17 16:11:22 +00:00
StoragePtr table = DatabaseCatalog : : instance ( ) . tryGetTable ( { database_name , table_name } , getQueryContext ( ) ) ;
if ( table )
{
2023-06-22 12:47:04 +00:00
if ( table . get ( ) - > isView ( ) & & table - > as < StorageView > ( ) & & table - > as < StorageView > ( ) - > isParameterizedView ( ) )
2022-11-07 14:22:45 +00:00
{
2022-11-16 11:27:08 +00:00
function - > prefer_subquery_to_function_formatting = true ;
2022-10-17 16:11:22 +00:00
return table ;
2022-11-07 14:22:45 +00:00
}
2022-10-17 16:11:22 +00:00
}
2018-03-01 01:49:36 +00:00
auto hash = table_expression - > getTreeHash ( ) ;
String key = toString ( hash . first ) + ' _ ' + toString ( hash . second ) ;
StoragePtr & res = table_function_results [ key ] ;
if ( ! res )
{
2022-10-17 16:11:22 +00:00
TableFunctionPtr table_function_ptr ;
try
2022-09-28 08:00:12 +00:00
{
2022-10-17 16:11:22 +00:00
table_function_ptr = TableFunctionFactory : : instance ( ) . get ( table_expression , shared_from_this ( ) ) ;
2022-09-28 08:00:12 +00:00
}
2022-10-17 16:11:22 +00:00
catch ( Exception & e )
2022-03-30 12:44:53 +00:00
{
2022-11-16 11:27:08 +00:00
if ( e . code ( ) = = ErrorCodes : : UNKNOWN_FUNCTION )
{
e . addMessage ( " or incorrect parameterized view " ) ;
}
2022-10-17 16:11:22 +00:00
throw ;
}
2023-03-28 17:03:35 +00:00
2023-03-29 03:50:21 +00:00
uint64_t use_structure_from_insertion_table_in_table_functions = getSettingsRef ( ) . use_structure_from_insertion_table_in_table_functions ;
2023-03-28 17:03:35 +00:00
if ( use_structure_from_insertion_table_in_table_functions & & table_function_ptr - > needStructureHint ( ) & & hasInsertionTable ( ) )
2022-03-30 12:44:53 +00:00
{
2023-07-05 11:20:03 +00:00
const auto & insert_structure = DatabaseCatalog : : instance ( )
. getTable ( getInsertionTable ( ) , shared_from_this ( ) )
- > getInMemoryMetadataPtr ( )
- > getColumns ( )
. getInsertable ( ) ;
2023-03-24 21:58:23 +00:00
DB : : ColumnsDescription structure_hint ;
2022-10-14 15:09:35 +00:00
bool use_columns_from_insert_query = true ;
2023-03-24 00:27:04 +00:00
/// Insert table matches columns against SELECT expression by position, so we want to map
/// insert table columns to table function columns through names from SELECT expression.
auto insert_column = insert_structure . begin ( ) ;
2023-03-28 17:03:35 +00:00
auto insert_structure_end = insert_structure . end ( ) ; /// end iterator of the range covered by possible asterisk
2023-03-24 21:58:23 +00:00
auto virtual_column_names = table_function_ptr - > getVirtualsToCheckBeforeUsingStructureHint ( ) ;
bool asterisk = false ;
const auto & expression_list = select_query_hint - > select ( ) - > as < ASTExpressionList > ( ) - > children ;
2023-03-25 05:15:10 +00:00
const auto * expression = expression_list . begin ( ) ;
2023-03-24 21:58:23 +00:00
2023-03-31 21:37:18 +00:00
/// We want to go through SELECT expression list and correspond each expression to column in insert table
/// which type will be used as a hint for the file structure inference.
2023-03-24 21:58:23 +00:00
for ( ; expression ! = expression_list . end ( ) & & insert_column ! = insert_structure_end ; + + expression )
2022-03-30 12:44:53 +00:00
{
2023-03-24 21:58:23 +00:00
if ( auto * identifier = ( * expression ) - > as < ASTIdentifier > ( ) )
2022-10-14 15:09:35 +00:00
{
2023-03-24 21:58:23 +00:00
if ( ! virtual_column_names . contains ( identifier - > name ( ) ) )
2022-10-14 15:09:35 +00:00
{
2023-03-24 21:58:23 +00:00
if ( asterisk )
2023-03-28 17:03:35 +00:00
{
if ( use_structure_from_insertion_table_in_table_functions = = 1 )
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN , " Asterisk cannot be mixed with column list in INSERT SELECT query. " ) ;
use_columns_from_insert_query = false ;
break ;
}
2023-03-24 21:58:23 +00:00
structure_hint . add ( { identifier - > name ( ) , insert_column - > type } ) ;
2022-10-14 15:09:35 +00:00
}
2023-03-24 21:58:23 +00:00
2023-03-31 21:37:18 +00:00
/// Once we hit asterisk we want to find end of the range covered by asterisk
/// contributing every further SELECT expression to the tail of insert structure
2023-03-24 21:58:23 +00:00
if ( asterisk )
- - insert_structure_end ;
else
+ + insert_column ;
}
else if ( ( * expression ) - > as < ASTAsterisk > ( ) )
{
if ( asterisk )
2022-10-14 15:09:35 +00:00
{
2023-03-28 17:03:35 +00:00
if ( use_structure_from_insertion_table_in_table_functions = = 1 )
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN , " Only one asterisk can be used in INSERT SELECT query. " ) ;
use_columns_from_insert_query = false ;
break ;
2022-10-14 15:09:35 +00:00
}
2023-03-24 21:58:23 +00:00
if ( ! structure_hint . empty ( ) )
2022-10-14 15:09:35 +00:00
{
2023-03-28 17:03:35 +00:00
if ( use_structure_from_insertion_table_in_table_functions = = 1 )
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN , " Asterisk cannot be mixed with column list in INSERT SELECT query. " ) ;
2022-10-14 15:09:35 +00:00
use_columns_from_insert_query = false ;
break ;
}
2023-03-24 21:58:23 +00:00
asterisk = true ;
2022-10-14 15:09:35 +00:00
}
2023-04-13 08:32:44 +00:00
else if ( auto * func = ( * expression ) - > as < ASTFunction > ( ) )
2022-10-14 15:09:35 +00:00
{
2023-04-13 08:32:44 +00:00
if ( use_structure_from_insertion_table_in_table_functions = = 2 & & findIdentifier ( func ) )
2022-10-14 15:09:35 +00:00
{
use_columns_from_insert_query = false ;
break ;
}
2023-03-31 21:37:18 +00:00
/// Once we hit asterisk we want to find end of the range covered by asterisk
/// contributing every further SELECT expression to the tail of insert structure
2023-03-28 17:03:35 +00:00
if ( asterisk )
- - insert_structure_end ;
else
+ + insert_column ;
}
2023-03-24 00:27:04 +00:00
else
2022-10-14 15:09:35 +00:00
{
2023-03-31 21:37:18 +00:00
/// Once we hit asterisk we want to find end of the range covered by asterisk
/// contributing every further SELECT expression to the tail of insert structure
2023-03-24 21:58:23 +00:00
if ( asterisk )
- - insert_structure_end ;
2022-10-14 15:09:35 +00:00
else
2023-03-24 21:58:23 +00:00
+ + insert_column ;
2022-10-14 15:09:35 +00:00
}
2022-03-30 12:44:53 +00:00
}
2022-10-14 15:09:35 +00:00
2023-04-03 16:05:21 +00:00
if ( use_structure_from_insertion_table_in_table_functions = = 2 & & ! asterisk )
{
/// For input function we should check if input format supports reading subset of columns.
if ( table_function_ptr - > getName ( ) = = " input " )
use_columns_from_insert_query = FormatFactory : : instance ( ) . checkIfFormatSupportsSubsetOfColumns ( getInsertFormat ( ) ) ;
else
use_columns_from_insert_query = table_function_ptr - > supportsReadingSubsetOfColumns ( ) ;
}
2022-10-14 15:09:35 +00:00
if ( use_columns_from_insert_query )
2023-03-24 21:58:23 +00:00
{
2023-03-28 17:03:35 +00:00
if ( expression = = expression_list . end ( ) )
{
2023-03-31 21:37:18 +00:00
/// Append tail of insert structure to the hint
2023-03-28 17:03:35 +00:00
if ( asterisk )
{
for ( ; insert_column ! = insert_structure_end ; + + insert_column )
structure_hint . add ( { insert_column - > name , insert_column - > type } ) ;
}
if ( ! structure_hint . empty ( ) )
table_function_ptr - > setStructureHint ( structure_hint ) ;
2022-10-14 15:09:35 +00:00
2023-03-28 17:03:35 +00:00
} else if ( use_structure_from_insertion_table_in_table_functions = = 1 )
throw Exception ( ErrorCodes : : NUMBER_OF_COLUMNS_DOESNT_MATCH , " Number of columns in insert table less than required by SELECT expression. " ) ;
}
2022-03-30 12:44:53 +00:00
}
2021-04-10 23:33:54 +00:00
res = table_function_ptr - > execute ( table_expression , shared_from_this ( ) , table_function_ptr - > getName ( ) ) ;
2021-11-30 18:53:33 +00:00
/// Since ITableFunction::parseArguments() may change table_expression, i.e.:
///
/// remote('127.1', system.one) -> remote('127.1', 'system.one'),
///
auto new_hash = table_expression - > getTreeHash ( ) ;
if ( hash ! = new_hash )
{
key = toString ( new_hash . first ) + ' _ ' + toString ( new_hash . second ) ;
table_function_results [ key ] = res ;
}
2018-03-01 01:49:36 +00:00
}
return res ;
2017-08-15 12:34:28 +00:00
}
2014-03-04 15:31:56 +00:00
2023-05-22 16:36:34 +00:00
StoragePtr Context : : executeTableFunction ( const ASTPtr & table_expression , const TableFunctionPtr & table_function_ptr )
{
auto hash = table_expression - > getTreeHash ( ) ;
String key = toString ( hash . first ) + ' _ ' + toString ( hash . second ) ;
StoragePtr & res = table_function_results [ key ] ;
if ( ! res )
{
res = table_function_ptr - > execute ( table_expression , shared_from_this ( ) , table_function_ptr - > getName ( ) ) ;
}
return res ;
}
2018-03-01 01:49:36 +00:00
2018-12-09 17:50:35 +00:00
void Context : : addViewSource ( const StoragePtr & storage )
{
if ( view_source )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : TABLE_ALREADY_EXISTS , " Temporary view source storage {} already exists. " ,
backQuoteIfNeed ( view_source - > getName ( ) ) ) ;
2018-12-09 17:50:35 +00:00
view_source = storage ;
}
2021-05-31 14:49:02 +00:00
StoragePtr Context : : getViewSource ( ) const
2018-12-09 17:50:35 +00:00
{
return view_source ;
}
2023-04-28 12:42:23 +00:00
bool Context : : displaySecretsInShowAndSelect ( ) const
2023-04-27 21:27:18 +00:00
{
2023-04-28 12:42:23 +00:00
return shared - > server_settings . display_secrets_in_show_and_select ;
2023-04-27 21:27:18 +00:00
}
2012-08-02 17:33:31 +00:00
Settings Context : : getSettings ( ) const
{
2020-09-24 17:08:15 +00:00
auto lock = getLock ( ) ;
2012-08-02 17:33:31 +00:00
return settings ;
}
void Context : : setSettings ( const Settings & settings_ )
{
2020-01-12 21:00:55 +00:00
auto lock = getLock ( ) ;
2012-08-02 17:33:31 +00:00
settings = settings_ ;
2023-07-01 15:39:50 +00:00
need_recalculate_access = true ;
2023-03-16 10:39:13 +00:00
}
2012-08-02 17:33:31 +00:00
2022-07-17 15:22:12 +00:00
void Context : : setSetting ( std : : string_view name , const String & value )
2012-08-02 19:03:32 +00:00
{
2019-04-18 23:29:32 +00:00
auto lock = getLock ( ) ;
2014-02-13 07:17:22 +00:00
if ( name = = " profile " )
2018-03-02 05:44:17 +00:00
{
2021-07-22 16:07:03 +00:00
setCurrentProfile ( value ) ;
2019-04-18 23:29:32 +00:00
return ;
2018-03-02 05:44:17 +00:00
}
2022-07-17 15:22:12 +00:00
settings . set ( name , value ) ;
2023-07-01 15:39:50 +00:00
if ( ContextAccessParams : : dependsOnSettingName ( name ) )
need_recalculate_access = true ;
2014-02-13 07:17:22 +00:00
}
2022-07-17 15:22:12 +00:00
void Context : : setSetting ( std : : string_view name , const Field & value )
2014-02-13 07:17:22 +00:00
{
2019-04-18 23:29:32 +00:00
auto lock = getLock ( ) ;
2014-02-13 07:17:22 +00:00
if ( name = = " profile " )
2018-03-02 05:44:17 +00:00
{
2021-07-22 16:07:03 +00:00
setCurrentProfile ( value . safeGet < String > ( ) ) ;
2019-04-18 23:29:32 +00:00
return ;
2018-03-02 05:44:17 +00:00
}
2022-07-17 15:22:12 +00:00
settings . set ( name , value ) ;
2023-07-01 15:39:50 +00:00
if ( ContextAccessParams : : dependsOnSettingName ( name ) )
need_recalculate_access = true ;
2019-04-18 23:29:32 +00:00
}
void Context : : applySettingChange ( const SettingChange & change )
{
2020-06-14 01:07:47 +00:00
try
{
setSetting ( change . name , change . value ) ;
}
catch ( Exception & e )
{
2023-02-04 18:41:41 +00:00
e . addMessage ( fmt : : format (
" in attempt to set the value of setting '{}' to {} " ,
change . name , applyVisitor ( FieldVisitorToString ( ) , change . value ) ) ) ;
2020-06-14 01:07:47 +00:00
throw ;
}
2019-04-18 23:29:32 +00:00
}
void Context : : applySettingsChanges ( const SettingsChanges & changes )
{
auto lock = getLock ( ) ;
for ( const SettingChange & change : changes )
applySettingChange ( change ) ;
2021-03-25 06:34:28 +00:00
applySettingsQuirks ( settings ) ;
2019-04-18 23:29:32 +00:00
}
2022-12-07 00:29:48 +00:00
void Context : : checkSettingsConstraints ( const SettingsProfileElements & profile_elements ) const
{
getSettingsConstraintsAndCurrentProfiles ( ) - > constraints . check ( settings , profile_elements ) ;
}
2020-02-28 18:55:21 +00:00
void Context : : checkSettingsConstraints ( const SettingChange & change ) const
2019-04-18 23:29:32 +00:00
{
2021-07-22 16:07:03 +00:00
getSettingsConstraintsAndCurrentProfiles ( ) - > constraints . check ( settings , change ) ;
2019-04-18 23:29:32 +00:00
}
2020-02-28 18:55:21 +00:00
void Context : : checkSettingsConstraints ( const SettingsChanges & changes ) const
2019-04-18 23:29:32 +00:00
{
2021-07-22 16:07:03 +00:00
getSettingsConstraintsAndCurrentProfiles ( ) - > constraints . check ( settings , changes ) ;
2012-08-02 19:03:32 +00:00
}
2020-07-19 14:59:07 +00:00
void Context : : checkSettingsConstraints ( SettingsChanges & changes ) const
2020-02-28 18:55:21 +00:00
{
2021-07-22 16:07:03 +00:00
getSettingsConstraintsAndCurrentProfiles ( ) - > constraints . check ( settings , changes ) ;
2020-02-28 18:55:21 +00:00
}
void Context : : clampToSettingsConstraints ( SettingsChanges & changes ) const
{
2021-07-22 16:07:03 +00:00
getSettingsConstraintsAndCurrentProfiles ( ) - > constraints . clamp ( settings , changes ) ;
2020-02-28 18:55:21 +00:00
}
2022-12-02 19:30:14 +00:00
void Context : : checkMergeTreeSettingsConstraints ( const MergeTreeSettings & merge_tree_settings , const SettingsChanges & changes ) const
{
getSettingsConstraintsAndCurrentProfiles ( ) - > constraints . check ( merge_tree_settings , changes ) ;
}
2022-10-11 18:25:28 +00:00
void Context : : resetSettingsToDefaultValue ( const std : : vector < String > & names )
{
auto lock = getLock ( ) ;
for ( const String & name : names )
{
settings . setDefaultValue ( name ) ;
}
}
2021-07-22 16:07:03 +00:00
std : : shared_ptr < const SettingsConstraintsAndProfileIDs > Context : : getSettingsConstraintsAndCurrentProfiles ( ) const
2020-03-04 22:27:03 +00:00
{
2021-07-22 16:07:03 +00:00
auto lock = getLock ( ) ;
if ( settings_constraints_and_current_profiles )
return settings_constraints_and_current_profiles ;
2021-11-02 11:06:20 +00:00
static auto no_constraints_or_profiles = std : : make_shared < SettingsConstraintsAndProfileIDs > ( getAccessControl ( ) ) ;
2021-07-22 16:07:03 +00:00
return no_constraints_or_profiles ;
2020-03-04 22:27:03 +00:00
}
2012-08-02 17:33:31 +00:00
String Context : : getCurrentDatabase ( ) const
{
2020-02-03 12:54:36 +00:00
auto lock = getLock ( ) ;
2012-08-02 17:33:31 +00:00
return current_database ;
}
2019-08-31 12:18:14 +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 )
{
2021-04-10 23:33:54 +00:00
if ( ! isGlobalContext ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR ,
" Cannot set current database for non global context, this method should "
" be used during server initialization " ) ;
2020-09-25 11:27:00 +00:00
auto lock = getLock ( ) ;
if ( ! current_database . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Default database name cannot be changed in global context without server restart " ) ;
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 )
{
2020-02-10 13:10:17 +00:00
DatabaseCatalog : : instance ( ) . assertDatabaseExists ( name ) ;
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2012-08-02 17:33:31 +00:00
current_database = name ;
2023-07-01 15:39:50 +00:00
need_recalculate_access = true ;
2012-08-02 17:33:31 +00:00
}
2014-02-12 17:31:02 +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
2017-05-24 19:31:50 +00:00
{
2020-08-20 20:59:40 +00:00
char bytes [ 16 ] ;
struct
2017-05-24 19:31:50 +00:00
{
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 ;
2017-05-24 19:31:50 +00:00
2023-02-19 22:15:09 +00:00
random . words . a = thread_local_rng ( ) ;
random . words . b = thread_local_rng ( ) ;
2020-08-28 19:02:50 +00:00
2017-05-24 19:31:50 +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.
{
2017-05-24 19:31:50 +00:00
/// Use protected constructor.
2020-03-23 02:12:31 +00:00
struct QueryUUID : Poco : : UUID
2017-05-24 19:31:50 +00:00
{
2020-03-23 02:12:31 +00:00
QueryUUID ( const char * bytes , Poco : : UUID : : Version version )
2017-05-24 19:31:50 +00:00
: Poco : : UUID ( bytes , version ) { }
} ;
2020-03-23 02:12:31 +00:00
query_id_to_set = QueryUUID ( random . bytes , Poco : : UUID : : UUID_RANDOM ) . toString ( ) ;
2017-05-24 19:31:50 +00:00
}
2015-07-25 09:49:09 +00:00
2016-10-24 21:40:39 +00:00
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 ;
2014-02-12 17:31:02 +00:00
}
2022-12-02 12:00:30 +00:00
void Context : : killCurrentQuery ( ) const
2019-02-01 01:48:25 +00:00
{
2022-12-01 16:20:39 +00:00
if ( auto elem = getProcessListElement ( ) )
2022-10-17 02:21:08 +00:00
elem - > cancelQuery ( true ) ;
2022-05-16 18:59:27 +00:00
}
2014-02-12 17:31:02 +00:00
2013-06-29 18:03:57 +00:00
String Context : : getDefaultFormat ( ) const
{
return default_format . empty ( ) ? " TabSeparated " : default_format ;
}
void Context : : setDefaultFormat ( const String & name )
{
default_format = name ;
}
2022-10-14 15:09:35 +00:00
String Context : : getInsertFormat ( ) const
{
return insert_format ;
}
void Context : : setInsertFormat ( const String & name )
{
insert_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
2018-03-02 05:44:17 +00:00
{
2021-04-10 23:33:54 +00:00
auto ptr = query_context . lock ( ) ;
2023-01-23 21:13:58 +00:00
if ( ! ptr ) throw Exception ( ErrorCodes : : THERE_IS_NO_QUERY , " There is no query or query context has expired " ) ;
2021-04-10 23:33:54 +00:00
return ptr ;
2018-03-02 05:44:17 +00:00
}
2021-04-10 23:33:54 +00:00
bool Context : : isInternalSubquery ( ) const
2018-03-02 05:44:17 +00:00
{
2021-04-10 23:33:54 +00:00
auto ptr = query_context . lock ( ) ;
return ptr & & ptr . get ( ) ! = this ;
2018-03-02 05:44:17 +00:00
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context : : getSessionContext ( ) const
2016-11-16 11:29:51 +00:00
{
2021-04-10 23:33:54 +00:00
auto ptr = session_context . lock ( ) ;
2023-01-23 21:13:58 +00:00
if ( ! ptr ) throw Exception ( ErrorCodes : : THERE_IS_NO_SESSION , " There is no session or session context has expired " ) ;
2021-04-10 23:33:54 +00:00
return ptr ;
2016-11-16 11:29:51 +00:00
}
2013-06-29 18:03:57 +00:00
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context : : getGlobalContext ( ) const
2012-08-02 17:33:31 +00:00
{
2021-04-10 23:33:54 +00:00
auto ptr = global_context . lock ( ) ;
2023-01-23 21:13:58 +00:00
if ( ! ptr ) throw Exception ( ErrorCodes : : LOGICAL_ERROR , " There is no global context or global context has expired " ) ;
2021-04-10 23:33:54 +00:00
return ptr ;
2012-08-02 17:33:31 +00:00
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context : : getBufferContext ( ) const
2016-11-15 10:57:11 +00:00
{
2023-01-23 21:13:58 +00:00
if ( ! buffer_context ) throw Exception ( ErrorCodes : : LOGICAL_ERROR , " There is no buffer context " ) ;
2021-04-10 23:33:54 +00:00
return buffer_context ;
2021-01-27 18:05:18 +00:00
}
2023-03-30 16:50:12 +00:00
void Context : : makeQueryContext ( )
{
query_context = shared_from_this ( ) ;
2023-03-30 17:00:34 +00:00
2023-05-11 14:59:19 +00:00
/// Throttling should not be inherited, otherwise if you will set
/// throttling for default profile you will not able to overwrite it
/// per-user/query.
///
/// Note, that if you need to set it server-wide, you should use
/// per-server settings, i.e.:
/// - max_backup_bandwidth_for_server
/// - max_remote_read_network_bandwidth_for_server
/// - max_remote_write_network_bandwidth_for_server
/// - max_local_read_bandwidth_for_server
/// - max_local_write_bandwidth_for_server
remote_read_query_throttler . reset ( ) ;
remote_write_query_throttler . reset ( ) ;
local_read_query_throttler . reset ( ) ;
local_write_query_throttler . reset ( ) ;
backups_query_throttler . reset ( ) ;
2023-03-30 16:50:12 +00:00
}
void Context : : makeSessionContext ( )
{
session_context = shared_from_this ( ) ;
}
void Context : : makeGlobalContext ( )
{
initGlobal ( ) ;
global_context = shared_from_this ( ) ;
}
2012-12-19 20:15:15 +00:00
2017-01-21 04:24:28 +00:00
const EmbeddedDictionaries & Context : : getEmbeddedDictionaries ( ) const
2012-12-19 20:15:15 +00:00
{
2017-01-21 04:24:28 +00:00
return getEmbeddedDictionariesImpl ( false ) ;
2015-02-10 17:40:40 +00:00
}
2017-08-24 18:19:06 +00:00
EmbeddedDictionaries & Context : : getEmbeddedDictionaries ( )
{
return getEmbeddedDictionariesImpl ( false ) ;
}
2015-02-10 17:40:40 +00:00
2019-09-26 10:41:33 +00:00
const ExternalDictionariesLoader & Context : : getExternalDictionariesLoader ( ) const
2015-03-27 13:11:22 +00:00
{
2021-04-10 23:33:54 +00:00
return const_cast < Context * > ( this ) - > getExternalDictionariesLoader ( ) ;
2015-04-02 16:30:18 +00:00
}
2019-09-26 10:41:33 +00:00
ExternalDictionariesLoader & Context : : getExternalDictionariesLoader ( )
2019-06-02 12:11:01 +00:00
{
2021-04-10 23:33:54 +00:00
std : : lock_guard lock ( shared - > external_dictionaries_mutex ) ;
2021-09-29 12:52:58 +00:00
return getExternalDictionariesLoaderUnlocked ( ) ;
}
ExternalDictionariesLoader & Context : : getExternalDictionariesLoaderUnlocked ( )
{
2021-04-10 23:33:54 +00:00
if ( ! shared - > external_dictionaries_loader )
2022-03-06 10:23:50 +00:00
shared - > external_dictionaries_loader =
std : : make_unique < ExternalDictionariesLoader > ( getGlobalContext ( ) ) ;
2021-04-10 23:33:54 +00:00
return * shared - > external_dictionaries_loader ;
2019-06-02 12:11:01 +00:00
}
2021-09-07 23:55:17 +00:00
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 ) ;
2021-09-29 12:52:58 +00:00
return getExternalUserDefinedExecutableFunctionsLoaderUnlocked ( ) ;
}
ExternalUserDefinedExecutableFunctionsLoader & Context : : getExternalUserDefinedExecutableFunctionsLoaderUnlocked ( )
{
2021-09-07 23:55:17 +00:00
if ( ! shared - > external_user_defined_executable_functions_loader )
2022-03-06 10:23:50 +00:00
shared - > external_user_defined_executable_functions_loader =
std : : make_unique < ExternalUserDefinedExecutableFunctionsLoader > ( getGlobalContext ( ) ) ;
2021-09-07 23:55:17 +00:00
return * shared - > external_user_defined_executable_functions_loader ;
}
2019-06-02 12:11:01 +00:00
EmbeddedDictionaries & Context : : getEmbeddedDictionariesImpl ( const bool throw_on_error ) const
2015-04-02 16:30:18 +00:00
{
2019-06-02 12:11:01 +00:00
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 > ( ) ;
2019-06-02 12:11:01 +00:00
2022-03-06 10:23:50 +00:00
shared - > embedded_dictionaries = std : : make_unique < EmbeddedDictionaries > (
2019-06-02 12:11:01 +00:00
std : : move ( geo_dictionaries_loader ) ,
2021-04-10 23:33:54 +00:00
getGlobalContext ( ) ,
2019-06-02 12:11:01 +00:00
throw_on_error ) ;
}
return * shared - > embedded_dictionaries ;
2012-12-19 20:15:15 +00:00
}
2013-02-16 14:55:14 +00:00
2021-09-29 12:52:58 +00:00
void Context : : tryCreateEmbeddedDictionaries ( const Poco : : Util : : AbstractConfiguration & config ) const
2017-10-17 10:44:46 +00:00
{
2021-09-29 12:52:58 +00:00
if ( ! config . getBool ( " dictionaries_lazy_load " , true ) )
static_cast < void > ( getEmbeddedDictionariesImpl ( true ) ) ;
2017-10-17 10:44:46 +00:00
}
2021-09-29 12:52:58 +00:00
void Context : : loadOrReloadDictionaries ( const Poco : : Util : : AbstractConfiguration & config )
2021-05-12 18:16:01 +00:00
{
2021-09-29 12:52:58 +00:00
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 )
2021-05-12 18:16:01 +00:00
{
2021-09-29 12:52:58 +00:00
shared - > external_dictionaries_config_repository - > updatePatterns ( patterns ) ;
external_dictionaries_loader . reloadConfig ( shared - > external_dictionaries_config_repository - > getName ( ) ) ;
return ;
2021-05-12 18:16:01 +00:00
}
2021-09-29 12:52:58 +00:00
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 ) ) ;
2021-05-12 18:16:01 +00:00
}
2017-10-17 10:44:46 +00:00
2021-09-29 12:52:58 +00:00
void Context : : loadOrReloadUserDefinedExecutableFunctions ( const Poco : : Util : : AbstractConfiguration & config )
2021-09-07 23:55:17 +00:00
{
2021-09-29 12:52:58 +00:00
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 ) ) ;
2021-09-07 23:55:17 +00:00
}
2022-09-24 21:24:39 +00:00
const IUserDefinedSQLObjectsLoader & Context : : getUserDefinedSQLObjectsLoader ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > user_defined_sql_objects_loader )
shared - > user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader ( getGlobalContext ( ) ) ;
return * shared - > user_defined_sql_objects_loader ;
}
IUserDefinedSQLObjectsLoader & Context : : getUserDefinedSQLObjectsLoader ( )
{
auto lock = getLock ( ) ;
if ( ! shared - > user_defined_sql_objects_loader )
shared - > user_defined_sql_objects_loader = createUserDefinedSQLObjectsLoader ( getGlobalContext ( ) ) ;
return * shared - > user_defined_sql_objects_loader ;
}
2021-07-30 13:30:30 +00:00
# if USE_NLP
2021-06-04 00:33:41 +00:00
SynonymsExtensions & Context : : getSynonymsExtensions ( ) const
2021-06-03 02:20:42 +00:00
{
auto lock = getLock ( ) ;
2021-06-19 18:52:09 +00:00
if ( ! shared - > synonyms_extensions )
2021-06-03 02:20:42 +00:00
shared - > synonyms_extensions . emplace ( getConfigRef ( ) ) ;
2021-06-19 18:52:09 +00:00
return * shared - > synonyms_extensions ;
2021-06-03 02:20:42 +00:00
}
2021-06-05 00:52:35 +00:00
Lemmatizers & Context : : getLemmatizers ( ) const
{
auto lock = getLock ( ) ;
2021-06-19 18:52:09 +00:00
if ( ! shared - > lemmatizers )
2021-06-05 00:52:35 +00:00
shared - > lemmatizers . emplace ( getConfigRef ( ) ) ;
2021-06-19 18:52:09 +00:00
return * shared - > lemmatizers ;
2021-06-05 00:52:35 +00:00
}
2021-07-30 13:30:30 +00:00
# endif
2021-06-05 00:52:35 +00:00
2022-05-02 17:09:51 +00:00
BackupsWorker & Context : : getBackupsWorker ( ) const
{
auto lock = getLock ( ) ;
2023-01-23 07:59:44 +00:00
const bool allow_concurrent_backups = this - > getConfigRef ( ) . getBool ( " backups.allow_concurrent_backups " , true ) ;
const bool allow_concurrent_restores = this - > getConfigRef ( ) . getBool ( " backups.allow_concurrent_restores " , true ) ;
2023-01-09 17:14:39 +00:00
2023-03-23 14:44:15 +00:00
const auto & config = getConfigRef ( ) ;
2023-06-24 03:06:20 +00:00
const auto & settings_ref = getSettingsRef ( ) ;
UInt64 backup_threads = config . getUInt64 ( " backup_threads " , settings_ref . backup_threads ) ;
UInt64 restore_threads = config . getUInt64 ( " restore_threads " , settings_ref . restore_threads ) ;
2023-03-23 14:44:15 +00:00
2022-05-02 17:09:51 +00:00
if ( ! shared - > backups_worker )
2023-03-23 14:44:15 +00:00
shared - > backups_worker . emplace ( backup_threads , restore_threads , allow_concurrent_backups , allow_concurrent_restores ) ;
2022-05-02 17:09:51 +00:00
return * shared - > backups_worker ;
}
2013-02-16 14:55:14 +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.
2013-02-16 14:55:14 +00:00
progress_callback = callback ;
}
ProgressCallback Context : : getProgressCallback ( ) const
{
return progress_callback ;
}
2013-09-03 20:21:28 +00:00
2022-10-17 02:21:08 +00:00
void Context : : setProcessListElement ( QueryStatusPtr elem )
2013-11-03 05:32:42 +00:00
{
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.
2013-11-03 05:32:42 +00:00
process_list_elem = elem ;
2022-12-01 16:20:39 +00:00
has_process_list_elem = elem . get ( ) ;
2013-11-03 05:32:42 +00:00
}
2022-10-17 02:21:08 +00:00
QueryStatusPtr Context : : getProcessListElement ( ) const
2017-08-29 13:23:04 +00:00
{
2022-12-01 16:20:39 +00:00
if ( ! has_process_list_elem )
return { } ;
if ( auto res = process_list_elem . lock ( ) )
return res ;
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Weak pointer to process_list_elem expired during query execution, it's a bug " ) ;
2017-08-29 13:23:04 +00:00
}
2013-11-03 05:32:42 +00:00
2023-03-13 09:06:58 +00:00
void Context : : setUncompressedCache ( const String & uncompressed_cache_policy , size_t max_size_in_bytes )
2013-09-08 05:53:10 +00:00
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2013-09-08 05:53:10 +00:00
if ( shared - > uncompressed_cache )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Uncompressed cache has been already created. " ) ;
2013-09-08 05:53:10 +00:00
2023-03-13 09:06:58 +00:00
shared - > uncompressed_cache = std : : make_shared < UncompressedCache > ( uncompressed_cache_policy , max_size_in_bytes ) ;
2013-09-08 05:53:10 +00:00
}
UncompressedCachePtr Context : : getUncompressedCache ( ) const
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2013-09-08 05:53:10 +00:00
return shared - > uncompressed_cache ;
}
2017-08-07 17:01:04 +00:00
void Context : : dropUncompressedCache ( ) const
{
auto lock = getLock ( ) ;
if ( shared - > uncompressed_cache )
shared - > uncompressed_cache - > reset ( ) ;
}
2023-03-13 09:06:58 +00:00
void Context : : setMarkCache ( const String & mark_cache_policy , size_t cache_size_in_bytes )
2014-02-11 13:30:42 +00:00
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2014-02-11 13:30:42 +00:00
if ( shared - > mark_cache )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Mark cache has been already created. " ) ;
2014-02-11 13:30:42 +00:00
2023-03-13 09:06:58 +00:00
shared - > mark_cache = std : : make_shared < MarkCache > ( mark_cache_policy , cache_size_in_bytes ) ;
2014-02-11 13:30:42 +00:00
}
MarkCachePtr Context : : getMarkCache ( ) const
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2014-02-11 13:30:42 +00:00
return shared - > mark_cache ;
}
2017-08-07 17:01:04 +00:00
void Context : : dropMarkCache ( ) const
{
auto lock = getLock ( ) ;
if ( shared - > mark_cache )
shared - > mark_cache - > reset ( ) ;
}
2022-09-05 18:12:40 +00:00
ThreadPool & Context : : getLoadMarksThreadpool ( ) const
{
2023-01-19 15:55:15 +00:00
const auto & config = getConfigRef ( ) ;
2022-09-05 18:12:40 +00:00
auto lock = getLock ( ) ;
if ( ! shared - > load_marks_threadpool )
{
2023-01-19 15:55:15 +00:00
auto pool_size = config . getUInt ( " .load_marks_threadpool_pool_size " , 50 ) ;
auto queue_size = config . getUInt ( " .load_marks_threadpool_queue_size " , 1000000 ) ;
2023-03-30 18:08:38 +00:00
shared - > load_marks_threadpool = std : : make_unique < ThreadPool > (
CurrentMetrics : : MarksLoaderThreads , CurrentMetrics : : MarksLoaderThreadsActive , pool_size , pool_size , queue_size ) ;
2022-09-05 18:12:40 +00:00
}
return * shared - > load_marks_threadpool ;
}
2017-08-07 17:01:04 +00:00
2023-02-07 17:50:31 +00:00
static size_t getPrefetchThreadpoolSizeFromConfig ( const Poco : : Util : : AbstractConfiguration & config )
{
return config . getUInt ( " .prefetch_threadpool_pool_size " , 100 ) ;
}
size_t Context : : getPrefetchThreadpoolSize ( ) const
{
const auto & config = getConfigRef ( ) ;
return getPrefetchThreadpoolSizeFromConfig ( config ) ;
}
ThreadPool & Context : : getPrefetchThreadpool ( ) const
{
const auto & config = getConfigRef ( ) ;
auto lock = getLock ( ) ;
if ( ! shared - > prefetch_threadpool )
{
auto pool_size = getPrefetchThreadpoolSize ( ) ;
auto queue_size = config . getUInt ( " .prefetch_threadpool_queue_size " , 1000000 ) ;
2023-03-30 18:08:38 +00:00
shared - > prefetch_threadpool = std : : make_unique < ThreadPool > (
CurrentMetrics : : IOPrefetchThreads , CurrentMetrics : : IOPrefetchThreadsActive , pool_size , pool_size , queue_size ) ;
2023-02-07 17:50:31 +00:00
}
return * shared - > prefetch_threadpool ;
}
2021-08-21 17:27:22 +00:00
void Context : : setIndexUncompressedCache ( size_t max_size_in_bytes )
{
auto lock = getLock ( ) ;
if ( shared - > index_uncompressed_cache )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Index uncompressed cache has been already created. " ) ;
2021-08-21 17:27:22 +00:00
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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Index mark cache has been already created. " ) ;
2021-08-21 17:27:22 +00:00
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 ( ) ;
}
2023-02-02 19:16:25 +00:00
void Context : : setQueryCache ( const Poco : : Util : : AbstractConfiguration & config )
2022-11-29 13:15:28 +00:00
{
auto lock = getLock ( ) ;
2023-01-26 22:23:10 +00:00
if ( shared - > query_cache )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Query cache has been already created. " ) ;
2022-11-29 13:15:28 +00:00
2023-01-29 20:43:52 +00:00
shared - > query_cache = std : : make_shared < QueryCache > ( ) ;
2023-02-02 19:16:25 +00:00
shared - > query_cache - > updateConfiguration ( config ) ;
2022-11-29 13:15:28 +00:00
}
2023-02-02 19:16:25 +00:00
void Context : : updateQueryCacheConfiguration ( const Poco : : Util : : AbstractConfiguration & config )
2022-11-29 13:15:28 +00:00
{
auto lock = getLock ( ) ;
2023-02-02 19:16:25 +00:00
if ( shared - > query_cache )
shared - > query_cache - > updateConfiguration ( config ) ;
2022-11-29 13:15:28 +00:00
}
2023-02-02 19:16:25 +00:00
QueryCachePtr Context : : getQueryCache ( ) const
2022-11-29 13:15:28 +00:00
{
auto lock = getLock ( ) ;
2023-02-02 19:16:25 +00:00
return shared - > query_cache ;
2022-11-29 13:15:28 +00:00
}
2021-08-21 17:27:22 +00:00
2023-02-02 19:16:25 +00:00
void Context : : dropQueryCache ( ) const
2023-01-29 20:43:52 +00:00
{
auto lock = getLock ( ) ;
if ( shared - > query_cache )
2023-02-02 19:16:25 +00:00
shared - > query_cache - > reset ( ) ;
2022-11-29 13:15:28 +00:00
}
2021-08-21 17:27:22 +00:00
2021-03-28 19:24:28 +00:00
void Context : : setMMappedFileCache ( size_t cache_size_in_num_entries )
2021-03-28 01:10:30 +00:00
{
auto lock = getLock ( ) ;
if ( shared - > mmap_cache )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Mapped file cache has been already created. " ) ;
2021-03-28 01:10:30 +00:00
2021-03-28 19:24:28 +00:00
shared - > mmap_cache = std : : make_shared < MMappedFileCache > ( cache_size_in_num_entries ) ;
2021-03-28 01:10:30 +00:00
}
2021-03-28 19:24:28 +00:00
MMappedFileCachePtr Context : : getMMappedFileCache ( ) const
2021-03-28 01:10:30 +00:00
{
auto lock = getLock ( ) ;
return shared - > mmap_cache ;
}
2021-03-28 19:24:28 +00:00
void Context : : dropMMappedFileCache ( ) const
2021-03-28 01:10:30 +00:00
{
auto lock = getLock ( ) ;
if ( shared - > mmap_cache )
shared - > mmap_cache - > reset ( ) ;
}
2017-08-07 17:01:04 +00:00
void Context : : dropCaches ( ) const
{
auto lock = getLock ( ) ;
if ( shared - > uncompressed_cache )
shared - > uncompressed_cache - > reset ( ) ;
if ( shared - > mark_cache )
shared - > mark_cache - > reset ( ) ;
2021-03-28 01:10:30 +00:00
2021-08-21 17:27:22 +00:00
if ( shared - > index_uncompressed_cache )
shared - > index_uncompressed_cache - > reset ( ) ;
if ( shared - > index_mark_cache )
shared - > index_mark_cache - > reset ( ) ;
2023-01-26 22:23:10 +00:00
if ( shared - > query_cache )
shared - > query_cache - > reset ( ) ;
2022-11-29 13:15:28 +00:00
2021-03-28 01:10:30 +00:00
if ( shared - > mmap_cache )
shared - > mmap_cache - > reset ( ) ;
2017-08-07 17:01:04 +00:00
}
2020-11-06 14:07:56 +00:00
BackgroundSchedulePool & Context : : getBufferFlushSchedulePool ( ) const
2020-04-16 07:48:49 +00:00
{
auto lock = getLock ( ) ;
if ( ! shared - > buffer_flush_schedule_pool )
2022-04-20 13:35:13 +00:00
{
2022-03-06 10:23:50 +00:00
shared - > buffer_flush_schedule_pool = std : : make_unique < BackgroundSchedulePool > (
2023-03-30 18:14:17 +00:00
shared - > server_settings . background_buffer_flush_schedule_pool_size ,
2020-04-22 23:00:24 +00:00
CurrentMetrics : : BackgroundBufferFlushSchedulePoolTask ,
2023-03-16 17:31:41 +00:00
CurrentMetrics : : BackgroundBufferFlushSchedulePoolSize ,
2020-04-23 00:14:58 +00:00
" BgBufSchPool " ) ;
2022-04-20 13:35:13 +00:00
}
2020-04-16 07:48:49 +00:00
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
2017-12-29 22:32:04 +00:00
{
auto lock = getLock ( ) ;
if ( ! shared - > schedule_pool )
2022-04-20 13:35:13 +00:00
{
2022-03-06 10:23:50 +00:00
shared - > schedule_pool = std : : make_unique < BackgroundSchedulePool > (
2023-03-30 18:14:17 +00:00
shared - > server_settings . background_schedule_pool_size ,
2020-04-22 23:00:24 +00:00
CurrentMetrics : : BackgroundSchedulePoolTask ,
2023-03-16 17:31:41 +00:00
CurrentMetrics : : BackgroundSchedulePoolSize ,
2020-04-23 00:14:58 +00:00
" BgSchPool " ) ;
2022-04-20 13:35:13 +00:00
}
2017-12-29 22:32:04 +00:00
return * shared - > schedule_pool ;
}
2020-11-06 14:07:56 +00:00
BackgroundSchedulePool & Context : : getDistributedSchedulePool ( ) const
2020-04-14 18:12:08 +00:00
{
auto lock = getLock ( ) ;
if ( ! shared - > distributed_schedule_pool )
2022-04-20 13:35:13 +00:00
{
2022-03-06 10:23:50 +00:00
shared - > distributed_schedule_pool = std : : make_unique < BackgroundSchedulePool > (
2023-03-30 18:14:17 +00:00
shared - > server_settings . background_distributed_schedule_pool_size ,
2020-04-22 23:00:24 +00:00
CurrentMetrics : : BackgroundDistributedSchedulePoolTask ,
2023-03-16 17:31:41 +00:00
CurrentMetrics : : BackgroundDistributedSchedulePoolSize ,
2020-04-23 00:14:58 +00:00
" BgDistSchPool " ) ;
2022-04-20 13:35:13 +00:00
}
2020-04-14 18:12:08 +00:00
return * shared - > distributed_schedule_pool ;
}
2021-01-27 18:08:06 +00:00
BackgroundSchedulePool & Context : : getMessageBrokerSchedulePool ( ) const
2020-05-13 17:00:47 +00:00
{
auto lock = getLock ( ) ;
2021-01-27 18:08:06 +00:00
if ( ! shared - > message_broker_schedule_pool )
2022-04-20 13:35:13 +00:00
{
2022-03-06 10:23:50 +00:00
shared - > message_broker_schedule_pool = std : : make_unique < BackgroundSchedulePool > (
2023-03-30 18:14:17 +00:00
shared - > server_settings . background_message_broker_schedule_pool_size ,
2021-08-09 07:56:59 +00:00
CurrentMetrics : : BackgroundMessageBrokerSchedulePoolTask ,
2023-03-16 17:31:41 +00:00
CurrentMetrics : : BackgroundMessageBrokerSchedulePoolSize ,
2021-03-25 13:57:47 +00:00
" BgMBSchPool " ) ;
2022-04-20 13:35:13 +00:00
}
2021-01-27 18:08:06 +00:00
return * shared - > message_broker_schedule_pool ;
2020-05-13 17:00:47 +00:00
}
2021-05-26 20:37:44 +00:00
ThrottlerPtr Context : : getReplicatedFetchesThrottler ( ) const
{
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
{
2021-05-27 12:54:47 +00:00
return shared - > replicated_sends_throttler ;
2021-05-26 20:37:44 +00:00
}
2022-07-11 12:59:39 +00:00
ThrottlerPtr Context : : getRemoteReadThrottler ( ) const
{
2023-05-11 14:59:19 +00:00
ThrottlerPtr throttler = shared - > remote_read_throttler ;
if ( auto bandwidth = getSettingsRef ( ) . max_remote_read_network_bandwidth )
2023-03-30 17:00:34 +00:00
{
auto lock = getLock ( ) ;
if ( ! remote_read_query_throttler )
2023-05-11 14:59:19 +00:00
remote_read_query_throttler = std : : make_shared < Throttler > ( bandwidth , throttler ) ;
2023-03-30 17:00:34 +00:00
throttler = remote_read_query_throttler ;
}
2023-03-29 14:18:56 +00:00
return throttler ;
2022-07-11 12:59:39 +00:00
}
ThrottlerPtr Context : : getRemoteWriteThrottler ( ) const
{
2023-05-11 14:59:19 +00:00
ThrottlerPtr throttler = shared - > remote_write_throttler ;
if ( auto bandwidth = getSettingsRef ( ) . max_remote_write_network_bandwidth )
2023-03-30 17:00:34 +00:00
{
auto lock = getLock ( ) ;
if ( ! remote_write_query_throttler )
2023-05-11 14:59:19 +00:00
remote_write_query_throttler = std : : make_shared < Throttler > ( bandwidth , throttler ) ;
2023-03-30 17:00:34 +00:00
throttler = remote_write_query_throttler ;
}
2023-03-29 14:18:56 +00:00
return throttler ;
2022-07-11 12:59:39 +00:00
}
2023-03-30 17:02:28 +00:00
ThrottlerPtr Context : : getLocalReadThrottler ( ) const
{
2023-05-11 14:59:19 +00:00
ThrottlerPtr throttler = shared - > local_read_throttler ;
if ( auto bandwidth = getSettingsRef ( ) . max_local_read_bandwidth )
2023-03-30 17:02:28 +00:00
{
auto lock = getLock ( ) ;
if ( ! local_read_query_throttler )
2023-05-11 14:59:19 +00:00
local_read_query_throttler = std : : make_shared < Throttler > ( bandwidth , throttler ) ;
2023-03-30 17:02:28 +00:00
throttler = local_read_query_throttler ;
}
return throttler ;
}
ThrottlerPtr Context : : getLocalWriteThrottler ( ) const
{
2023-05-11 14:59:19 +00:00
ThrottlerPtr throttler = shared - > local_write_throttler ;
if ( auto bandwidth = getSettingsRef ( ) . max_local_write_bandwidth )
2023-03-30 17:02:28 +00:00
{
auto lock = getLock ( ) ;
if ( ! local_write_query_throttler )
2023-05-11 14:59:19 +00:00
local_write_query_throttler = std : : make_shared < Throttler > ( bandwidth , throttler ) ;
2023-03-30 17:02:28 +00:00
throttler = local_write_query_throttler ;
}
return throttler ;
}
2023-03-30 17:12:00 +00:00
ThrottlerPtr Context : : getBackupsThrottler ( ) const
2023-03-30 17:06:49 +00:00
{
2023-05-11 14:59:19 +00:00
ThrottlerPtr throttler = shared - > backups_server_throttler ;
if ( auto bandwidth = getSettingsRef ( ) . max_backup_bandwidth )
2023-03-30 17:06:49 +00:00
{
auto lock = getLock ( ) ;
2023-03-30 17:12:00 +00:00
if ( ! backups_query_throttler )
2023-05-11 14:59:19 +00:00
backups_query_throttler = std : : make_shared < Throttler > ( bandwidth , throttler ) ;
2023-03-30 17:12:00 +00:00
throttler = backups_query_throttler ;
2023-03-30 17:06:49 +00:00
}
return throttler ;
2022-07-11 12:59:39 +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 " ) ;
}
2019-02-21 16:41:27 +00:00
void Context : : setDDLWorker ( std : : unique_ptr < DDLWorker > ddl_worker )
2017-04-13 13:42:29 +00:00
{
auto lock = getLock ( ) ;
if ( shared - > ddl_worker )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " DDL background thread has already been initialized " ) ;
2020-12-01 17:20:42 +00:00
ddl_worker - > startup ( ) ;
2019-02-21 16:41:27 +00:00
shared - > ddl_worker = std : : move ( ddl_worker ) ;
2017-04-13 13:42:29 +00:00
}
2017-09-04 17:49:39 +00:00
DDLWorker & Context : : getDDLWorker ( ) const
2017-04-13 13:42:29 +00:00
{
auto lock = getLock ( ) ;
if ( ! shared - > ddl_worker )
2020-11-06 18:28:59 +00:00
{
if ( ! hasZooKeeper ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG , " There is no Zookeeper configuration in server config " ) ;
2020-11-06 18:28:59 +00:00
if ( ! hasDistributedDDL ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG , " There is no DistributedDDL configuration in server config " ) ;
2020-11-06 18:28:59 +00:00
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG , " DDL background thread is not initialized " ) ;
2020-11-06 18:28:59 +00:00
}
2017-04-13 13:42:29 +00:00
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 ( ) ;
2018-04-03 19:43:33 +00:00
if ( ! shared - > zookeeper )
2023-03-23 08:58:56 +00:00
shared - > zookeeper = std : : make_shared < zkutil : : ZooKeeper > ( config , zkutil : : getZooKeeperConfigName ( config ) , getZooKeeperLog ( ) ) ;
2018-04-03 19:43:33 +00:00
else if ( shared - > zookeeper - > expired ( ) )
2022-10-25 11:26:29 +00:00
{
Stopwatch watch ;
LOG_DEBUG ( shared - > log , " Trying to establish a new connection with ZooKeeper " ) ;
2014-04-25 13:55:15 +00:00
shared - > zookeeper = shared - > zookeeper - > startNewSession ( ) ;
2022-10-25 11:26:29 +00:00
LOG_DEBUG ( shared - > log , " Establishing a new connection with ZooKeeper took {} ms " , watch . elapsedMilliseconds ( ) ) ;
}
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-14 10:21:41 +00:00
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
2021-10-14 10:21:41 +00:00
{
try
{
2023-03-23 08:58:56 +00:00
const auto config_name = zkutil : : getZooKeeperConfigName ( getConfigRef ( ) ) ;
2021-10-18 09:13:24 +00:00
/// If our server is part of main Keeper cluster
2023-03-23 08:58:56 +00:00
if ( config_name = = " keeper_server " | | checkZooKeeperConfigIsLocal ( getConfigRef ( ) , config_name ) )
2021-10-18 09:13:24 +00:00
{
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
2021-10-14 10:21:41 +00:00
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 ( ) ;
}
2021-07-29 14:48:49 +00:00
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
{
2021-10-14 10:21:41 +00:00
bool is_standalone_app = getApplicationType ( ) = = ApplicationType : : KEEPER ;
2021-10-18 09:13:24 +00:00
if ( start_async )
{
assert ( ! is_standalone_app ) ;
2021-10-22 08:24:18 +00:00
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 " ) ;
2021-10-18 09:13:24 +00:00
}
else
{
2021-10-22 08:24:18 +00:00
LOG_INFO ( shared - > log , " Cannot connect to ZooKeeper (or Keeper) before internal Keeper start, "
2021-10-18 09:13:24 +00:00
" will wait for Keeper synchronously " ) ;
}
2021-10-14 10:21:41 +00:00
2021-10-18 15:27:51 +00:00
shared - > keeper_dispatcher = std : : make_shared < KeeperDispatcher > ( ) ;
2022-12-23 14:09:24 +00:00
shared - > keeper_dispatcher - > initialize ( config , is_standalone_app , start_async , getMacros ( ) ) ;
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
2021-08-24 12:30:31 +00:00
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
}
2022-07-04 02:26:19 +00:00
std : : shared_ptr < KeeperDispatcher > & Context : : tryGetKeeperDispatcher ( ) 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 ) ;
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
2021-08-24 12:30:31 +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-29 11:51:34 +00:00
void Context : : updateKeeperConfiguration ( [[maybe_unused]] const Poco : : Util : : AbstractConfiguration & config )
2021-10-18 15:27:51 +00:00
{
# if USE_NURAFT
std : : lock_guard lock ( shared - > keeper_dispatcher_mutex ) ;
if ( ! shared - > keeper_dispatcher )
return ;
2022-12-23 14:09:24 +00:00
shared - > keeper_dispatcher - > updateConfiguration ( config , getMacros ( ) ) ;
2021-10-18 15:27:51 +00:00
# endif
}
2020-08-27 14:19:18 +00:00
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
{
2021-10-28 16:19:41 +00:00
if ( name . find ( ' : ' ) ! = std : : string : : npos | | name . find ( ' / ' ) ! = std : : string : : npos )
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " Invalid auxiliary ZooKeeper name {}: ':' and '/' are not allowed " , name ) ;
2020-11-03 07:52:31 +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
}
2020-08-27 14:19:18 +00:00
else if ( zookeeper - > second - > expired ( ) )
zookeeper - > second = zookeeper - > second - > startNewSession ( ) ;
return zookeeper - > second ;
}
2023-05-19 14:06:43 +00:00
std : : map < String , zkutil : : ZooKeeperPtr > Context : : getAuxiliaryZooKeepers ( ) const
{
std : : lock_guard lock ( shared - > auxiliary_zookeepers_mutex ) ;
2023-05-25 09:50:14 +00:00
return shared - > auxiliary_zookeepers ;
2023-05-19 14:06:43 +00:00
}
2021-12-28 03:57:43 +00:00
# if USE_ROCKSDB
2021-12-28 10:06:13 +00:00
MergeTreeMetadataCachePtr Context : : getMergeTreeMetadataCache ( ) const
2021-12-08 02:40:59 +00:00
{
2022-01-11 03:09:52 +00:00
auto cache = tryGetMergeTreeMetadataCache ( ) ;
if ( ! cache )
2022-01-10 11:52:54 +00:00
throw Exception (
ErrorCodes : : LOGICAL_ERROR ,
" Merge tree metadata cache is not initialized, please add config merge_tree_metadata_cache in config.xml and restart " ) ;
2022-01-11 03:09:52 +00:00
return cache ;
}
MergeTreeMetadataCachePtr Context : : tryGetMergeTreeMetadataCache ( ) const
{
2021-12-28 10:06:13 +00:00
return shared - > merge_tree_metadata_cache ;
2021-12-08 02:40:59 +00:00
}
2021-12-28 03:57:43 +00:00
# endif
2021-12-08 02:40:59 +00:00
2019-07-04 14:48:40 +00:00
void Context : : resetZooKeeper ( ) const
{
std : : lock_guard lock ( shared - > zookeeper_mutex ) ;
shared - > zookeeper . reset ( ) ;
}
2023-03-27 13:06:46 +00:00
static void reloadZooKeeperIfChangedImpl (
const ConfigurationPtr & config ,
const std : : string & config_name ,
zkutil : : ZooKeeperPtr & zk ,
std : : shared_ptr < ZooKeeperLog > zk_log ,
bool server_started )
2020-11-03 07:52:31 +00:00
{
if ( ! zk | | zk - > configChanged ( * config , config_name ) )
2021-02-26 19:05:25 +00:00
{
if ( zk )
2021-10-13 10:21:07 +00:00
zk - > finalize ( " Config changed " ) ;
2021-02-26 19:05:25 +00:00
2021-07-09 14:05:35 +00:00
zk = std : : make_shared < zkutil : : ZooKeeper > ( * config , config_name , std : : move ( zk_log ) ) ;
2023-03-27 13:06:46 +00:00
if ( server_started )
zk - > setServerCompletelyStarted ( ) ;
2021-02-26 19:05:25 +00:00
}
2020-11-03 07:52:31 +00:00
}
2020-09-10 04:00:33 +00:00
void Context : : reloadZooKeeperIfChanged ( const ConfigurationPtr & config ) const
{
2023-03-27 13:06:46 +00:00
bool server_started = isServerCompletelyStarted ( ) ;
2020-09-10 04:00:33 +00:00
std : : lock_guard lock ( shared - > zookeeper_mutex ) ;
2020-11-03 09:57:24 +00:00
shared - > zookeeper_config = config ;
2023-04-03 15:06:52 +00:00
reloadZooKeeperIfChangedImpl ( config , zkutil : : getZooKeeperConfigName ( * config ) , shared - > zookeeper , getZooKeeperLog ( ) , server_started ) ;
2020-11-03 07:52:31 +00:00
}
void Context : : reloadAuxiliaryZooKeepersConfigIfChanged ( const ConfigurationPtr & config )
{
2023-03-27 13:06:46 +00:00
bool server_started = isServerCompletelyStarted ( ) ;
2020-11-03 07:52:31 +00:00
std : : lock_guard lock ( shared - > auxiliary_zookeepers_mutex ) ;
shared - > auxiliary_zookeepers_config = config ;
2020-11-03 09:57:24 +00:00
2020-11-03 07:52:31 +00:00
for ( auto it = shared - > auxiliary_zookeepers . begin ( ) ; it ! = shared - > auxiliary_zookeepers . end ( ) ; )
2020-09-10 04:00:33 +00:00
{
2020-11-03 07:52:31 +00:00
if ( ! config - > has ( " auxiliary_zookeepers. " + it - > first ) )
it = shared - > auxiliary_zookeepers . erase ( it ) ;
else
{
2023-03-27 13:06:46 +00:00
reloadZooKeeperIfChangedImpl ( config , " auxiliary_zookeepers. " + it - > first , it - > second , getZooKeeperLog ( ) , server_started ) ;
2020-11-03 07:52:31 +00:00
+ + it ;
}
2020-09-10 04:00:33 +00:00
}
}
2020-11-03 07:52:31 +00:00
2017-06-15 20:08:26 +00:00
bool Context : : hasZooKeeper ( ) const
{
2023-03-23 12:58:39 +00:00
return zkutil : : hasZooKeeperConfig ( getConfigRef ( ) ) ;
2017-06-15 20:08:26 +00:00
}
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
2022-05-03 08:06:28 +00:00
InterserverCredentialsPtr Context : : getInterserverCredentials ( ) const
2020-08-26 08:36:58 +00:00
{
2021-04-07 13:52:11 +00:00
return shared - > interserver_io_credentials . get ( ) ;
2020-08-26 08:36:58 +00:00
}
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 ) ) ;
2020-08-26 08:36:58 +00:00
}
2014-11-19 20:40:51 +00:00
void Context : : setInterserverIOAddress ( const String & host , UInt16 port )
2014-03-21 19:49:27 +00:00
{
shared - > interserver_io_host = host ;
shared - > interserver_io_port = port ;
}
2014-11-19 20:40:51 +00:00
std : : pair < String , UInt16 > Context : : getInterserverIOAddress ( ) const
2014-03-21 19:49:27 +00:00
{
2014-11-19 20:40:51 +00:00
if ( shared - > interserver_io_host . empty ( ) | | shared - > interserver_io_port = = 0 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NO_ELEMENTS_IN_CONFIG ,
" Parameter 'interserver_http(s)_port' required for replication is not specified "
" in configuration file. " ) ;
2014-11-19 20:40:51 +00:00
return { shared - > interserver_io_host , shared - > interserver_io_port } ;
2014-03-21 19:49:27 +00:00
}
2018-07-30 18:32:21 +00:00
void Context : : setInterserverScheme ( const String & scheme )
2014-03-21 19:49:27 +00:00
{
2018-07-30 18:32:21 +00:00
shared - > interserver_scheme = scheme ;
}
2014-11-19 20:40:51 +00:00
2018-07-30 18:32:21 +00:00
String Context : : getInterserverScheme ( ) const
{
return shared - > interserver_scheme ;
2014-03-21 19:49:27 +00:00
}
2019-10-10 12:58:06 +00:00
void Context : : setRemoteHostFilter ( const Poco : : Util : : AbstractConfiguration & config )
2019-10-09 20:29:41 +00:00
{
2019-10-10 12:58:06 +00:00
shared - > remote_host_filter . setValuesFromConfig ( config ) ;
2019-10-09 20:29:41 +00:00
}
2019-10-21 14:36:24 +00:00
const RemoteHostFilter & Context : : getRemoteHostFilter ( ) const
2019-10-09 20:29:41 +00:00
{
2019-10-10 12:58:06 +00:00
return shared - > remote_host_filter ;
2019-10-09 20:29:41 +00:00
}
2023-06-15 13:49:49 +00:00
void Context : : setHTTPHeaderFilter ( const Poco : : Util : : AbstractConfiguration & config )
{
shared - > http_header_filter . setValuesFromConfig ( config ) ;
}
const HTTPHeaderFilter & Context : : getHTTPHeaderFilter ( ) const
{
return shared - > http_header_filter ;
}
2015-09-24 04:50:53 +00:00
UInt16 Context : : getTCPPort ( ) const
{
2017-08-24 14:51:13 +00:00
auto lock = getLock ( ) ;
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef ( ) ;
2019-06-14 15:35:45 +00:00
return config . getInt ( " tcp_port " , DBMS_DEFAULT_PORT ) ;
2015-09-24 04:50:53 +00:00
}
2018-11-06 14:42:30 +00:00
std : : optional < UInt16 > Context : : getTCPPortSecure ( ) const
{
auto lock = getLock ( ) ;
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef ( ) ;
2018-11-06 14:42:30 +00:00
if ( config . has ( " tcp_port_secure " ) )
return config . getInt ( " tcp_port_secure " ) ;
return { } ;
}
2014-03-21 19:49:27 +00:00
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 ;
}
2016-10-14 15:06:46 +00:00
std : : shared_ptr < Cluster > Context : : getCluster ( const std : : string & cluster_name ) const
2013-12-07 16:51:29 +00:00
{
2021-11-25 14:28:06 +00:00
if ( auto res = tryGetCluster ( cluster_name ) )
2021-03-09 17:05:24 +00:00
return res ;
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : BAD_GET , " Requested cluster '{}' not found " , cluster_name ) ;
2013-12-07 16:51:29 +00:00
}
2015-01-10 02:30:03 +00:00
2016-10-10 08:44:52 +00:00
2016-10-14 15:06:46 +00:00
std : : shared_ptr < Cluster > Context : : tryGetCluster ( const std : : string & cluster_name ) const
{
2023-04-18 12:01:45 +00:00
std : : shared_ptr < Cluster > res = nullptr ;
2023-04-14 19:41:26 +00:00
2023-04-18 12:01:45 +00:00
{
std : : lock_guard lock ( shared - > clusters_mutex ) ;
res = getClustersImpl ( lock ) - > getCluster ( cluster_name ) ;
2023-04-14 19:41:26 +00:00
2023-04-18 12:01:45 +00:00
if ( res = = nullptr & & shared - > cluster_discovery )
res = shared - > cluster_discovery - > getCluster ( cluster_name ) ;
}
2023-04-14 19:41:26 +00:00
2023-04-18 12:01:45 +00:00
if ( res = = nullptr & & ! cluster_name . empty ( ) )
2021-11-25 14:28:06 +00:00
res = tryGetReplicatedDatabaseCluster ( cluster_name ) ;
2023-04-14 19:41:26 +00:00
2021-11-25 14:28:06 +00:00
return res ;
2013-12-07 16:51:29 +00:00
}
2015-01-10 02:30:03 +00:00
2016-10-14 15:06:46 +00:00
2021-05-31 14:49:02 +00:00
void Context : : reloadClusterConfig ( ) const
2018-01-15 14:13:19 +00:00
{
2018-03-26 14:12:07 +00:00
while ( true )
{
ConfigurationPtr cluster_config ;
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( shared - > clusters_mutex ) ;
2018-03-26 14:12:07 +00:00
cluster_config = shared - > clusters_config ;
}
2020-04-22 06:01:33 +00:00
const auto & config = cluster_config ? * cluster_config : getConfigRef ( ) ;
2022-04-27 23:32:49 +00:00
auto new_clusters = std : : make_shared < Clusters > ( config , settings , getMacros ( ) ) ;
2018-03-26 14:12:07 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( shared - > clusters_mutex ) ;
2018-03-26 14:12:07 +00:00
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
2018-03-26 14:12:07 +00:00
}
}
2018-01-15 14:13:19 +00:00
}
2023-04-14 19:41:26 +00:00
std : : map < String , ClusterPtr > 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 ) ;
2023-04-18 12:01:45 +00:00
auto clusters = getClustersImpl ( lock ) - > getContainer ( ) ;
2023-04-14 19:41:26 +00:00
if ( shared - > cluster_discovery )
{
2023-04-18 14:10:46 +00:00
const auto & cluster_discovery_map = shared - > cluster_discovery - > getClusters ( ) ;
for ( const auto & [ name , cluster ] : cluster_discovery_map )
2023-04-14 19:41:26 +00:00
clusters . emplace ( name , cluster ) ;
}
return clusters ;
}
2018-01-15 14:13:19 +00:00
2023-04-18 12:01:45 +00:00
std : : shared_ptr < Clusters > Context : : getClustersImpl ( std : : lock_guard < std : : mutex > & /* lock */ ) const
2015-04-30 12:43:16 +00:00
{
2018-01-15 14:13:19 +00:00
if ( ! shared - > clusters )
2016-03-10 03:08:09 +00:00
{
2020-04-22 06:01:33 +00:00
const auto & config = shared - > clusters_config ? * shared - > clusters_config : getConfigRef ( ) ;
2022-04-27 23:32:49 +00:00
shared - > clusters = std : : make_shared < Clusters > ( config , settings , getMacros ( ) ) ;
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
}
2015-01-10 02:30:03 +00:00
2021-11-18 08:57:26 +00:00
void Context : : startClusterDiscovery ( )
2021-11-11 09:03:53 +00:00
{
2023-04-18 12:01:45 +00:00
std : : lock_guard lock ( shared - > clusters_mutex ) ;
2021-11-11 09:03:53 +00:00
if ( ! shared - > cluster_discovery )
return ;
shared - > cluster_discovery - > start ( ) ;
}
2016-10-10 08:44:52 +00:00
2016-10-14 15:06:46 +00:00
/// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters
2021-11-16 13:01:57 +00:00
void Context : : setClustersConfig ( const ConfigurationPtr & config , bool enable_discovery , const String & config_name )
2016-10-10 08:44:52 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( shared - > clusters_mutex ) ;
2022-08-09 17:53:32 +00:00
if ( ConfigHelper : : getBool ( * config , " allow_experimental_cluster_discovery " ) & & enable_discovery & & ! shared - > cluster_discovery )
2021-11-11 09:03:53 +00:00
{
2021-11-18 08:57:26 +00:00
shared - > cluster_discovery = std : : make_unique < ClusterDiscovery > ( * config , getGlobalContext ( ) ) ;
2021-11-11 09:03:53 +00:00
}
2021-03-12 21:17:19 +00:00
/// 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 ) )
2021-03-12 21:17:19 +00:00
return ;
auto old_clusters_config = shared - > clusters_config ;
2016-10-10 08:44:52 +00:00
shared - > clusters_config = config ;
2017-10-13 19:13:41 +00:00
if ( ! shared - > clusters )
2022-04-27 23:32:49 +00:00
shared - > clusters = std : : make_shared < Clusters > ( * shared - > clusters_config , settings , getMacros ( ) , config_name ) ;
2017-10-13 19:13:41 +00:00
else
2021-03-12 21:17:19 +00:00
shared - > clusters - > updateClusters ( * shared - > clusters_config , settings , config_name , old_clusters_config ) ;
2015-04-30 12:43:16 +00:00
}
2015-01-10 02:30:03 +00:00
2016-10-10 08:44:52 +00:00
2017-11-03 19:53:10 +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 ) ;
2017-11-03 19:53:10 +00:00
if ( ! shared - > clusters )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Clusters are not set " ) ;
2017-11-03 19:53:10 +00:00
shared - > clusters - > setCluster ( cluster_name , cluster ) ;
2015-04-30 12:43:16 +00:00
}
2015-01-10 02:30:03 +00:00
2016-10-10 08:44:52 +00:00
2019-08-04 15:51:04 +00:00
void Context : : initializeSystemLogs ( )
2015-01-10 02:30:03 +00:00
{
2016-03-22 01:50:19 +00:00
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 ( ) ) ;
2015-01-10 02:30:03 +00:00
}
2019-02-03 21:30:45 +00:00
void Context : : initializeTraceCollector ( )
2019-07-06 20:29:00 +00:00
{
2019-02-03 21:30:45 +00:00
shared - > initializeTraceCollector ( getTraceLog ( ) ) ;
2019-07-06 20:29:00 +00:00
}
2015-01-10 02:30:03 +00:00
2021-12-27 03:50:59 +00:00
# if USE_ROCKSDB
2021-12-28 10:06:13 +00:00
void Context : : initializeMergeTreeMetadataCache ( const String & dir , size_t size )
2021-12-08 02:40:59 +00:00
{
2022-01-13 07:27:41 +00:00
shared - > merge_tree_metadata_cache = MergeTreeMetadataCache : : create ( dir , size ) ;
2021-12-08 02:40:59 +00:00
}
2021-12-27 03:50:59 +00:00
# endif
2021-12-08 02:40:59 +00:00
2020-03-03 00:24:44 +00:00
bool Context : : hasTraceCollector ( ) const
2018-03-10 19:57:13 +00:00
{
2020-03-03 00:24:44 +00:00
return shared - > hasTraceCollector ( ) ;
2018-03-10 19:57:13 +00:00
}
2021-05-31 14:49:02 +00:00
std : : shared_ptr < QueryLog > Context : : getQueryLog ( ) const
2015-06-26 20:48:10 +00:00
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2015-06-26 20:48:10 +00:00
2020-03-02 17:25:36 +00:00
if ( ! shared - > system_logs )
2019-03-21 19:22:38 +00:00
return { } ;
2017-06-05 13:59:38 +00:00
2019-03-21 19:22:38 +00:00
return shared - > system_logs - > query_log ;
2015-06-26 20:48:10 +00:00
}
2021-05-31 14:49:02 +00:00
std : : shared_ptr < QueryThreadLog > Context : : getQueryThreadLog ( ) const
2018-05-31 15:54:08 +00:00
{
auto lock = getLock ( ) ;
2020-03-02 17:25:36 +00:00
if ( ! shared - > system_logs )
2019-03-21 19:22:38 +00:00
return { } ;
2015-06-26 20:48:10 +00:00
2019-03-21 19:22:38 +00:00
return shared - > system_logs - > query_thread_log ;
2015-06-26 20:48:10 +00:00
}
2021-06-18 13:44:08 +00:00
std : : shared_ptr < QueryViewsLog > Context : : getQueryViewsLog ( ) const
2018-05-31 15:54:08 +00:00
{
auto lock = getLock ( ) ;
2020-03-02 17:25:36 +00:00
if ( ! shared - > system_logs )
2019-03-21 19:22:38 +00:00
return { } ;
2015-06-26 20:48:10 +00:00
2021-06-18 13:44:08 +00:00
return shared - > system_logs - > query_views_log ;
2015-06-26 20:48:10 +00:00
}
2021-05-31 14:49:02 +00:00
std : : shared_ptr < PartLog > Context : : getPartLog ( const String & part_database ) const
2017-03-07 17:13:54 +00:00
{
auto lock = getLock ( ) ;
2019-02-22 16:10:14 +00:00
/// No part log or system logs are shutting down.
2020-03-02 17:25:36 +00:00
if ( ! shared - > system_logs )
2019-03-21 19:22:38 +00:00
return { } ;
2017-06-05 13:59:38 +00:00
2018-03-10 19:57:13 +00:00
/// 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.
2020-03-02 17:25:36 +00:00
if ( part_database = = DatabaseCatalog : : SYSTEM_DATABASE )
2019-03-21 19:22:38 +00:00
return { } ;
2017-06-05 13:59:38 +00:00
2019-03-21 19:22:38 +00:00
return shared - > system_logs - > part_log ;
2017-03-07 17:13:54 +00:00
}
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 ( ) ;
2020-03-02 17:25:36 +00:00
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
2019-07-31 14:03:23 +00:00
{
auto lock = getLock ( ) ;
2020-03-02 17:25:36 +00:00
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 ;
2019-07-31 14:03:23 +00:00
}
2017-03-07 17:13:54 +00:00
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 ( ) ;
2020-03-02 17:25:36 +00:00
if ( ! shared - > system_logs )
2019-08-13 14:31:46 +00:00
return { } ;
return shared - > system_logs - > metric_log ;
}
2021-04-10 23:33:54 +00:00
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
}
2021-03-05 14:57:16 +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 ;
}
2022-01-14 14:03:00 +00:00
std : : shared_ptr < TransactionsInfoLog > Context : : getTransactionsInfoLog ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > system_logs )
return { } ;
return shared - > system_logs - > transactions_info_log ;
}
2022-02-05 16:33:42 +00:00
std : : shared_ptr < ProcessorsProfileLog > Context : : getProcessorsProfileLog ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > system_logs )
return { } ;
return shared - > system_logs - > processors_profile_log ;
}
2022-04-30 05:00:40 +00:00
std : : shared_ptr < FilesystemCacheLog > Context : : getFilesystemCacheLog ( ) const
2022-04-22 15:34:47 +00:00
{
auto lock = getLock ( ) ;
if ( ! shared - > system_logs )
return { } ;
2023-02-07 17:50:31 +00:00
return shared - > system_logs - > filesystem_cache_log ;
}
std : : shared_ptr < FilesystemReadPrefetchesLog > Context : : getFilesystemReadPrefetchesLog ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > system_logs )
return { } ;
return shared - > system_logs - > filesystem_read_prefetches_log ;
2022-04-22 15:34:47 +00:00
}
2022-02-05 16:33:42 +00:00
2022-10-03 18:52:14 +00:00
std : : shared_ptr < AsynchronousInsertLog > Context : : getAsynchronousInsertLog ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > system_logs )
return { } ;
return shared - > system_logs - > asynchronous_insert_log ;
}
2018-12-21 12:17:30 +00:00
CompressionCodecPtr Context : : chooseCompressionCodec ( size_t part_size , double part_size_ratio ) const
2015-04-17 05:35:53 +00:00
{
2016-03-22 01:50:19 +00:00
auto lock = getLock ( ) ;
2015-04-17 05:35:53 +00:00
2018-12-21 12:17:30 +00:00
if ( ! shared - > compression_codec_selector )
2015-04-17 05:35:53 +00:00
{
constexpr auto config_name = " compression " ;
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef ( ) ;
2015-04-17 05:35:53 +00:00
if ( config . has ( config_name ) )
2018-12-21 12:17:30 +00:00
shared - > compression_codec_selector = std : : make_unique < CompressionCodecSelector > ( config , " compression " ) ;
2015-04-17 05:35:53 +00:00
else
2018-12-21 12:17:30 +00:00
shared - > compression_codec_selector = std : : make_unique < CompressionCodecSelector > ( ) ;
2015-04-17 05:35:53 +00:00
}
2018-12-21 12:17:30 +00:00
return shared - > compression_codec_selector - > choose ( part_size , part_size_ratio ) ;
2015-04-17 05:35:53 +00:00
}
2020-01-09 14:50:34 +00:00
DiskPtr Context : : getDisk ( const String & name ) const
2019-04-04 17:19:11 +00:00
{
2020-05-21 14:11:56 +00:00
std : : lock_guard lock ( shared - > storage_policies_mutex ) ;
2019-04-04 17:19:11 +00:00
2020-05-21 14:11:56 +00:00
auto disk_selector = getDiskSelector ( lock ) ;
2019-04-21 18:38:44 +00:00
2020-01-09 14:50:34 +00:00
return disk_selector - > get ( name ) ;
2019-04-21 18:38:44 +00:00
}
2022-09-30 12:12:06 +00:00
DiskPtr Context : : getOrCreateDisk ( const String & name , DiskCreator creator ) const
{
std : : lock_guard lock ( shared - > storage_policies_mutex ) ;
auto disk_selector = getDiskSelector ( lock ) ;
auto disk = disk_selector - > tryGet ( name ) ;
if ( ! disk )
2022-11-08 12:32:49 +00:00
{
disk = creator ( getDisksMap ( lock ) ) ;
const_cast < DiskSelector * > ( disk_selector . get ( ) ) - > addToDiskMap ( name , disk ) ;
}
2022-09-30 12:12:06 +00:00
return disk ;
}
2020-05-21 14:11:56 +00:00
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 ) ;
}
2023-01-06 15:10:00 +00:00
StoragePolicyPtr Context : : getStoragePolicyFromDisk ( const String & disk_name ) const
2022-09-27 20:34:31 +00:00
{
std : : lock_guard lock ( shared - > storage_policies_mutex ) ;
2023-01-06 15:10:00 +00:00
const std : : string storage_policy_name = StoragePolicySelector : : TMP_STORAGE_POLICY_PREFIX + disk_name ;
2022-09-27 20:34:31 +00:00
auto storage_policy_selector = getStoragePolicySelector ( lock ) ;
StoragePolicyPtr storage_policy = storage_policy_selector - > tryGet ( storage_policy_name ) ;
if ( ! storage_policy )
{
auto disk_selector = getDiskSelector ( lock ) ;
2023-01-06 15:10:00 +00:00
auto disk = disk_selector - > get ( disk_name ) ;
auto volume = std : : make_shared < SingleDiskVolume > ( " _volume_ " + disk_name , disk ) ;
2022-09-27 20:34:31 +00:00
static const auto move_factor_for_single_disk_volume = 0.0 ;
storage_policy = std : : make_shared < StoragePolicy > ( storage_policy_name , Volumes { volume } , move_factor_for_single_disk_volume ) ;
const_cast < StoragePolicySelector * > ( storage_policy_selector . get ( ) ) - > add ( storage_policy ) ;
}
/// Note: it is important to put storage policy into disk selector (and not recreate it on each call)
/// because in some places there are checks that storage policy pointers are the same from different tables.
/// (We can assume that tables with the same `disk` setting are on the same storage policy).
return storage_policy ;
}
2019-04-21 18:38:44 +00:00
2020-05-22 10:33:57 +00:00
DisksMap Context : : getDisksMap ( ) const
2019-04-21 18:38:44 +00:00
{
2020-05-21 14:11:56 +00:00
std : : lock_guard lock ( shared - > storage_policies_mutex ) ;
2022-09-30 12:12:06 +00:00
return getDisksMap ( lock ) ;
}
DisksMap Context : : getDisksMap ( std : : lock_guard < std : : mutex > & lock ) const
{
2020-05-21 14:11:56 +00:00
return getDiskSelector ( lock ) - > getDisksMap ( ) ;
}
2020-05-22 10:33:57 +00:00
StoragePoliciesMap Context : : getPoliciesMap ( ) const
2020-05-21 14:11:56 +00:00
{
std : : lock_guard lock ( shared - > storage_policies_mutex ) ;
return getStoragePolicySelector ( lock ) - > getPoliciesMap ( ) ;
}
2019-04-21 18:38:44 +00:00
2020-05-21 14:11:56 +00:00
DiskSelectorPtr Context : : getDiskSelector ( std : : lock_guard < std : : mutex > & /* lock */ ) const
{
2019-04-21 18:38:44 +00:00
if ( ! shared - > merge_tree_disk_selector )
2019-04-04 17:19:11 +00:00
{
2019-04-21 18:38:44 +00:00
constexpr auto config_name = " storage_configuration.disks " ;
2020-04-22 06:01:33 +00:00
const auto & config = getConfigRef ( ) ;
2019-04-04 17:19:11 +00:00
2022-04-12 08:52:37 +00:00
auto disk_selector = std : : make_shared < DiskSelector > ( ) ;
disk_selector - > initialize ( config , config_name , shared_from_this ( ) ) ;
shared - > merge_tree_disk_selector = disk_selector ;
2019-04-04 17:19:11 +00:00
}
2020-01-09 14:50:34 +00:00
return shared - > merge_tree_disk_selector ;
2019-04-21 18:38:44 +00:00
}
2019-04-04 17:19:11 +00:00
2020-05-21 14:11:56 +00:00
StoragePolicySelectorPtr Context : : getStoragePolicySelector ( std : : lock_guard < std : : mutex > & lock ) const
2019-05-24 19:03:07 +00:00
{
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 ( ) ;
2019-05-24 19:03:07 +00:00
2020-05-21 14:11:56 +00:00
shared - > merge_tree_storage_policy_selector = std : : make_shared < StoragePolicySelector > ( config , config_name , getDiskSelector ( lock ) ) ;
2020-01-09 14:50:34 +00:00
}
return shared - > merge_tree_storage_policy_selector ;
}
void Context : : updateStorageConfiguration ( const Poco : : Util : : AbstractConfiguration & config )
{
2020-05-21 14:11:56 +00:00
std : : lock_guard lock ( shared - > storage_policies_mutex ) ;
2020-01-09 14:50:34 +00:00
if ( shared - > merge_tree_disk_selector )
2021-04-10 23:33:54 +00:00
shared - > merge_tree_disk_selector
= shared - > merge_tree_disk_selector - > updateFromConfig ( config , " storage_configuration.disks " , shared_from_this ( ) ) ;
2020-01-09 14:50:34 +00:00
if ( shared - > merge_tree_storage_policy_selector )
{
try
{
2021-04-10 23:33:54 +00:00
shared - > merge_tree_storage_policy_selector = shared - > merge_tree_storage_policy_selector - > updateFromConfig (
config , " storage_configuration.policies " , shared - > merge_tree_disk_selector ) ;
2020-01-09 14:50:34 +00:00
}
catch ( Exception & e )
{
2021-04-10 23:33:54 +00:00
LOG_ERROR (
shared - > log , " An error has occurred while reloading storage policies, storage policies were not applied: {} " , e . message ( ) ) ;
2020-01-09 14:50:34 +00:00
}
2019-05-24 19:03:07 +00:00
}
2020-06-01 17:16:09 +00:00
if ( shared - > storage_s3_settings )
{
2022-04-03 22:33:59 +00:00
shared - > storage_s3_settings - > loadFromConfig ( " s3 " , config , getSettingsRef ( ) ) ;
2020-06-01 17:16:09 +00:00
}
2019-04-04 17:19:11 +00:00
}
2018-08-10 17:42:12 +00:00
const MergeTreeSettings & Context : : getMergeTreeSettings ( ) const
2015-07-16 21:32:51 +00:00
{
2016-03-22 01:50:19 +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 ;
}
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 ;
}
2020-06-01 17:16:09 +00:00
const StorageS3Settings & Context : : getStorageS3Settings ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > storage_s3_settings )
{
const auto & config = getConfigRef ( ) ;
2022-04-03 22:33:59 +00:00
shared - > storage_s3_settings . emplace ( ) . loadFromConfig ( " s3 " , config , getSettingsRef ( ) ) ;
2020-06-01 17:16:09 +00:00
}
return * shared - > storage_s3_settings ;
}
2015-07-16 21:32:51 +00:00
2019-01-04 12:10:00 +00:00
void Context : : checkCanBeDropped ( const String & database , const String & table , const size_t & size , const size_t & max_size_to_drop ) const
2017-01-19 19:11:12 +00:00
{
2018-08-06 08:25:29 +00:00
if ( ! max_size_to_drop | | size < = max_size_to_drop )
2017-01-19 19:11:12 +00:00
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 ) ;
2017-01-19 19:11:12 +00:00
if ( force_file_exists )
{
try
{
2021-04-28 20:48:34 +00:00
fs : : remove ( force_file ) ;
2017-01-19 19:11:12 +00:00
return ;
}
catch ( . . . )
{
2017-01-23 19:18:25 +00:00
/// User should recreate force file on each drop, it shouldn't be protected
2018-08-06 08:25:29 +00:00
tryLogCurrentException ( " Drop table check " , " Can't remove force file to enable table or partition drop " ) ;
2017-01-19 19:11:12 +00:00
}
}
2018-08-06 08:25:29 +00:00
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. \n Reason: \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: \n sudo 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 ( ) ) ;
2017-01-19 19:11:12 +00:00
}
2018-08-06 08:25:29 +00:00
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
2019-11-20 16:40:27 +00:00
shared - > max_table_size_to_drop . store ( max_size , std : : memory_order_relaxed ) ;
2018-08-06 08:25:29 +00:00
}
2019-01-04 12:10:00 +00:00
void Context : : checkTableCanBeDropped ( const String & database , const String & table , const size_t & table_size ) const
2018-08-06 08:25:29 +00:00
{
2019-11-20 16:40:27 +00:00
size_t max_table_size_to_drop = shared - > max_table_size_to_drop . load ( std : : memory_order_relaxed ) ;
2018-08-06 08:25:29 +00:00
2018-08-06 08:43:34 +00:00
checkCanBeDropped ( database , table , table_size , max_table_size_to_drop ) ;
2018-08-06 08:25:29 +00:00
}
2018-08-03 08:33:57 +00:00
void Context : : setMaxPartitionSizeToDrop ( size_t max_size )
2018-08-01 17:41:18 +00:00
{
2019-11-22 13:56:16 +00:00
// Is initialized at server startup and updated at config reload
2019-11-20 16:40:27 +00:00
shared - > max_partition_size_to_drop . store ( max_size , std : : memory_order_relaxed ) ;
2018-08-01 17:41:18 +00:00
}
2019-01-04 12:10:00 +00:00
void Context : : checkPartitionCanBeDropped ( const String & database , const String & table , const size_t & partition_size ) const
2018-08-01 17:41:18 +00:00
{
2019-11-20 16:40:27 +00:00
size_t max_partition_size_to_drop = shared - > max_partition_size_to_drop . load ( std : : memory_order_relaxed ) ;
2018-08-01 17:41:18 +00:00
2018-08-06 08:43:34 +00:00
checkCanBeDropped ( database , table , partition_size , max_partition_size_to_drop ) ;
2018-08-01 17:41:18 +00:00
}
2023-04-29 08:34:59 +00:00
InputFormatPtr Context : : getInputFormat ( const String & name , ReadBuffer & buf , const Block & sample , UInt64 max_block_size , const std : : optional < FormatSettings > & format_settings , const std : : optional < size_t > max_parsing_threads ) const
2016-02-13 06:37:19 +00:00
{
2023-04-29 08:34:59 +00:00
return FormatFactory : : instance ( ) . getInput ( name , buf , sample , shared_from_this ( ) , max_block_size , format_settings , max_parsing_threads ) ;
2016-02-13 06:37:19 +00:00
}
2021-10-11 16:11:50 +00:00
OutputFormatPtr Context : : getOutputFormat ( const String & name , WriteBuffer & buf , const Block & sample ) const
2016-02-13 06:37:19 +00:00
{
2021-10-11 16:11:50 +00:00
return FormatFactory : : instance ( ) . getOutputFormat ( name , buf , sample , shared_from_this ( ) ) ;
2016-02-13 06:37:19 +00:00
}
2020-12-30 03:07:30 +00:00
OutputFormatPtr Context : : getOutputFormatParallelIfPossible ( const String & name , WriteBuffer & buf , const Block & sample ) const
2016-02-13 06:37:19 +00:00
{
2021-04-10 23:33:54 +00:00
return FormatFactory : : instance ( ) . getOutputFormatParallelIfPossible ( name , buf , sample , shared_from_this ( ) ) ;
2016-02-13 06:37:19 +00:00
}
2022-09-11 01:21:34 +00:00
double Context : : getUptimeSeconds ( ) const
2016-04-09 07:47:08 +00:00
{
auto lock = getLock ( ) ;
return shared - > uptime_watch . elapsedSeconds ( ) ;
}
2018-03-13 10:41:47 +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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Can't reload config because config_reload_callback is not set. " ) ;
2018-03-13 10:41:47 +00:00
shared - > config_reload_callback ( ) ;
}
2023-07-26 17:58:41 +00:00
void Context : : setStartServersCallback ( StartStopServersCallback & & callback )
{
/// Is initialized at server startup, so lock isn't required. Otherwise use mutex.
shared - > start_servers_callback = std : : move ( callback ) ;
}
void Context : : setStopServersCallback ( StartStopServersCallback & & callback )
{
/// Is initialized at server startup, so lock isn't required. Otherwise use mutex.
shared - > stop_servers_callback = std : : move ( callback ) ;
}
void Context : : startServers ( const ServerType & server_type ) const
{
/// Use mutex if callback may be changed after startup.
if ( ! shared - > start_servers_callback )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Can't start servers because start_servers_callback is not set. " ) ;
shared - > start_servers_callback ( server_type ) ;
}
void Context : : stopServers ( const ServerType & server_type ) const
{
/// Use mutex if callback may be changed after startup.
if ( ! shared - > stop_servers_callback )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Can't stop servers because stop_servers_callback is not set. " ) ;
shared - > stop_servers_callback ( server_type ) ;
}
2018-03-13 10:41:47 +00:00
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.
2020-10-23 14:57:06 +00:00
if ( shared - > merge_tree_disk_selector )
2020-09-04 14:17:27 +00:00
{
2020-10-23 14:57:06 +00:00
for ( auto & [ disk_name , disk ] : getDisksMap ( ) )
{
LOG_INFO ( shared - > log , " Shutdown disk {} " , disk_name ) ;
disk - > shutdown ( ) ;
}
2020-09-04 14:17:27 +00:00
}
2022-09-27 08:01:08 +00:00
/// Special volumes might also use disks that require shutdown.
2023-04-05 10:07:44 +00:00
auto & tmp_data = shared - > root_temp_data_on_disk ;
2022-09-27 08:01:08 +00:00
if ( tmp_data & & tmp_data - > getVolume ( ) )
2022-01-31 20:47:04 +00:00
{
2022-09-27 08:01:08 +00:00
auto & disks = tmp_data - > getVolume ( ) - > getDisks ( ) ;
2022-05-02 17:09:51 +00:00
for ( auto & disk : disks )
disk - > shutdown ( ) ;
2022-01-31 20:47:04 +00:00
}
2015-04-16 06:12:35 +00:00
shared - > shutdown ( ) ;
}
2016-11-11 17:01:02 +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 ;
Do not require proper Poco Application for Context
$ yes | head | clickhouse-obfuscator --structure 'id String' --seed foo --input-format TSV --output-format TSV
Poco::Exception. Code: 1000, e.code() = 0, Null pointer: _pInstance in file "/src/ch/clickhouse/base/poco/Util/include/Poco/Util/Application.h", line 446, Stack trace (when copying this message, always include the lines below):
0. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/exception:134: std::exception::capture() @ 0x15605622 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
1. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/exception:112: std::exception::exception[abi:v15000]() @ 0x156055ed in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
2. /src/ch/clickhouse/base/poco/Foundation/src/Exception.cpp:27: Poco::Exception::Exception(String const&, int) @ 0x2ebd2d80 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
3. /src/ch/clickhouse/base/poco/Foundation/src/Exception.cpp:132: Poco::LogicException::LogicException(String const&, int) @ 0x2ebd3667 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
4. /src/ch/clickhouse/base/poco/Foundation/src/Exception.cpp:134: Poco::NullPointerException::NullPointerException(String const&, int) @ 0x2ebd3da7 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
5. /src/ch/clickhouse/base/poco/Foundation/src/Bugcheck.cpp:42: Poco::Bugcheck::nullPointer(char const*, char const*, int) @ 0x2ebc5851 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
6. /src/ch/clickhouse/base/poco/Util/include/Poco/Util/Application.h:446: Poco::Util::Application::instance() @ 0x1ee3e6cd in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
7. /src/ch/clickhouse/src/Interpreters/Context.cpp:358: DB::ContextSharedPart::ContextSharedPart() @ 0x263b0cca in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
8. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:714: std::__unique_if<DB::ContextSharedPart>::__unique_single std::make_unique[abi:v15000]<DB::ContextSharedPart>() @ 0x26387b6e in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
9. /src/ch/clickhouse/src/Interpreters/Context.cpp:652: DB::Context::createShared() @ 0x26361a3c in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
10. /src/ch/clickhouse/programs/obfuscator/Obfuscator.cpp:1293: mainEntryClickHouseObfuscator(int, char**) @ 0x1ef6eb26 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
11. /src/ch/clickhouse/programs/main.cpp:481: main @ 0x155e569a in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
12. ? @ 0x7ffff7dd2790 in ?
13. __libc_start_main @ 0x7ffff7dd284a in ?
14. _start @ 0x155e51ae in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
(version 23.3.1.2537)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-03-30 19:10:57 +00:00
if ( type = = ApplicationType : : SERVER )
2023-05-11 14:59:19 +00:00
{
Do not require proper Poco Application for Context
$ yes | head | clickhouse-obfuscator --structure 'id String' --seed foo --input-format TSV --output-format TSV
Poco::Exception. Code: 1000, e.code() = 0, Null pointer: _pInstance in file "/src/ch/clickhouse/base/poco/Util/include/Poco/Util/Application.h", line 446, Stack trace (when copying this message, always include the lines below):
0. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/exception:134: std::exception::capture() @ 0x15605622 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
1. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/exception:112: std::exception::exception[abi:v15000]() @ 0x156055ed in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
2. /src/ch/clickhouse/base/poco/Foundation/src/Exception.cpp:27: Poco::Exception::Exception(String const&, int) @ 0x2ebd2d80 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
3. /src/ch/clickhouse/base/poco/Foundation/src/Exception.cpp:132: Poco::LogicException::LogicException(String const&, int) @ 0x2ebd3667 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
4. /src/ch/clickhouse/base/poco/Foundation/src/Exception.cpp:134: Poco::NullPointerException::NullPointerException(String const&, int) @ 0x2ebd3da7 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
5. /src/ch/clickhouse/base/poco/Foundation/src/Bugcheck.cpp:42: Poco::Bugcheck::nullPointer(char const*, char const*, int) @ 0x2ebc5851 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
6. /src/ch/clickhouse/base/poco/Util/include/Poco/Util/Application.h:446: Poco::Util::Application::instance() @ 0x1ee3e6cd in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
7. /src/ch/clickhouse/src/Interpreters/Context.cpp:358: DB::ContextSharedPart::ContextSharedPart() @ 0x263b0cca in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
8. /src/ch/clickhouse/contrib/llvm-project/libcxx/include/__memory/unique_ptr.h:714: std::__unique_if<DB::ContextSharedPart>::__unique_single std::make_unique[abi:v15000]<DB::ContextSharedPart>() @ 0x26387b6e in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
9. /src/ch/clickhouse/src/Interpreters/Context.cpp:652: DB::Context::createShared() @ 0x26361a3c in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
10. /src/ch/clickhouse/programs/obfuscator/Obfuscator.cpp:1293: mainEntryClickHouseObfuscator(int, char**) @ 0x1ef6eb26 in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
11. /src/ch/clickhouse/programs/main.cpp:481: main @ 0x155e569a in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
12. ? @ 0x7ffff7dd2790 in ?
13. __libc_start_main @ 0x7ffff7dd284a in ?
14. _start @ 0x155e51ae in /src/ch/clickhouse/.cmake-debug/programs/clickhouse
(version 23.3.1.2537)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-03-30 19:10:57 +00:00
shared - > server_settings . loadSettingsFromConfig ( Poco : : Util : : Application : : instance ( ) . config ( ) ) ;
2023-05-11 14:59:19 +00:00
shared - > configureServerWideThrottling ( ) ;
}
2016-11-11 17:01:02 +00:00
}
2018-02-01 13:52:29 +00:00
void Context : : setDefaultProfiles ( const Poco : : Util : : AbstractConfiguration & config )
{
shared - > default_profile_name = config . getString ( " default_profile " , " default " ) ;
2021-11-02 11:06:20 +00:00
getAccessControl ( ) . setDefaultProfileName ( shared - > default_profile_name ) ;
2020-03-04 22:27:03 +00:00
2018-02-01 13:52:29 +00:00
shared - > system_profile_name = config . getString ( " system_profile " , shared - > default_profile_name ) ;
2021-07-22 16:07:03 +00:00
setCurrentProfile ( shared - > system_profile_name ) ;
2021-01-27 18:05:18 +00:00
2021-03-25 06:34:28 +00:00
applySettingsQuirks ( settings , & Poco : : Logger : : get ( " SettingsQuirks " ) ) ;
2021-01-27 18:05:18 +00:00
shared - > buffer_profile_name = config . getString ( " buffer_profile " , shared - > system_profile_name ) ;
2021-04-10 23:33:54 +00:00
buffer_context = Context : : createCopy ( shared_from_this ( ) ) ;
2021-07-22 16:07:03 +00:00
buffer_context - > setCurrentProfile ( shared - > buffer_profile_name ) ;
2018-02-01 13:52:29 +00:00
}
2016-11-11 17:01:02 +00:00
2017-03-23 14:14:56 +00:00
String Context : : getDefaultProfileName ( ) const
{
return shared - > default_profile_name ;
}
2018-02-01 13:52:29 +00:00
String Context : : getSystemProfileName ( ) const
2017-03-23 14:14:56 +00:00
{
2018-02-01 13:52:29 +00:00
return shared - > system_profile_name ;
2017-03-23 14:14:56 +00:00
}
2017-11-10 06:48:28 +00:00
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
2018-05-30 19:23:15 +00:00
{
2021-09-28 18:42:41 +00:00
assert ( hasQueryContext ( ) ) ;
2021-04-10 23:33:54 +00:00
return getQueryContext ( ) - > sample_block_cache ;
2018-05-30 19:23:15 +00:00
}
2018-08-30 16:31:20 +00:00
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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " Duplicate name {} of query parameter " , backQuote ( name ) ) ;
2019-05-18 21:07:23 +00:00
}
2022-08-12 12:28:35 +00:00
void Context : : addQueryParameters ( const NameToNameMap & parameters )
{
for ( const auto & [ name , value ] : parameters )
query_parameters . insert_or_assign ( name , value ) ;
}
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
2018-11-22 15:59:00 +00:00
{
auto lock = getLock ( ) ;
shared - > bridge_commands . emplace_back ( std : : move ( cmd ) ) ;
}
2019-03-06 16:41:35 +00:00
IHostContextPtr & Context : : getHostContext ( )
{
return host_context ;
}
const IHostContextPtr & Context : : getHostContext ( ) const
{
return host_context ;
}
2022-09-09 16:27:19 +00:00
std : : shared_ptr < ActionLocksManager > Context : : getActionLocksManager ( ) const
2018-05-21 13:49:54 +00:00
{
auto lock = getLock ( ) ;
if ( ! shared - > action_locks_manager )
2021-04-10 23:33:54 +00:00
shared - > action_locks_manager = std : : make_shared < ActionLocksManager > ( shared_from_this ( ) ) ;
2018-05-21 13:49:54 +00:00
return shared - > action_locks_manager ;
}
2018-08-13 09:11:58 +00:00
2018-06-01 15:32:27 +00:00
void Context : : setExternalTablesInitializer ( ExternalTablesInitializer & & initializer )
{
if ( external_tables_initializer_callback )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " External tables initializer is already set " ) ;
2018-06-01 15:32:27 +00:00
external_tables_initializer_callback = std : : move ( initializer ) ;
}
void Context : : initializeExternalTablesIfSet ( )
{
if ( external_tables_initializer_callback )
{
2021-04-10 23:33:54 +00:00
external_tables_initializer_callback ( shared_from_this ( ) ) ;
2018-06-01 15:32:27 +00:00
/// Reset callback
external_tables_initializer_callback = { } ;
}
}
2018-08-14 20:29:42 +00:00
2019-05-28 18:30:10 +00:00
void Context : : setInputInitializer ( InputInitializer & & initializer )
{
if ( input_initializer_callback )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Input initializer is already set " ) ;
2019-05-28 18:30:10 +00:00
input_initializer_callback = std : : move ( initializer ) ;
}
void Context : : initializeInput ( const StoragePtr & input_storage )
{
if ( ! input_initializer_callback )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Input initializer is not set " ) ;
2019-05-28 18:30:10 +00:00
2021-04-10 23:33:54 +00:00
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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Input blocks reader is already set " ) ;
2019-05-28 18:30:10 +00:00
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-03-10 10:15:27 +00:00
2023-07-07 10:49:50 +00:00
void Context : : setClientInfo ( const ClientInfo & client_info_ )
{
client_info = client_info_ ;
need_recalculate_access = true ;
}
void Context : : setClientName ( const String & client_name )
{
client_info . client_name = client_name ;
}
void Context : : setClientInterface ( ClientInfo : : Interface interface )
{
client_info . interface = interface ;
need_recalculate_access = true ;
}
void Context : : setClientVersion ( UInt64 client_version_major , UInt64 client_version_minor , UInt64 client_version_patch , unsigned client_tcp_protocol_version )
{
client_info . client_version_major = client_version_major ;
client_info . client_version_minor = client_version_minor ;
client_info . client_version_patch = client_version_patch ;
client_info . client_tcp_protocol_version = client_tcp_protocol_version ;
}
void Context : : setClientConnectionId ( uint32_t connection_id_ )
{
client_info . connection_id = connection_id_ ;
}
void Context : : setHttpClientInfo ( ClientInfo : : HTTPMethod http_method , const String & http_user_agent , const String & http_referer )
{
client_info . http_method = http_method ;
client_info . http_user_agent = http_user_agent ;
client_info . http_referer = http_referer ;
need_recalculate_access = true ;
}
void Context : : setForwardedFor ( const String & forwarded_for )
{
client_info . forwarded_for = forwarded_for ;
need_recalculate_access = true ;
}
void Context : : setQueryKind ( ClientInfo : : QueryKind query_kind )
{
client_info . query_kind = query_kind ;
}
void Context : : setQueryKindInitial ( )
{
/// TODO: Try to combine this function with setQueryKind().
client_info . setInitialQuery ( ) ;
}
void Context : : setQueryKindReplicatedDatabaseInternal ( )
{
/// TODO: Try to combine this function with setQueryKind().
client_info . is_replicated_database_internal = true ;
}
void Context : : setCurrentUserName ( const String & current_user_name )
{
/// TODO: Try to combine this function with setUser().
client_info . current_user = current_user_name ;
need_recalculate_access = true ;
}
void Context : : setCurrentAddress ( const Poco : : Net : : SocketAddress & current_address )
{
client_info . current_address = current_address ;
need_recalculate_access = true ;
}
void Context : : setInitialUserName ( const String & initial_user_name )
{
client_info . initial_user = initial_user_name ;
need_recalculate_access = true ;
}
void Context : : setInitialAddress ( const Poco : : Net : : SocketAddress & initial_address )
{
client_info . initial_address = initial_address ;
}
void Context : : setInitialQueryId ( const String & initial_query_id )
{
client_info . initial_query_id = initial_query_id ;
}
void Context : : setInitialQueryStartTime ( std : : chrono : : time_point < std : : chrono : : system_clock > initial_query_start_time )
{
client_info . initial_query_start_time = timeInSeconds ( initial_query_start_time ) ;
client_info . initial_query_start_time_microseconds = timeInMicroseconds ( initial_query_start_time ) ;
}
void Context : : setQuotaClientKey ( const String & quota_key_ )
{
client_info . quota_key = quota_key_ ;
need_recalculate_access = true ;
}
void Context : : setConnectionClientVersion ( UInt64 client_version_major , UInt64 client_version_minor , UInt64 client_version_patch , unsigned client_tcp_protocol_version )
{
client_info . connection_client_version_major = client_version_major ;
client_info . connection_client_version_minor = client_version_minor ;
client_info . connection_client_version_patch = client_version_patch ;
client_info . connection_tcp_protocol_version = client_tcp_protocol_version ;
}
void Context : : setReplicaInfo ( bool collaborate_with_initiator , size_t all_replicas_count , size_t number_of_current_replica )
{
client_info . collaborate_with_initiator = collaborate_with_initiator ;
client_info . count_participating_replicas = all_replicas_count ;
client_info . number_of_current_replica = number_of_current_replica ;
}
void Context : : increaseDistributedDepth ( )
{
+ + client_info . distributed_depth ;
}
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
2020-01-30 19:00:51 +00:00
{
if ( storage_id . uuid ! = UUIDHelpers : : Nil )
2020-01-30 19:00:51 +00:00
return storage_id ;
2020-02-10 18:19:35 +00:00
2020-03-13 15:41:36 +00:00
if ( ! storage_id )
2020-01-30 19:00:51 +00:00
{
2020-02-12 18:14:12 +00:00
if ( exception )
2023-01-23 13:16:14 +00:00
exception - > emplace ( ErrorCodes : : UNKNOWN_TABLE , " Both table name and UUID are empty " ) ;
2020-01-30 19:00:51 +00:00
return storage_id ;
}
2020-02-12 18:14:12 +00:00
bool look_for_external_table = where & StorageNamespace : : ResolveExternal ;
2021-11-12 19:13:09 +00:00
/// Global context should not contain temporary tables
2021-11-12 19:13:10 +00:00
if ( isGlobalContext ( ) )
2021-11-12 19:13:09 +00:00
look_for_external_table = false ;
2020-02-12 18:14:12 +00:00
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-01-30 19:00:51 +00:00
{
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 )
2023-01-23 13:16:14 +00:00
exception - > emplace ( Exception ( ErrorCodes : : UNKNOWN_TABLE , " External and temporary tables have no database, but {} is specified " ,
storage_id . database_name ) ) ;
2020-02-12 18:14:12 +00:00
return StorageID : : createEmpty ( ) ;
2020-01-30 19:00:51 +00:00
}
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 )
2017-06-02 18:48:33 +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
2017-06-02 18:48:33 +00:00
{
2021-04-10 23:33:54 +00:00
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 ;
} ;
2017-06-02 18:48:33 +00:00
2020-03-11 19:10:55 +00:00
/// Firstly look for temporary table in current context
2021-04-10 23:33:54 +00:00
if ( try_resolve ( shared_from_this ( ) ) )
2020-03-11 19:10:55 +00:00
return resolved_id ;
2017-06-02 18:48:33 +00:00
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
2021-04-10 23:33:54 +00:00
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 ;
2017-06-02 18:48:33 +00:00
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
2021-04-10 23:33:54 +00:00
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
}
2017-06-02 18:48:33 +00:00
2020-03-13 15:41:36 +00:00
/// Temporary table not found. It's table in current database.
2017-06-02 18:48:33 +00:00
2020-02-10 18:19:35 +00:00
if ( in_current_database )
2017-06-02 18:48:33 +00:00
{
2020-01-30 19:00:51 +00:00
if ( current_database . empty ( ) )
2020-02-12 18:14:12 +00:00
{
if ( exception )
2023-01-23 13:16:14 +00:00
exception - > emplace ( ErrorCodes : : UNKNOWN_DATABASE , " Default database is not selected " ) ;
2020-02-12 18:14:12 +00:00
return StorageID : : createEmpty ( ) ;
}
2020-01-30 19:00:51 +00:00
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 ;
2017-06-02 18:48:33 +00:00
}
2020-02-12 18:14:12 +00:00
if ( exception )
2023-01-23 13:16:14 +00:00
exception - > emplace ( Exception ( ErrorCodes : : UNKNOWN_TABLE , " Cannot resolve database name for table {} " , storage_id . getNameForLogs ( ) ) ) ;
2020-02-12 18:14:12 +00:00
return StorageID : : createEmpty ( ) ;
2020-01-30 19:00:51 +00:00
}
2017-06-02 18:48:33 +00:00
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 ) ;
2021-04-10 23:33:54 +00:00
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-09-23 19:53:27 +00:00
2022-01-31 22:27:55 +00:00
void Context : : checkTransactionsAreAllowed ( bool explicit_tcl_query /* = false */ ) const
{
2022-03-16 19:16:26 +00:00
if ( getConfigRef ( ) . getInt ( " allow_experimental_transactions " , 0 ) )
2022-01-31 22:27:55 +00:00
return ;
if ( explicit_tcl_query )
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Transactions are not supported " ) ;
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Experimental support for transactions is disabled, "
2022-02-04 18:18:20 +00:00
" however, some query or background task tried to access TransactionLog. "
2022-02-03 18:57:09 +00:00
" If you have not enabled this feature explicitly, then it's a bug. " ) ;
2022-01-31 22:27:55 +00:00
}
2021-04-09 12:53:51 +00:00
void Context : : initCurrentTransaction ( MergeTreeTransactionPtr txn )
{
2022-03-09 20:38:18 +00:00
merge_tree_transaction_holder = MergeTreeTransactionHolder ( txn , false , this ) ;
2021-04-09 12:53:51 +00:00
setCurrentTransaction ( std : : move ( txn ) ) ;
}
2021-03-31 17:55:04 +00:00
void Context : : setCurrentTransaction ( MergeTreeTransactionPtr txn )
{
assert ( ! merge_tree_transaction | | ! txn ) ;
2021-05-13 18:48:36 +00:00
assert ( this = = session_context . lock ( ) . get ( ) | | this = = query_context . lock ( ) . get ( ) ) ;
2021-03-31 17:55:04 +00:00
merge_tree_transaction = std : : move ( txn ) ;
2021-06-08 10:01:49 +00:00
if ( ! merge_tree_transaction )
merge_tree_transaction_holder = { } ;
2021-03-31 17:55:04 +00:00
}
MergeTreeTransactionPtr Context : : getCurrentTransaction ( ) const
{
return merge_tree_transaction ;
}
2022-03-14 20:43:34 +00:00
bool Context : : isServerCompletelyStarted ( ) const
{
auto lock = getLock ( ) ;
assert ( getApplicationType ( ) = = ApplicationType : : SERVER ) ;
return shared - > is_server_completely_started ;
}
void Context : : setServerCompletelyStarted ( )
{
2023-03-27 13:06:46 +00:00
{
std : : lock_guard lock ( shared - > zookeeper_mutex ) ;
if ( shared - > zookeeper )
shared - > zookeeper - > setServerCompletelyStarted ( ) ;
for ( auto & zk : shared - > auxiliary_zookeepers )
zk . second - > setServerCompletelyStarted ( ) ;
}
2022-03-14 20:43:34 +00:00
auto lock = getLock ( ) ;
assert ( global_context . lock ( ) . get ( ) = = this ) ;
assert ( ! shared - > is_server_completely_started ) ;
assert ( getApplicationType ( ) = = ApplicationType : : SERVER ) ;
shared - > is_server_completely_started = true ;
}
2021-05-31 14:49:02 +00:00
PartUUIDsPtr Context : : getPartUUIDs ( ) const
2020-11-20 17:23:53 +00:00
{
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 > ( ) ;
2020-11-20 17:23:53 +00:00
return part_uuids ;
}
2021-04-06 11:05:47 +00:00
2021-04-08 19:00:39 +00:00
ReadTaskCallback Context : : getReadTaskCallback ( ) const
2021-04-06 11:05:47 +00:00
{
if ( ! next_task_callback . has_value ( ) )
2021-12-09 10:39:28 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Next task callback is not set for query {} " , getInitialQueryId ( ) ) ;
2021-04-06 11:05:47 +00:00
return next_task_callback . value ( ) ;
}
2021-04-08 19:00:39 +00:00
void Context : : setReadTaskCallback ( ReadTaskCallback & & callback )
2021-04-06 11:05:47 +00:00
{
next_task_callback = callback ;
}
2021-12-09 10:39:28 +00:00
MergeTreeReadTaskCallback Context : : getMergeTreeReadTaskCallback ( ) const
{
if ( ! merge_tree_read_task_callback . has_value ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Next task callback for is not set for query {} " , getInitialQueryId ( ) ) ;
return merge_tree_read_task_callback . value ( ) ;
}
void Context : : setMergeTreeReadTaskCallback ( MergeTreeReadTaskCallback & & callback )
{
merge_tree_read_task_callback = callback ;
}
2023-02-03 13:34:18 +00:00
MergeTreeAllRangesCallback Context : : getMergeTreeAllRangesCallback ( ) const
{
if ( ! merge_tree_all_ranges_callback . has_value ( ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Next task callback is not set for query with id: {} " , getInitialQueryId ( ) ) ;
return merge_tree_all_ranges_callback . value ( ) ;
}
void Context : : setMergeTreeAllRangesCallback ( MergeTreeAllRangesCallback & & callback )
{
merge_tree_all_ranges_callback = callback ;
}
void Context : : setParallelReplicasGroupUUID ( UUID uuid )
{
parallel_replicas_group_uuid = uuid ;
}
UUID Context : : getParallelReplicasGroupUUID ( ) const
{
return parallel_replicas_group_uuid ;
}
2021-05-31 14:49:02 +00:00
PartUUIDsPtr Context : : getIgnoredPartUUIDs ( ) const
2020-11-20 17:23:53 +00:00
{
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 > ( ) ;
2020-11-20 17:23:53 +00:00
return ignored_part_uuids ;
}
2021-04-21 13:16:16 +00:00
AsynchronousInsertQueue * Context : : getAsynchronousInsertQueue ( ) const
2021-03-17 14:11:47 +00:00
{
2021-04-21 13:16:16 +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 ) = = 0 ms )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INVALID_SETTING_VALUE , " Setting async_insert_busy_timeout_ms can't be zero " ) ;
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 ( ) ;
2022-04-19 15:01:41 +00:00
if ( shared - > are_background_executors_initialized )
2021-10-19 08:19:43 +00:00
return ;
2021-10-16 10:17:00 +00:00
2023-03-30 18:14:17 +00:00
const ServerSettings & server_settings = shared - > server_settings ;
Refactor reading the pool setting & from server config. (#48055)
After #36425 there was a lot of confusions/problems with configuring pools - when the message was confusing, and settings need to be ajusted in several places.
See some examples in #44251, #43351, #47900, #46515.
The commit includes the following changes:
1) Introduced a unified mechanism for reading pool sizes from the configuration file(s). Previously, pool sizes were read from the Context.cpp with fallbacks to profiles, whereas main_config_reloader in Server.cpp read them directly without fallbacks.
2) Corrected the data type for background_merges_mutations_concurrency_ratio. It should be float instead of int.
3) Refactored the default values for settings. Previously, they were defined in multiple places throughout the codebase, but they are now defined in one place (or two, to be exact: Settings.h and ServerSettings.h).
4) Improved documentation, including the correct message in system.settings.
Additionally make the code more conform with #46550.
2023-03-30 14:44:11 +00:00
size_t background_pool_size = server_settings . background_pool_size ;
auto background_merges_mutations_concurrency_ratio = server_settings . background_merges_mutations_concurrency_ratio ;
size_t background_pool_max_tasks_count = static_cast < size_t > ( background_pool_size * background_merges_mutations_concurrency_ratio ) ;
String background_merges_mutations_scheduling_policy = server_settings . background_merges_mutations_scheduling_policy ;
size_t background_move_pool_size = server_settings . background_move_pool_size ;
size_t background_fetches_pool_size = server_settings . background_fetches_pool_size ;
size_t background_common_pool_size = server_settings . background_common_pool_size ;
2021-09-08 00:21:21 +00:00
2021-09-30 21:26:24 +00:00
/// With this executor we can execute more tasks than threads we have
2022-04-19 20:47:29 +00:00
shared - > merge_mutate_executor = std : : make_shared < MergeMutateBackgroundExecutor >
2021-09-02 17:40:29 +00:00
(
2021-09-30 21:26:24 +00:00
" MergeMutate " ,
2022-04-20 13:35:13 +00:00
/*max_threads_count*/ background_pool_size ,
Refactor reading the pool setting & from server config. (#48055)
After #36425 there was a lot of confusions/problems with configuring pools - when the message was confusing, and settings need to be ajusted in several places.
See some examples in #44251, #43351, #47900, #46515.
The commit includes the following changes:
1) Introduced a unified mechanism for reading pool sizes from the configuration file(s). Previously, pool sizes were read from the Context.cpp with fallbacks to profiles, whereas main_config_reloader in Server.cpp read them directly without fallbacks.
2) Corrected the data type for background_merges_mutations_concurrency_ratio. It should be float instead of int.
3) Refactored the default values for settings. Previously, they were defined in multiple places throughout the codebase, but they are now defined in one place (or two, to be exact: Settings.h and ServerSettings.h).
4) Improved documentation, including the correct message in system.settings.
Additionally make the code more conform with #46550.
2023-03-30 14:44:11 +00:00
/*max_tasks_count*/ background_pool_max_tasks_count ,
2023-02-14 19:37:09 +00:00
CurrentMetrics : : BackgroundMergesAndMutationsPoolTask ,
2023-03-16 17:31:41 +00:00
CurrentMetrics : : BackgroundMergesAndMutationsPoolSize ,
2023-02-14 19:37:09 +00:00
background_merges_mutations_scheduling_policy
2021-09-02 17:40:29 +00:00
) ;
2023-02-11 16:18:42 +00:00
LOG_INFO ( shared - > log , " Initialized background executor for merges and mutations with num_threads={}, num_tasks={}, scheduling_policy={} " ,
Refactor reading the pool setting & from server config. (#48055)
After #36425 there was a lot of confusions/problems with configuring pools - when the message was confusing, and settings need to be ajusted in several places.
See some examples in #44251, #43351, #47900, #46515.
The commit includes the following changes:
1) Introduced a unified mechanism for reading pool sizes from the configuration file(s). Previously, pool sizes were read from the Context.cpp with fallbacks to profiles, whereas main_config_reloader in Server.cpp read them directly without fallbacks.
2) Corrected the data type for background_merges_mutations_concurrency_ratio. It should be float instead of int.
3) Refactored the default values for settings. Previously, they were defined in multiple places throughout the codebase, but they are now defined in one place (or two, to be exact: Settings.h and ServerSettings.h).
4) Improved documentation, including the correct message in system.settings.
Additionally make the code more conform with #46550.
2023-03-30 14:44:11 +00:00
background_pool_size , background_pool_max_tasks_count , background_merges_mutations_scheduling_policy ) ;
2021-09-30 21:26:24 +00:00
2022-04-19 20:47:29 +00:00
shared - > moves_executor = std : : make_shared < OrdinaryBackgroundExecutor >
2021-09-02 17:40:29 +00:00
(
2021-09-30 21:26:24 +00:00
" Move " ,
2022-04-20 13:35:13 +00:00
background_move_pool_size ,
background_move_pool_size ,
2023-03-16 17:31:41 +00:00
CurrentMetrics : : BackgroundMovePoolTask ,
CurrentMetrics : : BackgroundMovePoolSize
2021-09-02 17:40:29 +00:00
) ;
2022-04-20 13:35:13 +00:00
LOG_INFO ( shared - > log , " Initialized background executor for move operations with num_threads={}, num_tasks={} " , background_move_pool_size , background_move_pool_size ) ;
2021-09-02 17:40:29 +00:00
2022-04-19 20:47:29 +00:00
shared - > fetch_executor = std : : make_shared < OrdinaryBackgroundExecutor >
2021-09-02 17:40:29 +00:00
(
2021-09-30 21:26:24 +00:00
" Fetch " ,
2022-04-20 13:35:13 +00:00
background_fetches_pool_size ,
background_fetches_pool_size ,
2023-03-16 17:31:41 +00:00
CurrentMetrics : : BackgroundFetchesPoolTask ,
CurrentMetrics : : BackgroundFetchesPoolSize
2021-09-02 17:40:29 +00:00
) ;
2022-04-20 13:35:13 +00:00
LOG_INFO ( shared - > log , " Initialized background executor for fetches with num_threads={}, num_tasks={} " , background_fetches_pool_size , background_fetches_pool_size ) ;
2021-09-30 21:26:24 +00:00
2022-04-19 20:47:29 +00:00
shared - > common_executor = std : : make_shared < OrdinaryBackgroundExecutor >
2021-09-30 21:26:24 +00:00
(
" Common " ,
2022-04-20 13:35:13 +00:00
background_common_pool_size ,
background_common_pool_size ,
2023-03-16 17:31:41 +00:00
CurrentMetrics : : BackgroundCommonPoolTask ,
CurrentMetrics : : BackgroundCommonPoolSize
2021-09-30 21:26:24 +00:00
) ;
2022-04-20 13:35:13 +00:00
LOG_INFO ( shared - > log , " Initialized background executor for common operations (e.g. clearing old parts) with num_threads={}, num_tasks={} " , background_common_pool_size , background_common_pool_size ) ;
2021-09-30 21:26:24 +00:00
2022-04-19 15:01:41 +00:00
shared - > are_background_executors_initialized = true ;
2021-08-30 19:37:03 +00:00
}
2022-04-19 15:01:41 +00:00
bool Context : : areBackgroundExecutorsInitialized ( )
{
auto lock = getLock ( ) ;
return shared - > are_background_executors_initialized ;
}
2021-08-30 19:37:03 +00:00
2021-09-30 21:26:24 +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
}
2021-09-30 21:26:24 +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
}
2021-09-30 21:26:24 +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
}
2021-09-30 21:26:24 +00:00
OrdinaryBackgroundExecutorPtr Context : : getCommonExecutor ( ) const
{
return shared - > common_executor ;
}
2022-09-23 17:35:16 +00:00
IAsynchronousReader & Context : : getThreadPoolReader ( FilesystemReaderType type ) const
2022-09-07 15:44:29 +00:00
{
auto lock = getLock ( ) ;
2022-09-23 17:35:16 +00:00
switch ( type )
2022-09-07 15:44:29 +00:00
{
2022-09-23 17:35:16 +00:00
case FilesystemReaderType : : ASYNCHRONOUS_REMOTE_FS_READER :
{
if ( ! shared - > asynchronous_remote_fs_reader )
2023-05-06 18:12:26 +00:00
shared - > asynchronous_remote_fs_reader = createThreadPoolReader ( type , getConfigRef ( ) ) ;
2022-09-23 17:35:16 +00:00
return * shared - > asynchronous_remote_fs_reader ;
}
case FilesystemReaderType : : ASYNCHRONOUS_LOCAL_FS_READER :
{
if ( ! shared - > asynchronous_local_fs_reader )
2023-05-06 18:12:26 +00:00
shared - > asynchronous_local_fs_reader = createThreadPoolReader ( type , getConfigRef ( ) ) ;
2022-09-23 17:35:16 +00:00
return * shared - > asynchronous_local_fs_reader ;
}
case FilesystemReaderType : : SYNCHRONOUS_LOCAL_FS_READER :
{
if ( ! shared - > synchronous_local_fs_reader )
2023-05-06 18:12:26 +00:00
shared - > synchronous_local_fs_reader = createThreadPoolReader ( type , getConfigRef ( ) ) ;
2022-09-23 17:35:16 +00:00
return * shared - > synchronous_local_fs_reader ;
}
}
2022-09-07 15:44:29 +00:00
}
ThreadPool & Context : : getThreadPoolWriter ( ) const
{
const auto & config = getConfigRef ( ) ;
auto lock = getLock ( ) ;
if ( ! shared - > threadpool_writer )
{
auto pool_size = config . getUInt ( " .threadpool_writer_pool_size " , 100 ) ;
auto queue_size = config . getUInt ( " .threadpool_writer_queue_size " , 1000000 ) ;
2023-03-30 18:08:38 +00:00
shared - > threadpool_writer = std : : make_unique < ThreadPool > (
CurrentMetrics : : IOWriterThreads , CurrentMetrics : : IOWriterThreadsActive , pool_size , pool_size , queue_size ) ;
2022-09-07 15:44:29 +00:00
}
return * shared - > threadpool_writer ;
}
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 ;
2021-09-20 13:59:44 +00:00
if ( auto opt_method = magic_enum : : enum_cast < LocalFSReadMethod > ( read_method_str ) )
2021-09-06 15:59:46 +00:00
res . local_fs_method = * opt_method ;
else
2021-09-20 13:59:44 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_READ_METHOD , " Unknown read method '{}' for local filesystem " , read_method_str ) ;
read_method_str = settings . remote_filesystem_read_method . value ;
if ( auto opt_method = magic_enum : : enum_cast < RemoteFSReadMethod > ( read_method_str ) )
res . remote_fs_method = * opt_method ;
else
throw Exception ( ErrorCodes : : UNKNOWN_READ_METHOD , " Unknown read method '{}' for remote filesystem " , 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 ;
2022-08-30 17:47:34 +00:00
res . load_marks_asynchronously = settings . load_marks_asynchronously ;
2023-02-07 17:50:31 +00:00
res . enable_filesystem_read_prefetches_log = settings . enable_filesystem_read_prefetches_log ;
2021-10-07 13:39:54 +00:00
res . remote_fs_read_max_backoff_ms = settings . remote_fs_read_max_backoff_ms ;
res . remote_fs_read_backoff_max_tries = settings . remote_fs_read_backoff_max_tries ;
2022-03-30 11:47:44 +00:00
res . enable_filesystem_cache = settings . enable_filesystem_cache ;
res . read_from_filesystem_cache_if_exists_otherwise_bypass_cache = settings . read_from_filesystem_cache_if_exists_otherwise_bypass_cache ;
2022-05-03 16:16:50 +00:00
res . enable_filesystem_cache_log = settings . enable_filesystem_cache_log ;
2021-09-21 06:40:59 +00:00
2023-01-25 15:28:47 +00:00
res . filesystem_cache_max_download_size = settings . filesystem_cache_max_download_size ;
2022-06-05 18:36:23 +00:00
res . skip_download_if_exceeds_query_cache = settings . skip_download_if_exceeds_query_cache ;
2022-06-05 08:21:36 +00:00
2021-10-31 19:53:24 +00:00
res . remote_read_min_bytes_for_seek = settings . remote_read_min_bytes_for_seek ;
2022-08-27 12:01:49 +00:00
/// Zero read buffer will not make progress.
if ( ! settings . max_read_buffer_size )
{
throw Exception ( ErrorCodes : : INVALID_SETTING_VALUE ,
" Invalid value '{}' for max_read_buffer_size " , settings . max_read_buffer_size ) ;
}
2023-05-01 12:43:43 +00:00
res . local_fs_buffer_size
= settings . max_read_buffer_size_local_fs ? settings . max_read_buffer_size_local_fs : settings . max_read_buffer_size ;
res . remote_fs_buffer_size
= settings . max_read_buffer_size_remote_fs ? settings . max_read_buffer_size_remote_fs : settings . max_read_buffer_size ;
2023-04-20 16:15:09 +00:00
res . prefetch_buffer_size = settings . prefetch_buffer_size ;
2021-08-24 21:45:58 +00:00
res . direct_io_threshold = settings . min_bytes_to_use_direct_io ;
res . mmap_threshold = settings . min_bytes_to_use_mmap_io ;
2023-05-26 13:55:30 +00:00
res . priority = Priority { settings . read_priority } ;
2021-08-24 21:45:58 +00:00
2022-07-14 15:33:22 +00:00
res . remote_throttler = getRemoteReadThrottler ( ) ;
2023-03-30 17:02:28 +00:00
res . local_throttler = getLocalReadThrottler ( ) ;
2022-07-13 17:07:28 +00:00
2021-10-07 13:39:54 +00:00
res . http_max_tries = settings . http_max_tries ;
res . http_retry_initial_backoff_ms = settings . http_retry_initial_backoff_ms ;
res . http_retry_max_backoff_ms = settings . http_retry_max_backoff_ms ;
2022-02-07 19:40:47 +00:00
res . http_skip_not_found_url_for_globs = settings . http_skip_not_found_url_for_globs ;
2021-10-07 13:39:54 +00:00
2021-08-24 21:45:58 +00:00
res . mmap_cache = getMMappedFileCache ( ) . get ( ) ;
return res ;
}
2023-03-30 17:06:49 +00:00
ReadSettings Context : : getBackupReadSettings ( ) const
{
2023-06-24 03:06:20 +00:00
ReadSettings read_settings = getReadSettings ( ) ;
read_settings . remote_throttler = getBackupsThrottler ( ) ;
read_settings . local_throttler = getBackupsThrottler ( ) ;
return read_settings ;
2023-03-30 17:06:49 +00:00
}
2022-03-21 08:52:48 +00:00
WriteSettings Context : : getWriteSettings ( ) const
{
WriteSettings res ;
2022-03-30 11:47:44 +00:00
res . enable_filesystem_cache_on_write_operations = settings . enable_filesystem_cache_on_write_operations ;
2022-05-14 12:26:04 +00:00
res . enable_filesystem_cache_log = settings . enable_filesystem_cache_log ;
2022-12-15 18:39:41 +00:00
res . throw_on_error_from_cache = settings . throw_on_error_from_cache_on_write_operations ;
2022-10-20 17:51:27 +00:00
res . s3_allow_parallel_part_upload = settings . s3_allow_parallel_part_upload ;
2022-03-21 08:52:48 +00:00
2022-07-14 15:33:22 +00:00
res . remote_throttler = getRemoteWriteThrottler ( ) ;
2023-03-30 17:02:28 +00:00
res . local_throttler = getLocalWriteThrottler ( ) ;
2022-07-13 17:48:57 +00:00
2022-03-21 08:52:48 +00:00
return res ;
}
2023-02-07 17:50:31 +00:00
std : : shared_ptr < AsyncReadCounters > Context : : getAsyncReadCounters ( ) const
{
auto lock = getLock ( ) ;
if ( ! async_read_counters )
async_read_counters = std : : make_shared < AsyncReadCounters > ( ) ;
return async_read_counters ;
}
2023-03-03 15:14:49 +00:00
Context : : ParallelReplicasMode Context : : getParallelReplicasMode ( ) const
{
2023-06-24 03:06:20 +00:00
const auto & settings_ref = getSettingsRef ( ) ;
2023-03-03 15:14:49 +00:00
using enum Context : : ParallelReplicasMode ;
2023-06-24 03:06:20 +00:00
if ( ! settings_ref . parallel_replicas_custom_key . value . empty ( ) )
2023-03-03 15:14:49 +00:00
return CUSTOM_KEY ;
2023-06-24 03:06:20 +00:00
if ( settings_ref . allow_experimental_parallel_reading_from_replicas > 0 & & ! settings_ref . use_hedged_requests )
2023-03-03 15:14:49 +00:00
return READ_TASKS ;
return SAMPLE_KEY ;
}
2023-02-03 13:34:18 +00:00
bool Context : : canUseParallelReplicasOnInitiator ( ) const
{
2023-06-24 03:06:20 +00:00
const auto & settings_ref = getSettingsRef ( ) ;
return getParallelReplicasMode ( ) = = ParallelReplicasMode : : READ_TASKS & & settings_ref . max_parallel_replicas > 1
2023-02-03 13:34:18 +00:00
& & ! getClientInfo ( ) . collaborate_with_initiator ;
}
bool Context : : canUseParallelReplicasOnFollower ( ) const
{
2023-06-24 03:06:20 +00:00
const auto & settings_ref = getSettingsRef ( ) ;
return getParallelReplicasMode ( ) = = ParallelReplicasMode : : READ_TASKS & & settings_ref . max_parallel_replicas > 1
2023-02-03 13:34:18 +00:00
& & getClientInfo ( ) . collaborate_with_initiator ;
}
2023-03-02 19:19:58 +00:00
void Context : : setPreparedSetsCache ( const PreparedSetsCachePtr & cache )
{
prepared_sets_cache = cache ;
}
PreparedSetsCachePtr Context : : getPreparedSetsCache ( ) const
{
return prepared_sets_cache ;
}
2023-02-10 10:47:06 +00:00
UInt64 Context : : getClientProtocolVersion ( ) const
{
return client_protocol_version ;
}
void Context : : setClientProtocolVersion ( UInt64 version )
{
client_protocol_version = version ;
}
2012-08-02 17:33:31 +00:00
}