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>
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-03-29 08:24:56 +00:00
# include <Coordination/KeeperStorageDispatcher.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>
2020-05-18 10:00:22 +00:00
# include <Processors/Formats/InputStreamFromInputFormat.h>
2017-10-13 01:02:16 +00:00
# include <Databases/IDatabase.h>
2017-04-01 09:19:00 +00:00
# include <Storages/IStorage.h>
# include <Storages/MarkCache.h>
# include <Storages/MergeTree/MergeList.h>
2020-10-27 12:47:42 +00:00
# include <Storages/MergeTree/ReplicatedFetchList.h>
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>
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>
2019-03-22 12:08:30 +00:00
# include <Core/Settings.h>
2021-03-25 06:34:28 +00:00
# include <Core/SettingsQuirks.h>
2019-11-09 15:33:07 +00:00
# include <Access/AccessControlManager.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>
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>
2019-09-26 10:23:14 +00:00
# include <Interpreters/ExternalModelsLoader.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>
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>
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>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTCreateQuery.h>
# include <Parsers/ParserCreateQuery.h>
# include <Parsers/parseQuery.h>
2019-07-29 22:26:44 +00:00
# include <Common/StackTrace.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>
2019-07-10 20:47:39 +00:00
# include <Common/TraceCollector.h>
2017-06-05 13:59:38 +00:00
# include <common/logger_useful.h>
2019-10-10 12:58:06 +00:00
# include <Common/RemoteHostFilter.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>
2020-10-14 16:09:18 +00:00
# include <Storages/MergeTree/BackgroundJobsExecutor.h>
2020-11-20 17:23:53 +00:00
# include <Storages/MergeTree/MergeTreeDataPartUUID.h>
2021-04-28 20:48:34 +00:00
# include <filesystem>
2020-01-30 19:00:51 +00:00
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
{
2017-04-01 07:20:54 +00:00
extern const Event ContextLock ;
2018-08-31 12:39:27 +00:00
extern const Event CompiledCacheSizeBytes ;
2016-12-07 22:49:42 +00:00
}
namespace CurrentMetrics
{
2017-04-01 07:20:54 +00:00
extern const Metric ContextLockWait ;
2019-12-12 22:08:54 +00:00
extern const Metric BackgroundMovePoolTask ;
2020-04-22 23:00:24 +00:00
extern const Metric BackgroundSchedulePoolTask ;
extern const Metric BackgroundBufferFlushSchedulePoolTask ;
extern const Metric BackgroundDistributedSchedulePoolTask ;
2020-08-31 11:06:35 +00:00
extern const Metric BackgroundMessageBrokerSchedulePoolTask ;
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 ;
2017-04-01 07:20:54 +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 ;
2017-04-01 07:20:54 +00:00
extern const int NO_ELEMENTS_IN_CONFIG ;
extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT ;
2017-06-02 18:48:33 +00:00
extern const int SESSION_NOT_FOUND ;
extern const int SESSION_IS_LOCKED ;
2019-09-03 19:53:59 +00:00
extern const int LOGICAL_ERROR ;
2020-03-05 20:02:55 +00:00
extern const int NOT_IMPLEMENTED ;
2016-01-11 21:46:36 +00:00
}
2015-04-16 06:12:35 +00:00
2020-03-05 19:23:39 +00:00
class NamedSessions
2020-01-30 19:00:51 +00:00
{
2020-03-05 03:12:07 +00:00
public :
2020-03-05 19:23:39 +00:00
using Key = NamedSessionKey ;
2020-01-30 19:00:51 +00:00
2020-03-05 19:23:39 +00:00
~ NamedSessions ( )
2020-01-30 19:00:51 +00:00
{
2020-03-05 03:12:07 +00:00
try
{
{
std : : lock_guard lock { mutex } ;
quit = true ;
}
cond . notify_one ( ) ;
thread . join ( ) ;
}
catch ( . . . )
2020-01-30 19:00:51 +00:00
{
2020-03-05 03:12:07 +00:00
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
2020-01-30 19:00:51 +00:00
}
}
2020-03-05 03:12:07 +00:00
/// Find existing session or create a new.
2020-03-05 19:23:39 +00:00
std : : shared_ptr < NamedSession > acquireSession (
const String & session_id ,
2021-05-31 14:49:02 +00:00
ContextMutablePtr context ,
2020-03-06 18:14:33 +00:00
std : : chrono : : steady_clock : : duration timeout ,
2020-03-05 19:23:39 +00:00
bool throw_if_not_found )
2020-01-30 19:00:51 +00:00
{
2020-03-05 03:12:07 +00:00
std : : unique_lock lock ( mutex ) ;
2021-04-10 23:33:54 +00:00
auto & user_name = context - > client_info . current_user ;
2020-03-05 03:57:31 +00:00
if ( user_name . empty ( ) )
throw Exception ( " Empty user name. " , ErrorCodes : : LOGICAL_ERROR ) ;
Key key ( user_name , session_id ) ;
2020-03-05 03:12:07 +00:00
auto it = sessions . find ( key ) ;
if ( it = = sessions . end ( ) )
{
if ( throw_if_not_found )
throw Exception ( " Session not found. " , ErrorCodes : : SESSION_NOT_FOUND ) ;
/// Create a new session from current context.
2020-03-05 19:23:39 +00:00
it = sessions . insert ( std : : make_pair ( key , std : : make_shared < NamedSession > ( key , context , timeout , * this ) ) ) . first ;
2020-03-05 03:12:07 +00:00
}
2021-04-10 23:33:54 +00:00
else if ( it - > second - > key . first ! = context - > client_info . current_user )
2020-03-05 03:12:07 +00:00
{
2020-11-06 18:02:16 +00:00
throw Exception ( " Session belongs to a different user " , ErrorCodes : : SESSION_IS_LOCKED ) ;
2020-03-05 03:12:07 +00:00
}
2020-03-05 03:57:31 +00:00
/// Use existing session.
2020-03-05 03:12:07 +00:00
const auto & session = it - > second ;
2020-03-05 04:10:48 +00:00
if ( ! session . unique ( ) )
2020-03-05 03:12:07 +00:00
throw Exception ( " Session is locked by a concurrent client. " , ErrorCodes : : SESSION_IS_LOCKED ) ;
2021-04-10 23:33:54 +00:00
session - > context - > client_info = context - > client_info ;
2020-06-10 14:24:42 +00:00
2020-03-05 03:12:07 +00:00
return session ;
2020-01-30 19:00:51 +00:00
}
2020-03-05 19:23:39 +00:00
void releaseSession ( NamedSession & session )
2020-01-30 19:00:51 +00:00
{
2020-03-05 03:12:07 +00:00
std : : unique_lock lock ( mutex ) ;
2020-03-06 18:14:33 +00:00
scheduleCloseSession ( session , lock ) ;
2020-01-30 19:00:51 +00:00
}
2020-03-05 03:12:07 +00:00
private :
class SessionKeyHash
{
public :
size_t operator ( ) ( const Key & key ) const
{
SipHash hash ;
hash . update ( key . first ) ;
hash . update ( key . second ) ;
return hash . get64 ( ) ;
}
} ;
2020-03-06 18:14:33 +00:00
/// TODO it's very complicated. Make simple std::map with time_t or boost::multi_index.
2020-03-05 19:23:39 +00:00
using Container = std : : unordered_map < Key , std : : shared_ptr < NamedSession > , SessionKeyHash > ;
2020-03-06 18:14:33 +00:00
using CloseTimes = std : : deque < std : : vector < Key > > ;
2020-03-05 03:12:07 +00:00
Container sessions ;
CloseTimes close_times ;
2020-03-06 18:14:33 +00:00
std : : chrono : : steady_clock : : duration close_interval = std : : chrono : : seconds ( 1 ) ;
std : : chrono : : steady_clock : : time_point close_cycle_time = std : : chrono : : steady_clock : : now ( ) ;
UInt64 close_cycle = 0 ;
void scheduleCloseSession ( NamedSession & session , std : : unique_lock < std : : mutex > & )
2020-01-30 19:00:51 +00:00
{
2020-03-06 18:14:33 +00:00
/// Push it on a queue of sessions to close, on a position corresponding to the timeout.
/// (timeout is measured from current moment of time)
const UInt64 close_index = session . timeout / close_interval + 1 ;
const auto new_close_cycle = close_cycle + close_index ;
if ( session . close_cycle ! = new_close_cycle )
{
session . close_cycle = new_close_cycle ;
if ( close_times . size ( ) < close_index + 1 )
close_times . resize ( close_index + 1 ) ;
close_times [ close_index ] . emplace_back ( session . key ) ;
}
2020-01-30 19:00:51 +00:00
}
2020-03-05 03:12:07 +00:00
void cleanThread ( )
2020-01-30 19:00:51 +00:00
{
2020-03-05 03:12:07 +00:00
setThreadName ( " SessionCleaner " ) ;
std : : unique_lock lock { mutex } ;
while ( true )
{
2020-03-06 18:14:33 +00:00
auto interval = closeSessions ( lock ) ;
if ( cond . wait_for ( lock , interval , [ this ] ( ) - > bool { return quit ; } ) )
2020-03-05 03:12:07 +00:00
break ;
}
2020-01-30 19:00:51 +00:00
}
2020-03-05 03:12:07 +00:00
/// Close sessions, that has been expired. Returns how long to wait for next session to be expired, if no new sessions will be added.
2020-03-06 18:14:33 +00:00
std : : chrono : : steady_clock : : duration closeSessions ( std : : unique_lock < std : : mutex > & lock )
2020-03-05 03:12:07 +00:00
{
2020-03-06 18:14:33 +00:00
const auto now = std : : chrono : : steady_clock : : now ( ) ;
/// The time to close the next session did not come
if ( now < close_cycle_time )
return close_cycle_time - now ; /// Will sleep until it comes.
const auto current_cycle = close_cycle ;
2020-03-05 03:12:07 +00:00
2020-03-06 18:14:33 +00:00
+ + close_cycle ;
close_cycle_time = now + close_interval ;
2020-03-05 03:12:07 +00:00
2020-03-06 18:14:33 +00:00
if ( close_times . empty ( ) )
return close_interval ;
2020-03-05 03:12:07 +00:00
2020-03-06 18:14:33 +00:00
auto & sessions_to_close = close_times . front ( ) ;
for ( const auto & key : sessions_to_close )
{
const auto session = sessions . find ( key ) ;
if ( session ! = sessions . end ( ) & & session - > second - > close_cycle < = current_cycle )
{
if ( ! session - > second . unique ( ) )
{
/// Skip but move it to close on the next cycle.
session - > second - > timeout = std : : chrono : : steady_clock : : duration { 0 } ;
scheduleCloseSession ( * session - > second , lock ) ;
}
else
sessions . erase ( session ) ;
}
2020-03-05 03:12:07 +00:00
}
2020-03-06 18:14:33 +00:00
close_times . pop_front ( ) ;
return close_interval ;
2020-03-05 03:12:07 +00:00
}
std : : mutex mutex ;
std : : condition_variable cond ;
std : : atomic < bool > quit { false } ;
2020-03-05 19:23:39 +00:00
ThreadFromGlobalPool thread { & NamedSessions : : cleanThread , this } ;
2020-01-30 19:00:51 +00:00
} ;
2015-04-16 06:12:35 +00:00
2020-03-05 19:23:39 +00:00
void NamedSession : : release ( )
2020-03-05 03:57:31 +00:00
{
parent . releaseSession ( * this ) ;
2016-01-11 21:46:36 +00:00
}
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
*/
2021-04-10 23:33:54 +00:00
struct ContextSharedPart
2015-04-16 06:12:35 +00:00
{
2020-05-30 21:57:37 +00:00
Poco : : Logger * log = & Poco : : Logger : : get ( " Context " ) ;
2017-04-01 07:20:54 +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-04-01 07:20:54 +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 ;
2017-10-17 10:44:46 +00:00
mutable std : : mutex external_models_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.
2017-04-01 07:20:54 +00:00
mutable std : : mutex zookeeper_mutex ;
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-05-18 14:08:56 +00:00
mutable std : : mutex keeper_storage_dispatcher_mutex ;
mutable std : : shared_ptr < KeeperStorageDispatcher > keeper_storage_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.
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.
2020-07-08 14:25:23 +00:00
mutable VolumePtr tmp_volume ; /// Volume for the the temporary files that occur when processing the request.
2020-01-19 14:26:28 +00:00
2019-02-04 23:18:04 +00:00
mutable std : : optional < EmbeddedDictionaries > embedded_dictionaries ; /// Metrica's dictionaries. Have lazy initialization.
2019-09-26 10:41:33 +00:00
mutable std : : optional < ExternalDictionariesLoader > external_dictionaries_loader ;
2019-09-26 10:23:14 +00:00
mutable std : : optional < ExternalModelsLoader > external_models_loader ;
2021-04-01 10:19:28 +00:00
ConfigurationPtr external_models_config ;
2021-06-15 19:55:21 +00:00
scope_guard models_repository_guard ;
2021-04-01 10:19:28 +00:00
2021-06-15 19:55:21 +00:00
scope_guard dictionaries_xmls ;
2021-05-12 18:16:01 +00:00
2017-04-01 07:20:54 +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
2019-11-09 15:33:07 +00:00
AccessControlManager access_control_manager ;
2017-04-02 17:37:49 +00:00
mutable UncompressedCachePtr uncompressed_cache ; /// The cache of decompressed blocks.
mutable MarkCachePtr mark_cache ; /// Cache of marks in compressed files.
2021-03-28 19:24:28 +00:00
mutable MMappedFileCachePtr mmap_cache ; /// Cache of mmapped files to avoid frequent open/map/unmap/close and to reuse from several threads.
2017-04-02 17:37:49 +00:00
ProcessList process_list ; /// Executing queries at the moment.
MergeList merge_list ; /// The list of executable merge (for (Replicated)?MergeTree)
2020-10-26 16:38:35 +00:00
ReplicatedFetchList replicated_fetch_list ;
2017-04-02 17:37:49 +00:00
ConfigurationPtr users_config ; /// Config with the users, profiles and quotas sections.
InterserverIOHandler interserver_io_handler ; /// Handler for interserver communication.
2020-11-24 11:38:02 +00:00
2020-11-06 14:07:56 +00:00
mutable std : : optional < BackgroundSchedulePool > buffer_flush_schedule_pool ; /// A thread pool that can do background flush for Buffer tables.
mutable std : : optional < BackgroundSchedulePool > schedule_pool ; /// A thread pool that can run different jobs in background (used in replicated tables)
mutable std : : optional < BackgroundSchedulePool > distributed_schedule_pool ; /// A thread pool that can run different jobs in background (used for distributed sends)
2021-01-27 18:08:06 +00:00
mutable std : : optional < BackgroundSchedulePool > message_broker_schedule_pool ; /// A thread pool that can run different jobs in background (used for message brokers, like RabbitMQ and Kafka)
2021-05-26 20:37:44 +00:00
mutable ThrottlerPtr replicated_fetches_throttler ; /// A server-wide throttler for replicated fetches
mutable ThrottlerPtr replicated_sends_throttler ; /// A server-wide throttler for replicated sends
2018-03-13 23:44:23 +00:00
MultiVersion < Macros > macros ; /// Substitutions extracted from config.
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
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
2019-10-10 12:58:06 +00:00
RemoteHostFilter remote_host_filter ; /// Allowed URL from config.xml
2019-02-03 09:57:12 +00:00
2020-03-03 00:24:44 +00:00
std : : optional < TraceCollector > trace_collector ; /// Thread collecting traces from threads executing queries
2020-03-05 19:23:39 +00:00
std : : optional < NamedSessions > named_sessions ; /// Controls named HTTP sessions.
2017-04-01 07:20:54 +00:00
/// Clusters for distributed tables
/// Initialized on demand (on distributed storages initialization) since Settings should be initialized
2021-06-26 14:15:57 +00:00
std : : shared_ptr < Clusters > clusters ;
2020-01-11 09:50:41 +00:00
ConfigurationPtr clusters_config ; /// Stores updated configs
2020-11-03 07:52:31 +00:00
mutable std : : mutex clusters_mutex ; /// Guards clusters and clusters_config
2017-04-01 07:20:54 +00:00
bool shutdown_called = false ;
Stopwatch uptime_watch ;
Context : : ApplicationType application_type = Context : : ApplicationType : : SERVER ;
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 ;
2021-04-10 23:33:54 +00:00
ContextSharedPart ( )
2019-09-26 16:12:15 +00:00
: 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 ( )
2017-04-01 07:20:54 +00:00
{
try
{
shutdown ( ) ;
}
catch ( . . . )
{
tryLogCurrentException ( __PRETTY_FUNCTION__ ) ;
}
}
2017-04-02 17:37:49 +00:00
/** Perform a complex job of destroying objects in advance.
2017-04-01 07:20:54 +00:00
*/
void shutdown ( )
{
if ( shutdown_called )
return ;
shutdown_called = true ;
2019-06-21 17:25:47 +00:00
/** After system_logs have been shut down it is guaranteed that no system table gets created or written to.
* Note that part changes at shutdown won ' t be logged to part log .
*/
2019-03-21 19:22:38 +00:00
2019-06-21 19:24:30 +00:00
if ( system_logs )
system_logs - > shutdown ( ) ;
2019-02-04 14:08:39 +00:00
2020-04-08 23:59:39 +00:00
DatabaseCatalog : : shutdown ( ) ;
2019-02-04 23:18:04 +00:00
Fix one more lock-order-inversion
TSan report [1]:
WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314)
Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368
Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689)
2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689)
3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f)
4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f)
5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f)
6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda)
7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e)
8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba)
9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba)
10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba)
11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba)
12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba)
13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba)
14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0)
15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0)
16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0)
17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600)
18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600)
19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600)
20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600)
21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600)
22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561)
23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561)
24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561)
25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561)
26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205)
27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205)
28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205)
29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8)
30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8)
31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8)
32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8)
Mutex M183938897938677368 previously acquired by the same thread here:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9)
2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9)
3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9)
4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9)
5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e)
6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba)
7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba)
8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba)
9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba)
10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba)
11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba)
12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0)
13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0)
14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0)
15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600)
16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600)
17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600)
18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600)
19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600)
20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561)
21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561)
22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561)
23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561)
24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205)
25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205)
26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205)
27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8)
28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8)
29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8)
30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8)
Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9)
2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9)
3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b)
4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b)
5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b)
6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768)
7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b)
8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b)
9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea)
10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea)
11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea)
12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea)
13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8)
14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8)
15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8)
16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8)
17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff)
18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff)
19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff)
20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff)
21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655)
22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655)
23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37)
24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2)
25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2)
26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2)
27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba)
28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b)
29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be)
30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093)
31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e)
32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9)
Mutex M2505 previously acquired by the same thread here:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689)
2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689)
3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f)
4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f)
5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37)
6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2)
7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2)
8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2)
9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba)
10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b)
11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be)
12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093)
13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e)
14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9)
[1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1
v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor)
v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_
v4: fix conflicts
2021-03-03 05:08:10 +00:00
std : : unique_ptr < SystemLogs > delete_system_logs ;
{
auto lock = std : : lock_guard ( mutex ) ;
2021-06-29 22:46:57 +00:00
/** Compiled expressions stored in cache need to be destroyed before destruction of static objects.
* Because CHJIT instance can be static object .
*/
2021-06-29 20:40:15 +00:00
# if USE_EMBEDDED_COMPILER
if ( auto * cache = CompiledExpressionCacheFactory : : instance ( ) . tryGetCache ( ) )
cache - > reset ( ) ;
# endif
Fix one more lock-order-inversion
TSan report [1]:
WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314)
Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368
Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689)
2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689)
3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f)
4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f)
5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f)
6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda)
7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e)
8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba)
9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba)
10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba)
11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba)
12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba)
13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba)
14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0)
15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0)
16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0)
17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600)
18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600)
19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600)
20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600)
21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600)
22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561)
23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561)
24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561)
25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561)
26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205)
27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205)
28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205)
29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8)
30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8)
31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8)
32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8)
Mutex M183938897938677368 previously acquired by the same thread here:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9)
2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9)
3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9)
4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9)
5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e)
6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba)
7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba)
8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba)
9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba)
10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba)
11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba)
12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0)
13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0)
14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0)
15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600)
16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600)
17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600)
18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600)
19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600)
20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561)
21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561)
22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561)
23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561)
24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205)
25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205)
26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205)
27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8)
28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8)
29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8)
30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8)
Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9)
2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9)
3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b)
4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b)
5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b)
6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768)
7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b)
8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b)
9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea)
10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea)
11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea)
12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea)
13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8)
14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8)
15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8)
16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8)
17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff)
18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff)
19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff)
20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff)
21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655)
22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655)
23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37)
24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2)
25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2)
26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2)
27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba)
28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b)
29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be)
30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093)
31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e)
32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9)
Mutex M2505 previously acquired by the same thread here:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689)
2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689)
3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f)
4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f)
5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37)
6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2)
7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2)
8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2)
9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba)
10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b)
11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be)
12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093)
13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e)
14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9)
[1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1
v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor)
v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_
v4: fix conflicts
2021-03-03 05:08:10 +00:00
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
/// TODO: Get rid of this.
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 ( ) ;
Fix one more lock-order-inversion
TSan report [1]:
WARNING: ThreadSanitizer: lock-order-inversion (potential deadlock) (pid=11314)
Cycle in lock order graph: M183938897938677368 (0x000000000000) => M2505 (0x7b9000002008) => M183938897938677368
Mutex M2505 acquired here while holding mutex M183938897938677368 in thread T6:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689)
2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689)
3 std::__1::unique_lock<std::__1::recursive_mutex>::unique_lock(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:119:61 (clickhouse-tsan+0x11e3506f)
4 DB::Context::getLock() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:517:12 (clickhouse-tsan+0x11e3506f)
5 DB::Context::getSchedulePool() const obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:1517:17 (clickhouse-tsan+0x11e3506f)
6 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:158:42 (clickhouse-tsan+0x12be1cda)
7 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e)
8 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba)
9 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba)
10 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba)
11 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba)
12 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba)
13 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba)
14 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0)
15 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0)
16 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0)
17 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600)
18 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600)
19 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600)
20 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600)
21 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600)
22 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561)
23 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561)
24 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561)
25 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561)
26 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205)
27 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205)
28 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205)
29 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8)
30 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8)
31 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8)
32 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8)
Mutex M183938897938677368 previously acquired by the same thread here:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9)
2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9)
3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be1ca9)
4 DB::IBackgroundJobExecutor::start() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:155:21 (clickhouse-tsan+0x12be1ca9)
5 DB::StorageMergeTree::startup() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:112:29 (clickhouse-tsan+0x129ed46e)
6 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_2::operator()(std::__1::shared_ptr<DB::IStorage> const&) const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:230:16 (clickhouse-tsan+0x11d71fba)
7 DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3::operator()() const obj-x86_64-linux-gnu/../src/Databases/DatabaseOrdinary.cpp:238:56 (clickhouse-tsan+0x11d71fba)
8 decltype(std::__1::forward<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(fp)()) std::__1::__invoke<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x11d71fba)
9 void std::__1::__invoke_void_return_wrapper<void>::__call<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&>(DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x11d71fba)
10 std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x11d71fba)
11 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<DB::DatabaseOrdinary::startupTables(ThreadPoolImpl<ThreadFromGlobalPool>&)::$_3, void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x11d71fba)
12 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b3b8e0)
13 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b3b8e0)
14 ThreadPoolImpl<ThreadFromGlobalPool>::worker(std::__1::__list_iterator<ThreadFromGlobalPool, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b3b8e0)
15 void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3e600)
16 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(fp0)...)) std::__1::__invoke_constexpr<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3682:1 (clickhouse-tsan+0x8b3e600)
17 decltype(auto) std::__1::__apply_tuple_impl<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::__tuple_indices<std::__1::tuple<>&...>) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1415:1 (clickhouse-tsan+0x8b3e600)
18 decltype(auto) std::__1::apply<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&, std::__1::tuple<>&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&) obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1424:1 (clickhouse-tsan+0x8b3e600)
19 ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()::operator()() obj-x86_64-linux-gnu/../src/Common/ThreadPool.h:178:13 (clickhouse-tsan+0x8b3e600)
20 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3e561)
21 void std::__1::__invoke_void_return_wrapper<void>::__call<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'()&>(void&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348:9 (clickhouse-tsan+0x8b3e561)
22 std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()>::operator()() obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1608:12 (clickhouse-tsan+0x8b3e561)
23 void std::__1::__function::__policy_invoker<void ()>::__call_impl<std::__1::__function::__default_alloc_func<ThreadFromGlobalPool::ThreadFromGlobalPool<void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<ThreadFromGlobalPool>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...)::'lambda'(), void ()> >(std::__1::__function::__policy_storage const*) obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2089:16 (clickhouse-tsan+0x8b3e561)
24 std::__1::__function::__policy_func<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2221:16 (clickhouse-tsan+0x8b39205)
25 std::__1::function<void ()>::operator()() const obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2560:12 (clickhouse-tsan+0x8b39205)
26 ThreadPoolImpl<std::__1::thread>::worker(std::__1::__list_iterator<std::__1::thread, void*>) obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:247:17 (clickhouse-tsan+0x8b39205)
27 void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()::operator()() const obj-x86_64-linux-gnu/../src/Common/ThreadPool.cpp:124:73 (clickhouse-tsan+0x8b3cea8)
28 decltype(std::__1::forward<void>(fp)(std::__1::forward<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(fp0)...)) std::__1::__invoke<void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(void&&, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()&&...) obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:3676:1 (clickhouse-tsan+0x8b3cea8)
29 void std::__1::__thread_execute<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>(std::__1::tuple<void, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()>&, std::__1::__tuple_indices<>) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:280:5 (clickhouse-tsan+0x8b3cea8)
30 void* std::__1::__thread_proxy<std::__1::tuple<std::__1::unique_ptr<std::__1::__thread_struct, std::__1::default_delete<std::__1::__thread_struct> >, void ThreadPoolImpl<std::__1::thread>::scheduleImpl<void>(std::__1::function<void ()>, int, std::__1::optional<unsigned long>)::'lambda1'()> >(void*) obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:291:5 (clickhouse-tsan+0x8b3cea8)
Mutex M183938897938677368 acquired here while holding mutex M2505 in main thread:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:405:10 (clickhouse-tsan+0x17cdb4f9)
2 std::__1::mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:33:14 (clickhouse-tsan+0x17cdb4f9)
3 std::__1::lock_guard<std::__1::mutex>::lock_guard(std::__1::mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x12be261b)
4 DB::IBackgroundJobExecutor::finish() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:167:21 (clickhouse-tsan+0x12be261b)
5 DB::IBackgroundJobExecutor::~IBackgroundJobExecutor() obj-x86_64-linux-gnu/../src/Storages/MergeTree/BackgroundJobsExecutor.cpp:185:5 (clickhouse-tsan+0x12be261b)
6 DB::StorageMergeTree::~StorageMergeTree() obj-x86_64-linux-gnu/../src/Storages/StorageMergeTree.cpp:174:1 (clickhouse-tsan+0x129ed768)
7 std::__1::default_delete<DB::StorageMergeTree>::operator()(DB::StorageMergeTree*) const obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:1397:5 (clickhouse-tsan+0x12e48b0b)
8 std::__1::__shared_ptr_pointer<DB::StorageMergeTree*, std::__1::shared_ptr<DB::StorageMergeTree>::__shared_ptr_default_delete<DB::StorageMergeTree, DB::StorageMergeTree>, std::__1::allocator<DB::StorageMergeTree> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2565:5 (clickhouse-tsan+0x12e48b0b)
9 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125b53ea)
10 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125b53ea)
11 std::__1::shared_ptr<DB::IStorage>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125b53ea)
12 DB::SystemLog<DB::AsynchronousMetricLogElement>::~SystemLog() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.h:118:7 (clickhouse-tsan+0x125b53ea)
13 std::__1::allocator<DB::AsynchronousMetricLog>::destroy(DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:891:15 (clickhouse-tsan+0x125b1dd8)
14 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::__destroy<DB::AsynchronousMetricLog>(std::__1::integral_constant<bool, true>, std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:539:21 (clickhouse-tsan+0x125b1dd8)
15 void std::__1::allocator_traits<std::__1::allocator<DB::AsynchronousMetricLog> >::destroy<DB::AsynchronousMetricLog>(std::__1::allocator<DB::AsynchronousMetricLog>&, DB::AsynchronousMetricLog*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__memory/allocator_traits.h:487:14 (clickhouse-tsan+0x125b1dd8)
16 std::__1::__shared_ptr_emplace<DB::AsynchronousMetricLog, std::__1::allocator<DB::AsynchronousMetricLog> >::__on_zero_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2611:9 (clickhouse-tsan+0x125b1dd8)
17 std::__1::__shared_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2475:9 (clickhouse-tsan+0x125904ff)
18 std::__1::__shared_weak_count::__release_shared() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2517:27 (clickhouse-tsan+0x125904ff)
19 std::__1::shared_ptr<DB::AsynchronousMetricLog>::~shared_ptr() obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3212:19 (clickhouse-tsan+0x125904ff)
20 DB::SystemLogs::~SystemLogs() obj-x86_64-linux-gnu/../src/Interpreters/SystemLog.cpp:155:1 (clickhouse-tsan+0x125904ff)
21 std::__1::__optional_destruct_base<DB::SystemLogs, false>::reset() obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:245:21 (clickhouse-tsan+0x11e43655)
22 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:443:21 (clickhouse-tsan+0x11e43655)
23 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37)
24 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2)
25 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2)
26 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2)
27 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba)
28 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b)
29 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be)
30 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093)
31 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e)
32 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9)
Mutex M2505 previously acquired by the same thread here:
0 pthread_mutex_lock <null> (clickhouse-tsan+0x8a327b6)
1 std::__1::__libcpp_recursive_mutex_lock(pthread_mutex_t*) obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:385:10 (clickhouse-tsan+0x17cdb689)
2 std::__1::recursive_mutex::lock() obj-x86_64-linux-gnu/../contrib/libcxx/src/mutex.cpp:71:14 (clickhouse-tsan+0x17cdb689)
3 std::__1::lock_guard<std::__1::recursive_mutex>::lock_guard(std::__1::recursive_mutex&) obj-x86_64-linux-gnu/../contrib/libcxx/include/__mutex_base:91:27 (clickhouse-tsan+0x11e4363f)
4 DB::ContextShared::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:438:21 (clickhouse-tsan+0x11e4363f)
5 DB::Context::shutdown() obj-x86_64-linux-gnu/../src/Interpreters/Context.cpp:2251:13 (clickhouse-tsan+0x11e3be37)
6 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5::operator()() const obj-x86_64-linux-gnu/../programs/server/Server.cpp:892:5 (clickhouse-tsan+0x8abacc2)
7 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::invoke() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:97:9 (clickhouse-tsan+0x8abacc2)
8 ext::basic_scope_guard<DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&)::$_5>::~basic_scope_guard() obj-x86_64-linux-gnu/../base/common/../ext/scope_guard.h:47:28 (clickhouse-tsan+0x8abacc2)
9 DB::Server::main(std::__1::vector<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> >, std::__1::allocator<std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > > > const&) obj-x86_64-linux-gnu/../programs/server/Server.cpp:1395:1 (clickhouse-tsan+0x8ab5cba)
10 Poco::Util::Application::run() obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:334:8 (clickhouse-tsan+0x15b4ac7b)
11 DB::Server::run() obj-x86_64-linux-gnu/../programs/server/Server.cpp:342:25 (clickhouse-tsan+0x8aa97be)
12 Poco::Util::ServerApplication::run(int, char**) obj-x86_64-linux-gnu/../contrib/poco/Util/src/ServerApplication.cpp:611:9 (clickhouse-tsan+0x15b67093)
13 mainEntryClickHouseServer(int, char**) obj-x86_64-linux-gnu/../programs/server/Server.cpp:134:20 (clickhouse-tsan+0x8aa838e)
14 main obj-x86_64-linux-gnu/../programs/main.cpp:368:12 (clickhouse-tsan+0x8aa68f9)
[1]: https://clickhouse-test-reports.s3.yandex.net/21318/f3b1ad0f5d1024275674e1beac24251ae97c8453/functional_stateful_tests_(thread).html#fail1
v2: Convert ContextSharedPart::system_logs to std::unique_ptr (to avoid copy ctor)
v3: Fix readability-identifier-naming,-warnings-as-errors for system_logs_
v4: fix conflicts
2021-03-03 05:08:10 +00:00
delete_system_logs = std : : move ( system_logs ) ;
embedded_dictionaries . reset ( ) ;
external_dictionaries_loader . reset ( ) ;
models_repository_guard . reset ( ) ;
external_models_loader . reset ( ) ;
buffer_flush_schedule_pool . reset ( ) ;
schedule_pool . reset ( ) ;
distributed_schedule_pool . reset ( ) ;
message_broker_schedule_pool . reset ( ) ;
ddl_worker . reset ( ) ;
/// Stop trace collector if any
trace_collector . reset ( ) ;
/// Stop zookeeper connection
zookeeper . reset ( ) ;
}
/// Can be removed w/o context lock
delete_system_logs . reset ( ) ;
2017-04-01 07:20:54 +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
}
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 ;
2020-04-19 15:46:12 +00:00
SharedContextHolder & SharedContextHolder : : operator = ( SharedContextHolder & & ) = default ;
2020-04-19 15:41:35 +00:00
SharedContextHolder : : SharedContextHolder ( ) = default ;
2020-04-16 14:51:33 +00:00
SharedContextHolder : : ~ SharedContextHolder ( ) = default ;
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-04-10 23:33:54 +00:00
DatabaseCatalog : : init ( shared_from_this ( ) ) ;
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
{
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 ( ) ;
if ( ! ptr ) throw Exception ( " Can't copy an expired context " , ErrorCodes : : LOGICAL_ERROR ) ;
return createCopy ( ptr ) ;
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context : : createCopy ( const ContextMutablePtr & other )
2021-04-10 23:33:54 +00:00
{
return createCopy ( std : : const_pointer_cast < const Context > ( other ) ) ;
}
void Context : : copyFrom ( const ContextPtr & other )
{
* this = * other ;
2020-04-16 14:37:38 +00:00
}
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 ; }
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
{
2017-04-01 07:20:54 +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 ; }
MergeList & Context : : getMergeList ( ) { return shared - > merge_list ; }
const MergeList & Context : : getMergeList ( ) const { return shared - > merge_list ; }
2020-10-26 16:38:35 +00:00
ReplicatedFetchList & Context : : getReplicatedFetchList ( ) { return shared - > replicated_fetch_list ; }
const ReplicatedFetchList & Context : : getReplicatedFetchList ( ) const { return shared - > replicated_fetch_list ; }
2015-04-16 06:12:35 +00:00
2020-03-05 19:23:39 +00:00
void Context : : enableNamedSessions ( )
2016-03-19 01:18:49 +00:00
{
2020-03-05 19:23:39 +00:00
shared - > named_sessions . emplace ( ) ;
2016-03-19 01:18:49 +00:00
}
2021-04-10 23:33:54 +00:00
std : : shared_ptr < NamedSession >
Context : : acquireNamedSession ( const String & session_id , std : : chrono : : steady_clock : : duration timeout , bool session_check )
2016-03-19 01:18:49 +00:00
{
2020-03-05 19:23:39 +00:00
if ( ! shared - > named_sessions )
throw Exception ( " Support for named sessions is not enabled " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2017-01-05 16:34:05 +00:00
2021-04-10 23:33:54 +00:00
return shared - > named_sessions - > acquireSession ( session_id , shared_from_this ( ) , timeout , session_check ) ;
2017-06-02 18:48:33 +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-04-01 07:20:54 +00:00
if ( res . empty ( ) )
throw Exception ( " Default database is not selected " , ErrorCodes : : UNKNOWN_DATABASE ) ;
return res ;
2017-01-05 16:34:05 +00:00
}
2012-08-02 17:33:31 +00:00
String Context : : getPath ( ) const
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
return shared - > path ;
2012-08-02 17:33:31 +00:00
}
2016-10-24 14:01:24 +00:00
String Context : : getFlagsPath ( ) const
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
return shared - > flags_path ;
2016-10-24 14:01:24 +00:00
}
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-07-09 09:27:51 +00:00
std : : vector < String > Context : : getWarnings ( ) const
{
auto lock = getLock ( ) ;
2021-07-13 00:59:52 +00:00
return shared - > warnings ;
2021-07-09 09:27:51 +00:00
}
2020-07-08 14:25:23 +00:00
VolumePtr Context : : getTemporaryVolume ( ) const
2020-01-19 14:26:28 +00:00
{
auto lock = getLock ( ) ;
return shared - > tmp_volume ;
}
2012-08-02 17:33:31 +00:00
void Context : : setPath ( const String & path )
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
2018-04-19 05:32:09 +00:00
2017-04-01 07:20:54 +00:00
shared - > path = path ;
2018-04-19 05:32:09 +00:00
2020-01-19 14:26:28 +00:00
if ( shared - > tmp_path . empty ( ) & & ! shared - > tmp_volume )
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/ " ;
2012-08-02 17:33:31 +00:00
}
2020-07-08 14:25:23 +00:00
VolumePtr Context : : setTemporaryStorage ( const String & path , const String & policy_name )
2015-01-07 17:19:23 +00:00
{
2020-05-21 14:11:56 +00:00
std : : lock_guard lock ( shared - > storage_policies_mutex ) ;
2020-01-19 14:26:28 +00:00
if ( policy_name . empty ( ) )
{
shared - > tmp_path = path ;
if ( ! shared - > tmp_path . ends_with ( ' / ' ) )
shared - > tmp_path + = ' / ' ;
auto disk = std : : make_shared < DiskLocal > ( " _tmp_default " , shared - > tmp_path , 0 ) ;
2020-10-20 15:10:24 +00:00
shared - > tmp_volume = std : : make_shared < SingleDiskVolume > ( " _tmp_default " , disk , 0 ) ;
2020-01-19 14:26:28 +00:00
}
else
{
2020-05-21 14:11:56 +00:00
StoragePolicyPtr tmp_policy = getStoragePolicySelector ( lock ) - > get ( policy_name ) ;
2020-01-19 14:26:28 +00:00
if ( tmp_policy - > getVolumes ( ) . size ( ) ! = 1 )
2020-11-06 18:02:16 +00:00
throw Exception ( " Policy " + policy_name + " is used temporary files, such policy should have exactly one volume " ,
ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
2020-01-19 14:26:28 +00:00
shared - > tmp_volume = tmp_policy - > getVolume ( 0 ) ;
}
2020-05-09 21:24:15 +00:00
if ( shared - > tmp_volume - > getDisks ( ) . empty ( ) )
2020-01-19 14:26:28 +00:00
throw Exception ( " No disks volume for temporary files " , ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
return shared - > tmp_volume ;
2015-01-07 17:19:23 +00:00
}
2016-10-24 14:01:24 +00:00
void Context : : setFlagsPath ( const String & path )
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
shared - > flags_path = path ;
2016-10-24 14:01:24 +00:00
}
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-07-09 14:40:32 +00:00
void Context : : addWarningMessage ( const String & msg )
{
auto lock = getLock ( ) ;
shared - > addWarningMessage ( msg ) ;
}
2017-08-24 14:51:13 +00:00
void Context : : setConfig ( const ConfigurationPtr & config )
{
auto lock = getLock ( ) ;
shared - > config = config ;
2020-06-10 22:48:15 +00:00
shared - > access_control_manager . 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
2019-11-09 15:33:07 +00:00
AccessControlManager & Context : : getAccessControlManager ( )
{
return shared - > access_control_manager ;
}
const AccessControlManager & Context : : getAccessControlManager ( ) const
{
return shared - > access_control_manager ;
}
2020-06-10 22:48:15 +00:00
void Context : : setExternalAuthenticatorsConfig ( const Poco : : Util : : AbstractConfiguration & config )
{
auto lock = getLock ( ) ;
shared - > access_control_manager . setExternalAuthenticatorsConfig ( config ) ;
}
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 ( ) ;
return std : : make_unique < GSSAcceptorContext > ( shared - > access_control_manager . getExternalAuthenticators ( ) . getKerberosParams ( ) ) ;
}
2016-10-14 15:06:46 +00:00
void Context : : setUsersConfig ( const ConfigurationPtr & config )
2013-08-10 07:46:45 +00:00
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
shared - > users_config = config ;
2020-02-21 19:27:12 +00:00
shared - > access_control_manager . setUsersConfig ( * shared - > users_config ) ;
2014-02-13 07:17:22 +00:00
}
ConfigurationPtr Context : : getUsersConfig ( )
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
return shared - > users_config ;
2013-08-10 07:46:45 +00:00
}
2020-02-12 03:03:33 +00:00
2021-03-11 20:41:10 +00:00
void Context : : setUser ( const Credentials & credentials , const Poco : : Net : : SocketAddress & address )
2013-08-10 07:46:45 +00:00
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
2013-08-10 07:46:45 +00:00
2021-03-11 20:41:10 +00:00
client_info . current_user = credentials . getUserName ( ) ;
2020-02-12 03:03:33 +00:00
client_info . current_address = address ;
2017-03-23 14:14:56 +00:00
2020-06-15 02:17:08 +00:00
# if defined(ARCADIA_BUILD)
/// This is harmful field that is used only in foreign "Arcadia" build.
2021-03-11 20:41:10 +00:00
client_info . current_password . clear ( ) ;
if ( const auto * basic_credentials = dynamic_cast < const BasicCredentials * > ( & credentials ) )
client_info . current_password = basic_credentials - > getPassword ( ) ;
2020-06-15 02:17:08 +00:00
# endif
2021-03-11 20:41:10 +00:00
/// Find a user with such name and check the credentials.
auto new_user_id = getAccessControlManager ( ) . login ( credentials , address . host ( ) ) ;
2020-09-17 09:58:34 +00:00
auto new_access = getAccessControlManager ( ) . getContextAccess (
new_user_id , /* current_roles = */ { } , /* use_default_roles = */ true ,
settings , current_database , client_info ) ;
2017-06-05 17:54:40 +00:00
2020-02-12 03:03:33 +00:00
user_id = new_user_id ;
2020-03-07 17:37:38 +00:00
access = std : : move ( new_access ) ;
2020-02-21 00:17:07 +00:00
current_roles . clear ( ) ;
use_default_roles = true ;
2019-11-04 19:17:27 +00:00
2021-07-22 16:07:03 +00:00
auto default_profile_info = access - > getDefaultProfileInfo ( ) ;
settings_constraints_and_current_profiles = default_profile_info - > getConstraintsAndProfileIDs ( ) ;
applySettingsChanges ( default_profile_info - > settings ) ;
2019-04-19 00:45:15 +00:00
}
2020-09-14 21:55:43 +00:00
void Context : : setUser ( const String & name , const String & password , const Poco : : Net : : SocketAddress & address )
{
2021-03-11 20:41:10 +00:00
setUser ( BasicCredentials ( name , password ) , address ) ;
2020-09-14 21:55:43 +00:00
}
void Context : : setUserWithoutCheckingPassword ( const String & name , const Poco : : Net : : SocketAddress & address )
{
2021-03-11 20:41:10 +00:00
setUser ( AlwaysAllowCredentials ( name ) , address ) ;
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-04-15 01:58:10 +00:00
void Context : : setQuotaKey ( String quota_key_ )
{
auto lock = getLock ( ) ;
client_info . quota_key = std : : move ( quota_key_ ) ;
}
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
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-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 ( ) ;
2020-02-21 00:17:07 +00:00
if ( current_roles = = current_roles_ & & ! use_default_roles )
return ;
current_roles = current_roles_ ;
use_default_roles = false ;
calculateAccessRights ( ) ;
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
{
auto lock = getLock ( ) ;
2020-02-21 00:17:07 +00:00
if ( use_default_roles )
return ;
current_roles . clear ( ) ;
use_default_roles = true ;
calculateAccessRights ( ) ;
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-02-12 03:03:33 +00:00
void Context : : calculateAccessRights ( )
2019-05-09 06:12:02 +00:00
{
auto lock = getLock ( ) ;
2020-02-12 03:03:33 +00:00
if ( user_id )
2020-03-07 17:37:38 +00:00
access = getAccessControlManager ( ) . getContextAccess ( * user_id , current_roles , use_default_roles , settings , current_database , client_info ) ;
2019-05-09 06:12:02 +00:00
}
2014-12-23 20:32:00 +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 ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const std : : string_view & database ) const { return checkAccessImpl ( flags , database ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const std : : string_view & database , const std : : string_view & table ) const { return checkAccessImpl ( flags , database , table ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const std : : string_view & database , const std : : string_view & table , const std : : string_view & column ) const { return checkAccessImpl ( flags , database , table , column ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const std : : string_view & database , const std : : string_view & table , const std : : vector < std : : string_view > & columns ) const { return checkAccessImpl ( flags , database , table , columns ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const std : : string_view & database , const std : : string_view & table , const Strings & columns ) const { return checkAccessImpl ( flags , database , table , columns ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const StorageID & table_id ) const { checkAccessImpl ( flags , table_id . getDatabaseName ( ) , table_id . getTableName ( ) ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const StorageID & table_id , const std : : string_view & column ) const { checkAccessImpl ( flags , table_id . getDatabaseName ( ) , table_id . getTableName ( ) , column ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const StorageID & table_id , const std : : vector < std : : string_view > & columns ) const { checkAccessImpl ( flags , table_id . getDatabaseName ( ) , table_id . getTableName ( ) , columns ) ; }
void Context : : checkAccess ( const AccessFlags & flags , const StorageID & table_id , const Strings & columns ) const { checkAccessImpl ( flags , table_id . getDatabaseName ( ) , table_id . getTableName ( ) , columns ) ; }
void Context : : checkAccess ( const AccessRightsElement & element ) const { return checkAccessImpl ( element ) ; }
void Context : : checkAccess ( const AccessRightsElements & elements ) const { return checkAccessImpl ( elements ) ; }
2014-12-23 20:32:00 +00:00
2013-08-12 00:36:18 +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
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
2020-03-07 17:37:38 +00:00
return access ? access : ContextAccess : : getFullAccess ( ) ;
2020-02-07 19:31:09 +00:00
}
2012-08-02 17:33:31 +00:00
2020-03-07 17:37:38 +00:00
ASTPtr Context : : getRowPolicyCondition ( const String & database , const String & table_name , RowPolicy : : ConditionType type ) const
2020-02-12 03:03:33 +00:00
{
2020-03-07 17:37:38 +00:00
auto lock = getLock ( ) ;
auto initial_condition = initial_row_policy ? initial_row_policy - > getCondition ( database , table_name , type ) : nullptr ;
return getAccess ( ) - > getRowPolicyCondition ( database , table_name , type , initial_condition ) ;
2012-08-02 17:33:31 +00:00
}
2020-03-07 17:37:38 +00:00
void Context : : setInitialRowPolicy ( )
2012-08-02 17:33:31 +00:00
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
2020-03-07 17:37:38 +00:00
auto initial_user_id = getAccessControlManager ( ) . find < User > ( client_info . initial_user ) ;
initial_row_policy = nullptr ;
if ( initial_user_id )
initial_row_policy = getAccessControlManager ( ) . getEnabledRowPolicies ( * initial_user_id , { } ) ;
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 ( ) ;
}
2021-07-22 16:07:03 +00:00
void Context : : setCurrentProfile ( const String & profile_name )
2012-08-02 17:33:31 +00:00
{
2021-07-22 16:07:03 +00:00
auto lock = getLock ( ) ;
2020-12-24 17:48:54 +00:00
try
{
2021-07-22 16:07:03 +00:00
UUID profile_id = getAccessControlManager ( ) . getID < SettingsProfile > ( profile_name ) ;
setCurrentProfile ( profile_id ) ;
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
}
void Context : : setCurrentProfile ( const UUID & profile_id )
{
auto lock = getLock ( ) ;
auto profile_info = getAccessControlManager ( ) . getSettingsProfileInfo ( profile_id ) ;
checkSettingsConstraints ( profile_info - > settings ) ;
applySettingsChanges ( profile_info - > settings ) ;
settings_constraints_and_current_profiles = profile_info - > getConstraintsAndProfileIDs ( settings_constraints_and_current_profiles ) ;
}
std : : vector < UUID > Context : : getCurrentProfiles ( ) const
{
auto lock = getLock ( ) ;
return settings_constraints_and_current_profiles - > current_profiles ;
}
std : : vector < UUID > Context : : getEnabledProfiles ( ) const
{
auto lock = getLock ( ) ;
return settings_constraints_and_current_profiles - > enabled_profiles ;
2012-08-02 17:33:31 +00:00
}
2019-10-19 20:36:35 +00:00
const Scalars & Context : : getScalars ( ) const
{
return scalars ;
}
const Block & Context : : getScalar ( const String & name ) const
{
auto it = scalars . find ( name ) ;
if ( scalars . end ( ) = = it )
2020-06-06 01:23:21 +00:00
{
// This should be a logical error, but it fails the sql_fuzz test too
// often, so 'bad arguments' for now.
throw Exception ( " Scalar " + backQuoteIfNeed ( name ) + " doesn't exist (internal bug) " , ErrorCodes : : BAD_ARGUMENTS ) ;
}
2019-10-19 20:36:35 +00:00
return it - > second ;
}
2014-03-13 15:00:06 +00:00
Tables Context : : getExternalTables ( ) const
{
2021-04-10 23:33:54 +00:00
assert ( ! isGlobalContext ( ) | | getApplicationType ( ) = = ApplicationType : : LOCAL ) ;
2017-04-01 07:20:54 +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 )
2017-04-01 07:20:54 +00:00
{
2021-04-10 23:33:54 +00:00
Tables buf = query_context_ptr - > getExternalTables ( ) ;
2017-04-01 07:20:54 +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 )
2017-04-01 07:20:54 +00:00
{
2021-04-10 23:33:54 +00:00
Tables buf = session_context_ptr - > getExternalTables ( ) ;
2017-04-01 07:20:54 +00:00
res . insert ( buf . begin ( ) , buf . end ( ) ) ;
}
return res ;
2014-03-13 15:00:06 +00:00
}
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-04-10 23:33:54 +00:00
assert ( ! isGlobalContext ( ) | | getApplicationType ( ) = = ApplicationType : : LOCAL ) ;
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 ) )
2018-04-03 21:29:11 +00:00
throw Exception ( " Temporary table " + backQuoteIfNeed ( table_name ) + " already exists. " , ErrorCodes : : TABLE_ALREADY_EXISTS ) ;
2020-03-10 19:36:17 +00:00
external_tables_mapping . emplace ( table_name , std : : make_shared < TemporaryTableHolder > ( std : : move ( temporary_table ) ) ) ;
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-04-10 23:33:54 +00:00
assert ( ! isGlobalContext ( ) | | getApplicationType ( ) = = ApplicationType : : LOCAL ) ;
2020-02-17 13:52:59 +00:00
std : : shared_ptr < TemporaryTableHolder > holder ;
2017-04-01 07:20:54 +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 ) ;
2017-04-01 07:20:54 +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-04-10 23:33:54 +00:00
assert ( ! isGlobalContext ( ) | | getApplicationType ( ) = = ApplicationType : : LOCAL ) ;
2019-10-19 20:36:35 +00:00
scalars [ name ] = block ;
}
bool Context : : hasScalar ( const String & name ) const
{
2021-04-10 23:33:54 +00:00
assert ( ! isGlobalContext ( ) | | getApplicationType ( ) = = ApplicationType : : LOCAL ) ;
2019-10-19 20:36:35 +00:00
return scalars . count ( name ) ;
}
2021-02-10 14:12:49 +00:00
void Context : : addQueryAccessInfo (
const String & quoted_database_name , const String & full_quoted_table_name , const Names & column_names , const String & projection_name )
2020-12-02 12:08:03 +00:00
{
2021-04-10 23:33:54 +00:00
assert ( ! isGlobalContext ( ) | | getApplicationType ( ) = = ApplicationType : : LOCAL ) ;
2021-01-20 12:30:41 +00:00
std : : lock_guard < std : : mutex > lock ( query_access_info . mutex ) ;
2020-12-14 03:30:39 +00:00
query_access_info . databases . emplace ( quoted_database_name ) ;
query_access_info . tables . emplace ( full_quoted_table_name ) ;
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 ) ) ;
2020-12-02 12:08:03 +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-04-10 23:33:54 +00:00
assert ( ! isGlobalContext ( ) | | getApplicationType ( ) = = ApplicationType : : LOCAL ) ;
2021-01-20 10:54:11 +00:00
auto lock = getLock ( ) ;
2021-01-21 21:15:11 +00:00
switch ( factory_type )
{
case QueryLogFactories : : AggregateFunction :
query_factories_info . aggregate_functions . emplace ( created_object ) ;
break ;
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
}
2018-03-01 01:49:36 +00:00
StoragePtr Context : : executeTableFunction ( const ASTPtr & table_expression )
{
/// Slightly suboptimal.
auto hash = table_expression - > getTreeHash ( ) ;
String key = toString ( hash . first ) + ' _ ' + toString ( hash . second ) ;
StoragePtr & res = table_function_results [ key ] ;
if ( ! res )
{
2021-04-10 23:33:54 +00:00
TableFunctionPtr table_function_ptr = TableFunctionFactory : : instance ( ) . get ( table_expression , shared_from_this ( ) ) ;
2018-03-01 01:49:36 +00:00
/// Run it and remember the result
2021-04-10 23:33:54 +00:00
res = table_function_ptr - > execute ( table_expression , shared_from_this ( ) , table_function_ptr - > getName ( ) ) ;
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
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 )
throw Exception (
" Temporary view source storage " + backQuoteIfNeed ( view_source - > getName ( ) ) + " already exists. " , ErrorCodes : : TABLE_ALREADY_EXISTS ) ;
view_source = storage ;
}
2021-05-31 14:49:02 +00:00
StoragePtr Context : : getViewSource ( ) const
2018-12-09 17:50:35 +00:00
{
return view_source ;
}
2012-08-02 17:33:31 +00:00
Settings Context : : getSettings ( ) const
{
2020-09-24 17:08:15 +00:00
auto lock = getLock ( ) ;
2017-04-01 07:20:54 +00:00
return settings ;
2012-08-02 17:33:31 +00:00
}
void Context : : setSettings ( const Settings & settings_ )
{
2020-01-12 21:00:55 +00:00
auto lock = getLock ( ) ;
2020-03-13 14:50:26 +00:00
auto old_readonly = settings . readonly ;
auto old_allow_ddl = settings . allow_ddl ;
auto old_allow_introspection_functions = settings . allow_introspection_functions ;
2020-01-12 21:00:55 +00:00
2017-04-01 07:20:54 +00:00
settings = settings_ ;
2020-01-12 21:00:55 +00:00
if ( ( settings . readonly ! = old_readonly ) | | ( settings . allow_ddl ! = old_allow_ddl ) | | ( settings . allow_introspection_functions ! = old_allow_introspection_functions ) )
calculateAccessRights ( ) ;
2012-08-02 17:33:31 +00:00
}
2020-03-15 13:52:46 +00:00
void Context : : setSetting ( const StringRef & name , const String & value )
2012-08-02 19:03:32 +00:00
{
2019-04-18 23:29:32 +00:00
auto lock = getLock ( ) ;
2017-04-01 07:20:54 +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
}
2020-07-20 09:57:17 +00:00
settings . set ( std : : string_view { name } , value ) ;
2020-01-12 21:00:55 +00:00
if ( name = = " readonly " | | name = = " allow_ddl " | | name = = " allow_introspection_functions " )
calculateAccessRights ( ) ;
2014-02-13 07:17:22 +00:00
}
2020-03-15 13:52:46 +00:00
void Context : : setSetting ( const StringRef & name , const Field & value )
2014-02-13 07:17:22 +00:00
{
2019-04-18 23:29:32 +00:00
auto lock = getLock ( ) ;
2017-04-01 07:20:54 +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
}
2020-07-20 09:57:17 +00:00
settings . set ( std : : string_view { name } , value ) ;
2020-01-12 21:00:55 +00:00
if ( name = = " readonly " | | name = = " allow_ddl " | | name = = " allow_introspection_functions " )
calculateAccessRights ( ) ;
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 )
{
e . addMessage ( fmt : : format ( " in attempt to set the value of setting '{}' to {} " ,
change . name , applyVisitor ( FieldVisitorToString ( ) , change . value ) ) ) ;
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
}
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
}
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 ;
static auto no_constraints_or_profiles = std : : make_shared < SettingsConstraintsAndProfileIDs > ( getAccessControlManager ( ) ) ;
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 ( ) ;
2017-04-01 07:20:54 +00:00
return current_database ;
2012-08-02 17:33:31 +00:00
}
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 ( ) )
2020-11-06 18:02:16 +00:00
throw Exception ( " Cannot set current database for non global context, this method should be used during server initialization " ,
ErrorCodes : : LOGICAL_ERROR ) ;
2020-09-25 11:27:00 +00:00
auto lock = getLock ( ) ;
if ( ! current_database . empty ( ) )
2020-11-06 18:02:16 +00:00
throw Exception ( " Default database name cannot be changed in global context without server restart " ,
ErrorCodes : : LOGICAL_ERROR ) ;
2020-09-25 11:27:00 +00:00
current_database = name ;
}
2012-08-02 17:33:31 +00:00
void Context : : setCurrentDatabase ( const String & name )
{
2020-02-10 13:10:17 +00:00
DatabaseCatalog : : instance ( ) . assertDatabaseExists ( name ) ;
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
current_database = name ;
2020-06-24 14:31:05 +00:00
calculateAccessRights ( ) ;
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
2020-08-20 20:59:40 +00:00
random . words . a = thread_local_rng ( ) ; //-V656
random . words . b = thread_local_rng ( ) ; //-V656
2020-08-28 19:02:50 +00:00
2021-05-04 22:42:14 +00:00
if ( client_info . client_trace_context . trace_id ! = UUID ( ) )
2020-11-18 17:43:18 +00:00
{
// Use the OpenTelemetry trace context we received from the client, and
// create a new span for the query.
query_trace_context = client_info . client_trace_context ;
query_trace_context . span_id = thread_local_rng ( ) ;
}
else if ( client_info . query_kind = = ClientInfo : : QueryKind : : INITIAL_QUERY )
2017-05-24 19:31:50 +00:00
{
2020-09-08 13:19:27 +00:00
// If this is an initial query without any parent OpenTelemetry trace, we
// might start the trace ourselves, with some configurable probability.
std : : bernoulli_distribution should_start_trace {
settings . opentelemetry_start_trace_probability } ;
2017-05-24 19:31:50 +00:00
2020-09-08 13:19:27 +00:00
if ( should_start_trace ( thread_local_rng ) )
2017-05-24 19:31:50 +00:00
{
2020-09-08 13:19:27 +00:00
// Use the randomly generated default query id as the new trace id.
2020-11-18 17:43:18 +00:00
query_trace_context . trace_id = random . uuid ;
query_trace_context . span_id = thread_local_rng ( ) ;
2020-09-08 13:19:27 +00:00
// Mark this trace as sampled in the flags.
2020-11-18 17:43:18 +00:00
query_trace_context . trace_flags = 1 ;
2020-09-08 13:19:27 +00:00
}
2020-08-28 01:21:08 +00:00
}
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
2017-04-01 07:20:54 +00:00
client_info . current_query_id = query_id_to_set ;
2014-02-12 17:31:02 +00:00
}
2019-02-01 01:48:25 +00:00
void Context : : killCurrentQuery ( )
{
if ( process_list_elem )
{
process_list_elem - > cancelQuery ( true ) ;
}
} ;
2014-02-12 17:31:02 +00:00
2013-06-29 18:03:57 +00:00
String Context : : getDefaultFormat ( ) const
{
2017-04-01 07:20:54 +00:00
return default_format . empty ( ) ? " TabSeparated " : default_format ;
2013-06-29 18:03:57 +00:00
}
void Context : : setDefaultFormat ( const String & name )
{
2017-04-01 07:20:54 +00:00
default_format = name ;
2013-06-29 18:03:57 +00:00
}
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 ( ) ;
if ( ! ptr ) throw Exception ( " There is no query or query context has expired " , ErrorCodes : : THERE_IS_NO_QUERY ) ;
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 ( ) ;
if ( ! ptr ) throw Exception ( " There is no session or session context has expired " , ErrorCodes : : THERE_IS_NO_SESSION ) ;
return ptr ;
2016-11-16 11:29:51 +00:00
}
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 ( ) ;
if ( ! ptr ) throw Exception ( " There is no global context or global context has expired " , ErrorCodes : : LOGICAL_ERROR ) ;
return ptr ;
2012-08-02 17:33:31 +00:00
}
2021-05-31 14:49:02 +00:00
ContextMutablePtr Context : : getBufferContext ( ) const
2016-11-15 10:57:11 +00:00
{
2021-04-10 23:33:54 +00:00
if ( ! buffer_context ) throw Exception ( " There is no buffer context " , ErrorCodes : : LOGICAL_ERROR ) ;
return buffer_context ;
2021-01-27 18:05:18 +00:00
}
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-04-01 07:20:54 +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 ) ;
if ( ! shared - > external_dictionaries_loader )
shared - > external_dictionaries_loader . emplace ( getGlobalContext ( ) ) ;
return * shared - > external_dictionaries_loader ;
2019-06-02 12:11:01 +00:00
}
2019-09-26 10:23:14 +00:00
const ExternalModelsLoader & Context : : getExternalModelsLoader ( ) const
2017-10-17 10:44:46 +00:00
{
2021-04-10 23:33:54 +00:00
return const_cast < Context * > ( this ) - > getExternalModelsLoader ( ) ;
2017-10-17 10:44:46 +00:00
}
2015-04-02 16:30:18 +00:00
2019-09-26 10:23:14 +00:00
ExternalModelsLoader & Context : : getExternalModelsLoader ( )
2015-04-02 16:30:18 +00:00
{
2021-04-10 23:33:54 +00:00
std : : lock_guard lock ( shared - > external_models_mutex ) ;
2021-04-12 09:28:46 +00:00
return getExternalModelsLoaderUnlocked ( ) ;
}
ExternalModelsLoader & Context : : getExternalModelsLoaderUnlocked ( )
{
2021-04-10 23:33:54 +00:00
if ( ! shared - > external_models_loader )
shared - > external_models_loader . emplace ( getGlobalContext ( ) ) ;
2021-04-12 09:28:46 +00:00
return * shared - > external_models_loader ;
2015-04-02 16:30:18 +00:00
}
2021-04-01 10:19:28 +00:00
void Context : : setExternalModelsConfig ( const ConfigurationPtr & config , const std : : string & config_name )
{
std : : lock_guard lock ( shared - > external_models_mutex ) ;
2021-04-02 10:17:28 +00:00
if ( shared - > external_models_config & & isSameConfigurationWithMultipleKeys ( * config , * shared - > external_models_config , " " , config_name ) )
2021-04-01 10:19:28 +00:00
return ;
shared - > external_models_config = config ;
2021-04-02 10:17:28 +00:00
shared - > models_repository_guard . reset ( ) ;
2021-04-01 10:19:28 +00:00
shared - > models_repository_guard = getExternalModelsLoaderUnlocked ( ) . addConfigRepository (
std : : make_unique < ExternalLoaderXMLConfigRepository > ( * config , config_name ) ) ;
}
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
shared - > embedded_dictionaries . emplace (
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
2019-06-02 12:11:01 +00:00
void Context : : tryCreateEmbeddedDictionaries ( ) const
2017-10-17 10:44:46 +00:00
{
2019-06-02 12:11:01 +00:00
static_cast < void > ( getEmbeddedDictionariesImpl ( true ) ) ;
2017-10-17 10:44:46 +00:00
}
2021-05-12 18:16:01 +00:00
void Context : : loadDictionaries ( const Poco : : Util : : AbstractConfiguration & config )
{
if ( ! config . getBool ( " dictionaries_lazy_load " , true ) )
{
tryCreateEmbeddedDictionaries ( ) ;
getExternalDictionariesLoader ( ) . enableAlwaysLoadEverything ( true ) ;
}
shared - > dictionaries_xmls = getExternalDictionariesLoader ( ) . addConfigRepository (
std : : make_unique < ExternalLoaderXMLConfigRepository > ( config , " dictionaries_config " ) ) ;
}
2017-10-17 10:44:46 +00:00
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.
2017-04-01 07:20:54 +00:00
progress_callback = callback ;
2013-02-16 14:55:14 +00:00
}
ProgressCallback Context : : getProgressCallback ( ) const
{
2017-04-01 07:20:54 +00:00
return progress_callback ;
2013-02-16 14:55:14 +00:00
}
2013-09-03 20:21:28 +00:00
2013-11-03 05:32:42 +00:00
void Context : : setProcessListElement ( ProcessList : : Element * elem )
{
2017-04-02 17:37:49 +00:00
/// Set to a session or query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
2017-04-01 07:20:54 +00:00
process_list_elem = elem ;
2013-11-03 05:32:42 +00:00
}
2017-09-04 17:49:39 +00:00
ProcessList : : Element * Context : : getProcessListElement ( ) const
2017-08-29 13:23:04 +00:00
{
return process_list_elem ;
}
2013-11-03 05:32:42 +00:00
2014-03-28 14:36:24 +00:00
void Context : : setUncompressedCache ( size_t max_size_in_bytes )
2013-09-08 05:53:10 +00:00
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
2013-09-08 05:53:10 +00:00
2017-04-01 07:20:54 +00:00
if ( shared - > uncompressed_cache )
throw Exception ( " Uncompressed cache has been already created. " , ErrorCodes : : LOGICAL_ERROR ) ;
2013-09-08 05:53:10 +00:00
2017-04-01 07:20:54 +00:00
shared - > uncompressed_cache = std : : make_shared < UncompressedCache > ( max_size_in_bytes ) ;
2013-09-08 05:53:10 +00:00
}
UncompressedCachePtr Context : : getUncompressedCache ( ) const
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
return shared - > uncompressed_cache ;
2013-09-08 05:53:10 +00:00
}
2017-08-07 17:01:04 +00:00
void Context : : dropUncompressedCache ( ) const
{
auto lock = getLock ( ) ;
if ( shared - > uncompressed_cache )
shared - > uncompressed_cache - > reset ( ) ;
}
2014-02-11 13:30:42 +00:00
void Context : : setMarkCache ( size_t cache_size_in_bytes )
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
2014-02-11 13:30:42 +00:00
2017-04-01 07:20:54 +00:00
if ( shared - > mark_cache )
2017-08-30 21:25:44 +00:00
throw Exception ( " Mark cache has been already created. " , ErrorCodes : : LOGICAL_ERROR ) ;
2014-02-11 13:30:42 +00:00
2019-12-31 02:47:10 +00:00
shared - > mark_cache = std : : make_shared < MarkCache > ( cache_size_in_bytes ) ;
2014-02-11 13:30:42 +00:00
}
MarkCachePtr Context : : getMarkCache ( ) const
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
return shared - > mark_cache ;
2014-02-11 13:30:42 +00:00
}
2017-08-07 17:01:04 +00:00
void Context : : dropMarkCache ( ) const
{
auto lock = getLock ( ) ;
if ( shared - > mark_cache )
shared - > mark_cache - > reset ( ) ;
}
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 )
throw Exception ( " Mapped file cache has been already created. " , ErrorCodes : : LOGICAL_ERROR ) ;
2021-03-28 19:24:28 +00:00
shared - > mmap_cache = std : : make_shared < MMappedFileCache > ( cache_size_in_num_entries ) ;
2021-03-28 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
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 )
2020-04-22 23:00:24 +00:00
shared - > buffer_flush_schedule_pool . emplace (
settings . background_buffer_flush_schedule_pool_size ,
CurrentMetrics : : BackgroundBufferFlushSchedulePoolTask ,
2020-04-23 00:14:58 +00:00
" BgBufSchPool " ) ;
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 )
2020-04-22 23:00:24 +00:00
shared - > schedule_pool . emplace (
settings . background_schedule_pool_size ,
CurrentMetrics : : BackgroundSchedulePoolTask ,
2020-04-23 00:14:58 +00:00
" BgSchPool " ) ;
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 )
2020-04-22 23:00:24 +00:00
shared - > distributed_schedule_pool . emplace (
settings . background_distributed_schedule_pool_size ,
CurrentMetrics : : BackgroundDistributedSchedulePoolTask ,
2020-04-23 00:14:58 +00:00
" BgDistSchPool " ) ;
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 )
shared - > message_broker_schedule_pool . emplace (
settings . background_message_broker_schedule_pool_size ,
CurrentMetrics : : BackgroundDistributedSchedulePoolTask ,
2021-03-25 13:57:47 +00:00
" BgMBSchPool " ) ;
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
{
auto lock = getLock ( ) ;
if ( ! shared - > replicated_fetches_throttler )
shared - > replicated_fetches_throttler = std : : make_shared < Throttler > (
settings . max_replicated_fetches_network_bandwidth_for_server ) ;
2021-05-27 12:54:47 +00:00
return shared - > replicated_fetches_throttler ;
2021-05-26 20:37:44 +00:00
}
ThrottlerPtr Context : : getReplicatedSendsThrottler ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > replicated_sends_throttler )
shared - > replicated_sends_throttler = std : : make_shared < Throttler > (
settings . max_replicated_sends_network_bandwidth_for_server ) ;
2021-05-27 12:54:47 +00:00
return shared - > replicated_sends_throttler ;
2021-05-26 20:37:44 +00:00
}
2020-11-12 20:11:04 +00:00
bool Context : : hasDistributedDDL ( ) const
2020-11-06 18:28:59 +00:00
{
return getConfigRef ( ) . has ( " distributed_ddl " ) ;
}
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 )
2020-11-06 18:28:59 +00:00
throw Exception ( " DDL background thread has already been initialized " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-12-01 17:20:42 +00:00
ddl_worker - > startup ( ) ;
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 ( ) )
throw Exception ( " There is no Zookeeper configuration in server config " , ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
if ( ! hasDistributedDDL ( ) )
throw Exception ( " There is no DistributedDDL configuration in server config " , ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
throw Exception ( " DDL background thread is not initialized " , ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
}
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 )
2020-11-03 09:57:24 +00:00
shared - > zookeeper = std : : make_shared < zkutil : : ZooKeeper > ( config , " zookeeper " ) ;
2018-04-03 19:43:33 +00:00
else if ( shared - > zookeeper - > expired ( ) )
2017-04-01 07:20:54 +00:00
shared - > zookeeper = shared - > zookeeper - > startNewSession ( ) ;
2014-04-25 13:55:15 +00:00
2017-04-01 07:20:54 +00:00
return shared - > zookeeper ;
2014-03-21 19:17:59 +00:00
}
2021-02-01 13:18:17 +00:00
2021-03-29 08:24:56 +00:00
void Context : : initializeKeeperStorageDispatcher ( ) const
2021-01-25 12:29:12 +00:00
{
2021-02-01 13:18:17 +00:00
# if USE_NURAFT
2021-05-18 14:08:56 +00:00
std : : lock_guard lock ( shared - > keeper_storage_dispatcher_mutex ) ;
2021-01-25 12:29:12 +00:00
2021-05-18 14:08:56 +00:00
if ( shared - > keeper_storage_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-05-18 14:08:56 +00:00
shared - > keeper_storage_dispatcher = std : : make_shared < KeeperStorageDispatcher > ( ) ;
shared - > keeper_storage_dispatcher - > initialize ( config , getApplicationType ( ) = = ApplicationType : : KEEPER ) ;
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-03-29 08:24:56 +00:00
std : : shared_ptr < KeeperStorageDispatcher > & Context : : getKeeperStorageDispatcher ( ) const
2020-10-30 14:16:47 +00:00
{
2021-05-18 14:08:56 +00:00
std : : lock_guard lock ( shared - > keeper_storage_dispatcher_mutex ) ;
if ( ! shared - > keeper_storage_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-05-18 14:08:56 +00:00
return shared - > keeper_storage_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-03-29 08:24:56 +00:00
void Context : : shutdownKeeperStorageDispatcher ( ) const
2021-01-27 17:54:25 +00:00
{
2021-02-01 13:18:17 +00:00
# if USE_NURAFT
2021-05-18 14:08:56 +00:00
std : : lock_guard lock ( shared - > keeper_storage_dispatcher_mutex ) ;
if ( shared - > keeper_storage_dispatcher )
2021-01-27 17:54:25 +00:00
{
2021-05-18 14:08:56 +00:00
shared - > keeper_storage_dispatcher - > shutdown ( ) ;
shared - > keeper_storage_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
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
{
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 ) ;
zookeeper
= shared - > auxiliary_zookeepers . emplace ( name , std : : make_shared < zkutil : : ZooKeeper > ( config , " auxiliary_zookeepers. " + name ) ) . 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 ;
}
2019-07-04 14:48:40 +00:00
void Context : : resetZooKeeper ( ) const
{
std : : lock_guard lock ( shared - > zookeeper_mutex ) ;
shared - > zookeeper . reset ( ) ;
}
2020-11-03 07:52:31 +00:00
static void reloadZooKeeperIfChangedImpl ( const ConfigurationPtr & config , const std : : string & config_name , zkutil : : ZooKeeperPtr & zk )
{
if ( ! zk | | zk - > configChanged ( * config , config_name ) )
2021-02-26 19:05:25 +00:00
{
if ( zk )
zk - > finalize ( ) ;
2020-11-03 07:52:31 +00:00
zk = std : : make_shared < zkutil : : ZooKeeper > ( * config , config_name ) ;
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
{
std : : lock_guard lock ( shared - > zookeeper_mutex ) ;
2020-11-03 09:57:24 +00:00
shared - > zookeeper_config = config ;
2020-11-03 07:52:31 +00:00
reloadZooKeeperIfChangedImpl ( config , " zookeeper " , shared - > zookeeper ) ;
}
void Context : : reloadAuxiliaryZooKeepersConfigIfChanged ( const ConfigurationPtr & config )
{
std : : lock_guard lock ( shared - > auxiliary_zookeepers_mutex ) ;
shared - > auxiliary_zookeepers_config = config ;
2020-11-03 09:57:24 +00:00
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
{
reloadZooKeeperIfChangedImpl ( config , " auxiliary_zookeepers. " + it - > first , it - > second ) ;
+ + 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
{
2018-04-21 18:41:06 +00:00
return getConfigRef ( ) . has ( " zookeeper " ) ;
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
2021-04-07 13:52:11 +00:00
InterserverCredentialsPtr Context : : getInterserverCredentials ( )
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
{
2017-04-01 07:20:54 +00:00
shared - > interserver_io_host = host ;
shared - > interserver_io_port = port ;
2014-03-21 19:49:27 +00:00
}
2014-11-19 20:40:51 +00:00
std : : pair < String , UInt16 > Context : : getInterserverIOAddress ( ) const
2014-03-21 19:49:27 +00:00
{
2017-04-01 07:20:54 +00:00
if ( shared - > interserver_io_host . empty ( ) | | shared - > interserver_io_port = = 0 )
2018-07-30 18:32:21 +00:00
throw Exception ( " Parameter 'interserver_http(s)_port' required for replication is not specified in configuration file. " ,
ErrorCodes : : NO_ELEMENTS_IN_CONFIG ) ;
2014-11-19 20:40:51 +00:00
2017-04-01 07:20:54 +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
}
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
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-06-26 14:15:57 +00:00
auto res = getClusters ( ) - > getCluster ( cluster_name ) ;
2021-03-09 17:05:24 +00:00
if ( res )
return res ;
2013-12-10 17:06:57 +00:00
2021-03-09 17:05:24 +00:00
res = tryGetReplicatedDatabaseCluster ( cluster_name ) ;
if ( res )
return res ;
2016-10-10 08:44:52 +00:00
2021-03-09 17:05:24 +00:00
throw Exception ( " Requested cluster ' " + cluster_name + " ' not found " , ErrorCodes : : BAD_GET ) ;
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
{
2021-06-26 14:15:57 +00:00
return getClusters ( ) - > getCluster ( cluster_name ) ;
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 ( ) ;
2021-06-26 14:15:57 +00:00
auto new_clusters = std : : make_shared < Clusters > ( config , settings ) ;
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
}
2021-06-26 14:15:57 +00:00
std : : shared_ptr < Clusters > Context : : getClusters ( ) const
2015-04-30 12:43:16 +00:00
{
2019-01-02 06:44:36 +00:00
std : : lock_guard lock ( shared - > clusters_mutex ) ;
2018-01-15 14:13:19 +00:00
if ( ! shared - > clusters )
2017-04-01 07:20:54 +00:00
{
2020-04-22 06:01:33 +00:00
const auto & config = shared - > clusters_config ? * shared - > clusters_config : getConfigRef ( ) ;
2021-06-26 14:15:57 +00:00
shared - > clusters = std : : make_shared < Clusters > ( config , settings ) ;
2017-04-01 07:20:54 +00:00
}
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
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
2017-10-13 19:13:41 +00:00
void Context : : setClustersConfig ( const ConfigurationPtr & config , 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 ) ;
2016-10-10 08:44:52 +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 ;
2017-04-01 07:20:54 +00:00
shared - > clusters_config = config ;
2017-10-13 19:13:41 +00:00
if ( ! shared - > clusters )
shared - > clusters = std : : make_unique < Clusters > ( * shared - > clusters_config , settings , config_name ) ;
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 )
throw Exception ( " Clusters are not set " , ErrorCodes : : LOGICAL_ERROR ) ;
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
{
2017-04-01 07:20:54 +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
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
{
2017-04-01 07:20:54 +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-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
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
2017-03-07 17:13:54 +00:00
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
}
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
{
2017-04-01 07:20:54 +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 )
2017-04-01 07:20:54 +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
2017-04-01 07:20:54 +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 " ) ;
2017-04-01 07:20:54 +00:00
else
2018-12-21 12:17:30 +00:00
shared - > compression_codec_selector = std : : make_unique < CompressionCodecSelector > ( ) ;
2017-04-01 07:20:54 +00:00
}
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
}
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 ) ;
}
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 ) ;
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
2021-04-10 23:33:54 +00:00
shared - > merge_tree_disk_selector = std : : make_shared < DiskSelector > ( config , config_name , shared_from_this ( ) ) ;
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
2020-08-03 01:44:58 +00:00
# if !defined(ARCADIA_BUILD)
2020-06-01 17:16:09 +00:00
if ( shared - > storage_s3_settings )
{
shared - > storage_s3_settings - > loadFromConfig ( " s3 " , config ) ;
}
2020-08-03 01:44:58 +00:00
# endif
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
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
2015-07-16 21:32:51 +00:00
2017-04-01 07:20:54 +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 ) ;
2017-04-01 07:20:54 +00:00
}
2015-07-16 21:32:51 +00:00
2017-04-01 07:20:54 +00:00
return * shared - > merge_tree_settings ;
2015-07-16 21:32:51 +00:00
}
2020-08-10 11:23:58 +00:00
const MergeTreeSettings & Context : : getReplicatedMergeTreeSettings ( ) const
{
auto lock = getLock ( ) ;
if ( ! shared - > replicated_merge_tree_settings )
{
const auto & config = getConfigRef ( ) ;
MergeTreeSettings mt_settings ;
mt_settings . loadFromConfig ( " merge_tree " , config ) ;
mt_settings . loadFromConfig ( " replicated_merge_tree " , config ) ;
shared - > replicated_merge_tree_settings . emplace ( mt_settings ) ;
}
return * shared - > replicated_merge_tree_settings ;
}
2020-06-01 17:16:09 +00:00
const StorageS3Settings & Context : : getStorageS3Settings ( ) const
{
2020-08-03 00:14:59 +00:00
# if !defined(ARCADIA_BUILD)
2020-06-01 17:16:09 +00:00
auto lock = getLock ( ) ;
if ( ! shared - > storage_s3_settings )
{
const auto & config = getConfigRef ( ) ;
shared - > storage_s3_settings . emplace ( ) . loadFromConfig ( " s3 " , config ) ;
}
return * shared - > storage_s3_settings ;
2020-08-03 00:14:59 +00:00
# else
throw Exception ( " S3 is unavailable in Arcadia " , ErrorCodes : : NOT_IMPLEMENTED ) ;
# endif
2020-06-01 17:16:09 +00:00
}
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-04-01 07:20:54 +00:00
return ;
2017-01-19 19:11:12 +00:00
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
2017-04-01 07:20:54 +00:00
if ( force_file_exists )
{
try
{
2021-04-28 20:48:34 +00:00
fs : : remove ( force_file ) ;
2017-04-01 07:20:54 +00:00
return ;
}
catch ( . . . )
{
/// 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-04-01 07:20:54 +00:00
}
}
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
}
2019-02-10 16:55:12 +00:00
BlockInputStreamPtr Context : : getInputFormat ( const String & name , ReadBuffer & buf , const Block & sample , UInt64 max_block_size ) const
2016-02-13 06:37:19 +00:00
{
2021-04-10 23:33:54 +00:00
return std : : make_shared < InputStreamFromInputFormat > (
FormatFactory : : instance ( ) . getInput ( name , buf , sample , shared_from_this ( ) , max_block_size ) ) ;
2016-02-13 06:37:19 +00:00
}
2020-12-30 03:07:30 +00:00
BlockOutputStreamPtr Context : : getOutputStreamParallelIfPossible ( 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 ( ) . getOutputStreamParallelIfPossible ( name , buf , sample , shared_from_this ( ) ) ;
2016-02-13 06:37:19 +00:00
}
2020-12-30 03:07:30 +00:00
BlockOutputStreamPtr Context : : getOutputStream ( 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 ( ) . getOutputStream ( 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
}
2016-04-09 07:47:08 +00:00
time_t Context : : getUptimeSeconds ( ) const
{
2017-04-01 07:20:54 +00:00
auto lock = getLock ( ) ;
return shared - > uptime_watch . elapsedSeconds ( ) ;
2016-04-09 07:47:08 +00:00
}
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 )
2020-06-27 19:05:00 +00:00
throw Exception ( " Can't reload config because config_reload_callback is not set. " , ErrorCodes : : LOGICAL_ERROR ) ;
2018-03-13 10:41:47 +00:00
shared - > config_reload_callback ( ) ;
}
2015-04-16 06:12:35 +00:00
void Context : : shutdown ( )
{
2020-10-26 16:09:53 +00:00
// Disk selector might not be initialized if there was some error during
// its initialization. Don't try to initialize it again on shutdown.
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
}
2017-04-01 07:20:54 +00:00
shared - > shutdown ( ) ;
2015-04-16 06:12:35 +00:00
}
2016-11-11 17:01:02 +00:00
Context : : ApplicationType Context : : getApplicationType ( ) const
{
2017-04-01 07:20:54 +00:00
return shared - > application_type ;
2016-11-11 17:01:02 +00:00
}
void Context : : setApplicationType ( ApplicationType type )
{
2017-04-01 07:20:54 +00:00
/// Lock isn't required, you should set it at start
shared - > application_type = type ;
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 " ) ;
2020-03-04 22:27:03 +00:00
getAccessControlManager ( ) . setDefaultProfileName ( shared - > default_profile_name ) ;
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
{
2017-04-01 07:20:54 +00:00
return shared - > default_profile_name ;
2017-03-23 14:14:56 +00:00
}
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-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 )
throw Exception ( " Duplicate name " + backQuote ( name ) + " of query parameter " , ErrorCodes : : BAD_ARGUMENTS ) ;
2019-05-18 21:07:23 +00:00
}
2021-03-17 08:21:04 +00:00
void Context : : addBridgeCommand ( std : : unique_ptr < ShellCommand > cmd ) const
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 ;
}
2018-05-21 13:49:54 +00:00
std : : shared_ptr < ActionLocksManager > Context : : getActionLocksManager ( )
{
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 )
throw Exception ( " External tables initializer is already set " , ErrorCodes : : LOGICAL_ERROR ) ;
external_tables_initializer_callback = std : : move ( initializer ) ;
}
void Context : : initializeExternalTablesIfSet ( )
{
if ( external_tables_initializer_callback )
{
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 )
throw Exception ( " Input initializer is already set " , ErrorCodes : : LOGICAL_ERROR ) ;
input_initializer_callback = std : : move ( initializer ) ;
}
void Context : : initializeInput ( const StoragePtr & input_storage )
{
if ( ! input_initializer_callback )
throw Exception ( " Input initializer is not set " , ErrorCodes : : LOGICAL_ERROR ) ;
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 )
throw Exception ( " Input blocks reader is already set " , ErrorCodes : : LOGICAL_ERROR ) ;
input_blocks_reader = std : : move ( reader ) ;
}
InputBlocksReader Context : : getInputBlocksReaderCallback ( ) const
{
return input_blocks_reader ;
}
void Context : : resetInputCallbacks ( )
{
if ( input_initializer_callback )
input_initializer_callback = { } ;
if ( input_blocks_reader )
input_blocks_reader = { } ;
}
2020-03-10 10:15:27 +00:00
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 )
exception - > emplace ( " Both table name and UUID are empty " , ErrorCodes : : UNKNOWN_TABLE ) ;
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 ;
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 )
exception - > emplace ( " External and temporary tables have no database, but " +
2020-02-10 18:19:35 +00:00
storage_id . database_name + " is specified " , ErrorCodes : : UNKNOWN_TABLE ) ;
2020-02-12 18:14:12 +00:00
return StorageID : : createEmpty ( ) ;
2020-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-03 19:53:18 +00:00
/// Global context should not contain temporary tables
2021-04-10 23:33:54 +00:00
assert ( ! isGlobalContext ( ) | | getApplicationType ( ) = = ApplicationType : : LOCAL ) ;
2020-03-03 19:53:18 +00:00
2020-03-11 19:10:55 +00:00
auto resolved_id = StorageID : : createEmpty ( ) ;
2021-06-01 12:20:52 +00:00
auto try_resolve = [ & ] ( ContextPtr context ) - > bool
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 )
exception - > emplace ( " Default database is not selected " , ErrorCodes : : UNKNOWN_DATABASE ) ;
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 )
exception - > emplace ( " Cannot resolve database name for table " + storage_id . getNameForLogs ( ) , ErrorCodes : : UNKNOWN_TABLE ) ;
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-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 ( ) )
throw Exception ( fmt : : format ( " Next task callback is not set for query {} " , getInitialQueryId ( ) ) , ErrorCodes : : LOGICAL_ERROR ) ;
return next_task_callback . value ( ) ;
}
2021-04-08 19:00:39 +00:00
void Context : : setReadTaskCallback ( ReadTaskCallback & & callback )
2021-04-06 11:05:47 +00:00
{
next_task_callback = callback ;
}
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 ;
}
2012-08-02 17:33:31 +00:00
}