add debug assert for Context

This commit is contained in:
Alexander Tokmakov 2019-11-18 19:01:36 +03:00
parent 2173bf5f4a
commit 2f708c72be
2 changed files with 201 additions and 21 deletions

View File

@ -331,19 +331,20 @@ Context Context::createGlobal()
Context::~Context() = default;
InterserverIOHandler & Context::getInterserverIOHandler() { return shared->interserver_io_handler; }
InterserverIOHandler & Context::getInterserverIOHandler() { assert_context_exists(); return shared->interserver_io_handler; }
std::unique_lock<std::recursive_mutex> Context::getLock() const
{
assert_context_exists();
ProfileEvents::increment(ProfileEvents::ContextLock);
CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait};
return std::unique_lock(shared->mutex);
}
ProcessList & Context::getProcessList() { return shared->process_list; }
const ProcessList & Context::getProcessList() const { return shared->process_list; }
MergeList & Context::getMergeList() { return shared->merge_list; }
const MergeList & Context::getMergeList() const { return shared->merge_list; }
ProcessList & Context::getProcessList() { assert_context_exists(); return shared->process_list; }
const ProcessList & Context::getProcessList() const { assert_context_exists(); return shared->process_list; }
MergeList & Context::getMergeList() { assert_context_exists(); return shared->merge_list; }
const MergeList & Context::getMergeList() const { assert_context_exists(); return shared->merge_list; }
const Databases Context::getDatabases() const
@ -361,6 +362,7 @@ Databases Context::getDatabases()
Context::SessionKey Context::getSessionKey(const String & session_id) const
{
assert_context_exists();
auto & user_name = client_info.current_user;
if (user_name.empty())
@ -372,6 +374,7 @@ Context::SessionKey Context::getSessionKey(const String & session_id) const
void Context::scheduleCloseSession(const Context::SessionKey & key, std::chrono::steady_clock::duration timeout)
{
assert_context_exists();
const UInt64 close_index = timeout / shared->close_interval + 1;
const auto new_close_cycle = shared->close_cycle + close_index;
@ -648,6 +651,7 @@ void Context::calculateUserSettings()
void Context::setProfile(const String & profile)
{
assert_context_exists();
settings.setProfile(profile, *shared->users_config);
auto new_constraints
@ -658,6 +662,7 @@ void Context::setProfile(const String & profile)
std::shared_ptr<const User> Context::getUser(const String & user_name)
{
assert_context_exists();
return shared->users_manager->getUser(user_name);
}
@ -715,6 +720,7 @@ bool Context::hasDictionaryAccessRights(const String & dictionary_name) const
void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) const
{
assert_context_exists();
if (client_info.current_user.empty() || (database_name == "system"))
{
/// An unnamed user, i.e. server, has access to all databases.
@ -727,6 +733,7 @@ void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) c
void Context::addDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where)
{
assert_context_exists();
checkDatabaseAccessRightsImpl(from.first);
checkDatabaseAccessRightsImpl(where.first);
shared->view_dependencies[from].insert(where);
@ -745,6 +752,7 @@ void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAnd
void Context::removeDependencyUnsafe(const DatabaseAndTableName & from, const DatabaseAndTableName & where)
{
assert_context_exists();
checkDatabaseAccessRightsImpl(from.first);
checkDatabaseAccessRightsImpl(where.first);
shared->view_dependencies[from].erase(where);
@ -816,6 +824,7 @@ bool Context::isDatabaseExist(const String & database_name) const
bool Context::isExternalTableExist(const String & table_name) const
{
assert_context_exists();
return external_tables.end() != external_tables.find(table_name);
}
@ -861,12 +870,14 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const
const Scalars & Context::getScalars() const
{
assert_context_exists();
return scalars;
}
const Block & Context::getScalar(const String & name) const
{
assert_context_exists();
auto it = scalars.find(name);
if (scalars.end() == it)
throw Exception("Scalar " + backQuoteIfNeed(name) + " doesn't exist (internal bug)", ErrorCodes::UNKNOWN_SCALAR);
@ -898,6 +909,7 @@ Tables Context::getExternalTables() const
StoragePtr Context::tryGetExternalTable(const String & table_name) const
{
assert_context_exists();
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
if (external_tables.end() == jt)
return StoragePtr();
@ -924,6 +936,7 @@ StoragePtr Context::tryGetTable(const String & database_name, const String & tab
StoragePtr Context::getTableImpl(const String & database_name, const String & table_name, Exception * exception) const
{
assert_context_exists();
String db;
DatabasePtr database;
@ -965,6 +978,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta
void Context::addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast)
{
assert_context_exists();
if (external_tables.end() != external_tables.find(table_name))
throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
@ -974,18 +988,21 @@ void Context::addExternalTable(const String & table_name, const StoragePtr & sto
void Context::addScalar(const String & name, const Block & block)
{
assert_context_exists();
scalars[name] = block;
}
bool Context::hasScalar(const String & name) const
{
assert_context_exists();
return scalars.count(name);
}
StoragePtr Context::tryRemoveExternalTable(const String & table_name)
{
assert_context_exists();
TableAndCreateASTs::const_iterator it = external_tables.find(table_name);
if (external_tables.end() == it)
@ -999,6 +1016,7 @@ StoragePtr Context::tryRemoveExternalTable(const String & table_name)
StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
{
assert_context_exists();
/// Slightly suboptimal.
auto hash = table_expression->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
@ -1019,6 +1037,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression)
void Context::addViewSource(const StoragePtr & storage)
{
assert_context_exists();
if (view_source)
throw Exception(
"Temporary view source storage " + backQuoteIfNeed(view_source->getName()) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
@ -1028,6 +1047,7 @@ void Context::addViewSource(const StoragePtr & storage)
StoragePtr Context::getViewSource()
{
assert_context_exists();
return view_source;
}
@ -1054,6 +1074,7 @@ DDLGuard::~DDLGuard()
std::unique_ptr<DDLGuard> Context::getDDLGuard(const String & database, const String & table) const
{
assert_context_exists();
std::unique_lock lock(shared->ddl_guards_mutex);
return std::make_unique<DDLGuard>(shared->ddl_guards[database], std::move(lock), table);
}
@ -1102,6 +1123,7 @@ ASTPtr Context::getCreateDictionaryQuery(const String & database_name, const Str
ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const
{
assert_context_exists();
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
if (external_tables.end() == jt)
throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist", ErrorCodes::UNKNOWN_TABLE);
@ -1121,12 +1143,14 @@ ASTPtr Context::getCreateDatabaseQuery(const String & database_name) const
Settings Context::getSettings() const
{
assert_context_exists();
return settings;
}
void Context::setSettings(const Settings & settings_)
{
assert_context_exists();
settings = settings_;
}
@ -1157,6 +1181,7 @@ void Context::setSetting(const String & name, const Field & value)
void Context::applySettingChange(const SettingChange & change)
{
assert_context_exists();
setSetting(change.name, change.value);
}
@ -1171,6 +1196,7 @@ void Context::applySettingsChanges(const SettingsChanges & changes)
void Context::checkSettingsConstraints(const SettingChange & change)
{
assert_context_exists();
if (settings_constraints)
settings_constraints->check(settings, change);
}
@ -1178,6 +1204,7 @@ void Context::checkSettingsConstraints(const SettingChange & change)
void Context::checkSettingsConstraints(const SettingsChanges & changes)
{
assert_context_exists();
if (settings_constraints)
settings_constraints->check(settings, changes);
}
@ -1185,18 +1212,21 @@ void Context::checkSettingsConstraints(const SettingsChanges & changes)
String Context::getCurrentDatabase() const
{
assert_context_exists();
return current_database;
}
String Context::getCurrentQueryId() const
{
assert_context_exists();
return client_info.current_query_id;
}
String Context::getInitialQueryId() const
{
assert_context_exists();
return client_info.initial_query_id;
}
@ -1211,6 +1241,7 @@ void Context::setCurrentDatabase(const String & name)
void Context::setCurrentQueryId(const String & query_id)
{
assert_context_exists();
if (!client_info.current_query_id.empty())
throw Exception("Logical error: attempt to set query_id twice", ErrorCodes::LOGICAL_ERROR);
@ -1251,6 +1282,7 @@ void Context::setCurrentQueryId(const String & query_id)
void Context::killCurrentQuery()
{
assert_context_exists();
if (process_list_elem)
{
process_list_elem->cancelQuery(true);
@ -1259,27 +1291,32 @@ void Context::killCurrentQuery()
String Context::getDefaultFormat() const
{
assert_context_exists();
return default_format.empty() ? "TabSeparated" : default_format;
}
void Context::setDefaultFormat(const String & name)
{
assert_context_exists();
default_format = name;
}
MultiVersion<Macros>::Version Context::getMacros() const
{
assert_context_exists();
return shared->macros.get();
}
void Context::setMacros(std::unique_ptr<Macros> && macros)
{
assert_context_exists();
shared->macros.set(std::move(macros));
}
const Context & Context::getQueryContext() const
{
assert_context_exists();
if (!query_context)
throw Exception("There is no query", ErrorCodes::THERE_IS_NO_QUERY);
return *query_context;
@ -1287,6 +1324,7 @@ const Context & Context::getQueryContext() const
Context & Context::getQueryContext()
{
assert_context_exists();
if (!query_context)
throw Exception("There is no query", ErrorCodes::THERE_IS_NO_QUERY);
return *query_context;
@ -1294,6 +1332,7 @@ Context & Context::getQueryContext()
const Context & Context::getSessionContext() const
{
assert_context_exists();
if (!session_context)
throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION);
return *session_context;
@ -1301,6 +1340,7 @@ const Context & Context::getSessionContext() const
Context & Context::getSessionContext()
{
assert_context_exists();
if (!session_context)
throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION);
return *session_context;
@ -1308,6 +1348,7 @@ Context & Context::getSessionContext()
const Context & Context::getGlobalContext() const
{
assert_context_exists();
if (!global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
return *global_context;
@ -1315,6 +1356,7 @@ const Context & Context::getGlobalContext() const
Context & Context::getGlobalContext()
{
assert_context_exists();
if (!global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
return *global_context;
@ -1323,17 +1365,20 @@ Context & Context::getGlobalContext()
const EmbeddedDictionaries & Context::getEmbeddedDictionaries() const
{
assert_context_exists();
return getEmbeddedDictionariesImpl(false);
}
EmbeddedDictionaries & Context::getEmbeddedDictionaries()
{
assert_context_exists();
return getEmbeddedDictionariesImpl(false);
}
const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() const
{
assert_context_exists();
std::lock_guard lock(shared->external_dictionaries_mutex);
if (!shared->external_dictionaries_loader)
{
@ -1347,12 +1392,14 @@ const ExternalDictionariesLoader & Context::getExternalDictionariesLoader() cons
ExternalDictionariesLoader & Context::getExternalDictionariesLoader()
{
assert_context_exists();
return const_cast<ExternalDictionariesLoader &>(const_cast<const Context *>(this)->getExternalDictionariesLoader());
}
const ExternalModelsLoader & Context::getExternalModelsLoader() const
{
assert_context_exists();
std::lock_guard lock(shared->external_models_mutex);
if (!shared->external_models_loader)
{
@ -1366,12 +1413,14 @@ const ExternalModelsLoader & Context::getExternalModelsLoader() const
ExternalModelsLoader & Context::getExternalModelsLoader()
{
assert_context_exists();
return const_cast<ExternalModelsLoader &>(const_cast<const Context *>(this)->getExternalModelsLoader());
}
EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_error) const
{
assert_context_exists();
std::lock_guard lock(shared->embedded_dictionaries_mutex);
if (!shared->embedded_dictionaries)
@ -1390,30 +1439,35 @@ EmbeddedDictionaries & Context::getEmbeddedDictionariesImpl(const bool throw_on_
void Context::tryCreateEmbeddedDictionaries() const
{
assert_context_exists();
static_cast<void>(getEmbeddedDictionariesImpl(true));
}
void Context::setProgressCallback(ProgressCallback callback)
{
assert_context_exists();
/// Callback is set to a session or to a query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
progress_callback = callback;
}
ProgressCallback Context::getProgressCallback() const
{
assert_context_exists();
return progress_callback;
}
void Context::setProcessListElement(ProcessList::Element * elem)
{
assert_context_exists();
/// Set to a session or query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
process_list_elem = elem;
}
ProcessList::Element * Context::getProcessListElement() const
{
assert_context_exists();
return process_list_elem;
}
@ -1515,6 +1569,7 @@ DDLWorker & Context::getDDLWorker() const
zkutil::ZooKeeperPtr Context::getZooKeeper() const
{
assert_context_exists();
std::lock_guard lock(shared->zookeeper_mutex);
if (!shared->zookeeper)
@ -1527,24 +1582,28 @@ zkutil::ZooKeeperPtr Context::getZooKeeper() const
void Context::resetZooKeeper() const
{
assert_context_exists();
std::lock_guard lock(shared->zookeeper_mutex);
shared->zookeeper.reset();
}
bool Context::hasZooKeeper() const
{
assert_context_exists();
return getConfigRef().has("zookeeper");
}
void Context::setInterserverIOAddress(const String & host, UInt16 port)
{
assert_context_exists();
shared->interserver_io_host = host;
shared->interserver_io_port = port;
}
std::pair<String, UInt16> Context::getInterserverIOAddress() const
{
assert_context_exists();
if (shared->interserver_io_host.empty() || shared->interserver_io_port == 0)
throw Exception("Parameter 'interserver_http(s)_port' required for replication is not specified in configuration file.",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
@ -1554,22 +1613,26 @@ std::pair<String, UInt16> Context::getInterserverIOAddress() const
void Context::setInterserverCredentials(const String & user, const String & password)
{
assert_context_exists();
shared->interserver_io_user = user;
shared->interserver_io_password = password;
}
std::pair<String, String> Context::getInterserverCredentials() const
{
assert_context_exists();
return { shared->interserver_io_user, shared->interserver_io_password };
}
void Context::setInterserverScheme(const String & scheme)
{
assert_context_exists();
shared->interserver_scheme = scheme;
}
String Context::getInterserverScheme() const
{
assert_context_exists();
return shared->interserver_scheme;
}
@ -1593,6 +1656,7 @@ std::optional<UInt16> Context::getTCPPortSecure() const
std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) const
{
assert_context_exists();
auto res = getClusters().getCluster(cluster_name);
if (!res)
@ -1604,12 +1668,14 @@ std::shared_ptr<Cluster> Context::getCluster(const std::string & cluster_name) c
std::shared_ptr<Cluster> Context::tryGetCluster(const std::string & cluster_name) const
{
assert_context_exists();
return getClusters().getCluster(cluster_name);
}
void Context::reloadClusterConfig()
{
assert_context_exists();
while (true)
{
ConfigurationPtr cluster_config;
@ -1637,6 +1703,7 @@ void Context::reloadClusterConfig()
Clusters & Context::getClusters() const
{
assert_context_exists();
std::lock_guard lock(shared->clusters_mutex);
if (!shared->clusters)
{
@ -1651,6 +1718,7 @@ Clusters & Context::getClusters() const
/// On repeating calls updates existing clusters and adds new clusters, doesn't delete old clusters
void Context::setClustersConfig(const ConfigurationPtr & config, const String & config_name)
{
assert_context_exists();
std::lock_guard lock(shared->clusters_mutex);
shared->clusters_config = config;
@ -1664,6 +1732,7 @@ void Context::setClustersConfig(const ConfigurationPtr & config, const String &
void Context::setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster)
{
assert_context_exists();
std::lock_guard lock(shared->clusters_mutex);
if (!shared->clusters)
@ -1681,11 +1750,13 @@ void Context::initializeSystemLogs()
bool Context::hasTraceCollector()
{
assert_context_exists();
return shared->hasTraceCollector();
}
void Context::initializeTraceCollector()
{
assert_context_exists();
shared->initializeTraceCollector(getTraceLog());
}
@ -1850,6 +1921,7 @@ const MergeTreeSettings & Context::getMergeTreeSettings() const
void Context::checkCanBeDropped(const String & database, const String & table, const size_t & size, const size_t & max_size_to_drop) const
{
assert_context_exists();
if (!max_size_to_drop || size <= max_size_to_drop)
return;
@ -1891,6 +1963,7 @@ void Context::checkCanBeDropped(const String & database, const String & table, c
void Context::setMaxTableSizeToDrop(size_t max_size)
{
assert_context_exists();
// Is initialized at server startup
shared->max_table_size_to_drop = max_size;
}
@ -1898,6 +1971,7 @@ void Context::setMaxTableSizeToDrop(size_t max_size)
void Context::checkTableCanBeDropped(const String & database, const String & table, const size_t & table_size) const
{
assert_context_exists();
size_t max_table_size_to_drop = shared->max_table_size_to_drop;
checkCanBeDropped(database, table, table_size, max_table_size_to_drop);
@ -1906,6 +1980,7 @@ void Context::checkTableCanBeDropped(const String & database, const String & tab
void Context::setMaxPartitionSizeToDrop(size_t max_size)
{
assert_context_exists();
// Is initialized at server startup
shared->max_partition_size_to_drop = max_size;
}
@ -1913,6 +1988,7 @@ void Context::setMaxPartitionSizeToDrop(size_t max_size)
void Context::checkPartitionCanBeDropped(const String & database, const String & table, const size_t & partition_size) const
{
assert_context_exists();
size_t max_partition_size_to_drop = shared->max_partition_size_to_drop;
checkCanBeDropped(database, table, partition_size, max_partition_size_to_drop);
@ -1921,16 +1997,19 @@ void Context::checkPartitionCanBeDropped(const String & database, const String &
BlockInputStreamPtr Context::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const
{
assert_context_exists();
return FormatFactory::instance().getInput(name, buf, sample, *this, max_block_size);
}
BlockOutputStreamPtr Context::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const
{
assert_context_exists();
return FormatFactory::instance().getOutput(name, buf, sample, *this);
}
OutputFormatPtr Context::getOutputFormatProcessor(const String & name, WriteBuffer & buf, const Block & sample) const
{
assert_context_exists();
return FormatFactory::instance().getOutputFormat(name, buf, sample, *this);
}
@ -1944,12 +2023,14 @@ time_t Context::getUptimeSeconds() const
void Context::setConfigReloadCallback(ConfigReloadCallback && callback)
{
assert_context_exists();
/// Is initialized at server startup, so lock isn't required. Otherwise use mutex.
shared->config_reload_callback = std::move(callback);
}
void Context::reloadConfig() const
{
assert_context_exists();
/// Use mutex if callback may be changed after startup.
if (!shared->config_reload_callback)
throw Exception("Can't reload config beacuse config_reload_callback is not set.", ErrorCodes::LOGICAL_ERROR);
@ -1960,23 +2041,27 @@ void Context::reloadConfig() const
void Context::shutdown()
{
assert_context_exists();
shared->shutdown();
}
Context::ApplicationType Context::getApplicationType() const
{
assert_context_exists();
return shared->application_type;
}
void Context::setApplicationType(ApplicationType type)
{
assert_context_exists();
/// Lock isn't required, you should set it at start
shared->application_type = type;
}
void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & config)
{
assert_context_exists();
shared->default_profile_name = config.getString("default_profile", "default");
shared->system_profile_name = config.getString("system_profile", shared->default_profile_name);
setSetting("profile", shared->system_profile_name);
@ -1984,44 +2069,52 @@ void Context::setDefaultProfiles(const Poco::Util::AbstractConfiguration & confi
String Context::getDefaultProfileName() const
{
assert_context_exists();
return shared->default_profile_name;
}
String Context::getSystemProfileName() const
{
assert_context_exists();
return shared->system_profile_name;
}
String Context::getFormatSchemaPath() const
{
assert_context_exists();
return shared->format_schema_path;
}
void Context::setFormatSchemaPath(const String & path)
{
assert_context_exists();
shared->format_schema_path = path;
}
Context::SampleBlockCache & Context::getSampleBlockCache() const
{
assert_context_exists();
return getQueryContext().sample_block_cache;
}
bool Context::hasQueryParameters() const
{
assert_context_exists();
return !query_parameters.empty();
}
const NameToNameMap & Context::getQueryParameters() const
{
assert_context_exists();
return query_parameters;
}
void Context::setQueryParameter(const String & name, const String & value)
{
assert_context_exists();
if (!query_parameters.emplace(name, value).second)
throw Exception("Duplicate name " + backQuote(name) + " of query parameter", ErrorCodes::BAD_ARGUMENTS);
}
@ -2065,12 +2158,14 @@ void Context::addXDBCBridgeCommand(std::unique_ptr<ShellCommand> cmd) const
IHostContextPtr & Context::getHostContext()
{
assert_context_exists();
return host_context;
}
const IHostContextPtr & Context::getHostContext() const
{
assert_context_exists();
return host_context;
}
@ -2088,6 +2183,7 @@ std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()
void Context::setExternalTablesInitializer(ExternalTablesInitializer && initializer)
{
assert_context_exists();
if (external_tables_initializer_callback)
throw Exception("External tables initializer is already set", ErrorCodes::LOGICAL_ERROR);
@ -2096,6 +2192,7 @@ void Context::setExternalTablesInitializer(ExternalTablesInitializer && initiali
void Context::initializeExternalTablesIfSet()
{
assert_context_exists();
if (external_tables_initializer_callback)
{
external_tables_initializer_callback(*this);
@ -2107,6 +2204,7 @@ void Context::initializeExternalTablesIfSet()
void Context::setInputInitializer(InputInitializer && initializer)
{
assert_context_exists();
if (input_initializer_callback)
throw Exception("Input initializer is already set", ErrorCodes::LOGICAL_ERROR);
@ -2116,6 +2214,7 @@ void Context::setInputInitializer(InputInitializer && initializer)
void Context::initializeInput(const StoragePtr & input_storage)
{
assert_context_exists();
if (!input_initializer_callback)
throw Exception("Input initializer is not set", ErrorCodes::LOGICAL_ERROR);
@ -2127,6 +2226,7 @@ void Context::initializeInput(const StoragePtr & input_storage)
void Context::setInputBlocksReaderCallback(InputBlocksReader && reader)
{
assert_context_exists();
if (input_blocks_reader)
throw Exception("Input blocks reader is already set", ErrorCodes::LOGICAL_ERROR);
@ -2136,12 +2236,14 @@ void Context::setInputBlocksReaderCallback(InputBlocksReader && reader)
InputBlocksReader Context::getInputBlocksReaderCallback() const
{
assert_context_exists();
return input_blocks_reader;
}
void Context::resetInputCallbacks()
{
assert_context_exists();
if (input_initializer_callback)
input_initializer_callback = {};
@ -2184,5 +2286,56 @@ void SessionCleaner::run()
}
}
#ifndef NDEBUG
std::unordered_map<const void *, StackTrace> __ContextDebug::__contexts;
std::list<std::tuple<const void *, StackTrace, StackTrace>> __ContextDebug::__destroyed_contexts;
std::mutex __ContextDebug::__contexts_lock;
__ContextDebug::__ContextDebug()
{
std::unique_lock lock{__contexts_lock};
__contexts.insert({this, StackTrace()});
}
__ContextDebug::__ContextDebug(const __ContextDebug &)
{
std::unique_lock lock{__contexts_lock};
__contexts.insert({this, StackTrace()});
}
__ContextDebug & __ContextDebug::operator=(const __ContextDebug &)
{
assert_context_exists();
return *this;
}
__ContextDebug::~__ContextDebug()
{
std::unique_lock lock{__contexts_lock};
auto it = __contexts.find(this);
__destroyed_contexts.emplace_front(it->first, std::move(it->second), StackTrace());
if (__max_traces < __destroyed_contexts.size())
__destroyed_contexts.pop_back();
__contexts.erase(it);
}
void __ContextDebug::assert_context_exists() const
{
std::unique_lock lock{__contexts_lock};
if (__contexts.find(static_cast<const void *>(this)) == __contexts.end())
{
String msg;
auto it = std::find_if(__destroyed_contexts.begin(), __destroyed_contexts.end(), [=](auto & tuple) { return static_cast<const void *>(this) == std::get<0>(tuple); });
if (it != __destroyed_contexts.end())
{
msg += "\nContext was created here:\n" + std::get<1>(*it).toString();
msg += "\nAnd was destroyed here:\n" + std::get<2>(*it).toString();
}
throw Exception("\n================================\n"
"Assertion failed: trying to access Context, which doesn't exist. Most likely it's because of dangling reference." + msg +
"\n================================\n", ErrorCodes::LOGICAL_ERROR);
}
}
#endif
}

View File

@ -117,6 +117,27 @@ struct IHostContext
using IHostContextPtr = std::shared_ptr<IHostContext>;
#ifndef NDEBUG
/// Unfortunately, AddressSanitizer cannot find all usages of dangling references to Context
class __ContextDebug
{
static std::unordered_map<const void *, StackTrace> __contexts;
static std::list<std::tuple<const void *, StackTrace, StackTrace>> __destroyed_contexts;
static const size_t __max_traces = 4096;
static std::mutex __contexts_lock;
protected:
__ContextDebug();
__ContextDebug(const __ContextDebug &);
__ContextDebug & operator=(const __ContextDebug &);
__ContextDebug(__ContextDebug &&) = delete;
__ContextDebug & operator=(__ContextDebug &&) = delete;
~__ContextDebug();
public:
void assert_context_exists() const;
};
#endif
/** A set of known objects that can be used in the query.
* Consists of a shared part (always common to all sessions and queries)
* and copied part (which can be its own for each session or query).
@ -124,8 +145,14 @@ using IHostContextPtr = std::shared_ptr<IHostContext>;
* Everything is encapsulated for all sorts of checks and locks.
*/
class Context
#ifndef NDEBUG
: public __ContextDebug
#endif
{
private:
#ifdef NDEBUG
void assert_context_exists() const {};
#endif
using Shared = std::shared_ptr<ContextShared>;
Shared shared;
@ -235,8 +262,8 @@ public:
InputBlocksReader getInputBlocksReaderCallback() const;
void resetInputCallbacks();
ClientInfo & getClientInfo() { return client_info; }
const ClientInfo & getClientInfo() const { return client_info; }
ClientInfo & getClientInfo() { assert_context_exists(); return client_info; }
const ClientInfo & getClientInfo() const { assert_context_exists(); return client_info; }
void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address);
QuotaForIntervals & getQuota();
@ -301,8 +328,8 @@ public:
void killCurrentQuery();
void setInsertionTable(std::pair<String, String> && db_and_table) { insertion_table = db_and_table; }
const std::pair<String, String> & getInsertionTable() const { return insertion_table; }
void setInsertionTable(std::pair<String, String> && db_and_table) { assert_context_exists(); insertion_table = db_and_table; }
const std::pair<String, String> & getInsertionTable() const { assert_context_exists(); return insertion_table; }
String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned.
void setDefaultFormat(const String & name);
@ -324,7 +351,7 @@ public:
void checkSettingsConstraints(const SettingsChanges & changes);
/// Returns the current constraints (can return null).
std::shared_ptr<const SettingsConstraints> getSettingsConstraints() const { return settings_constraints; }
std::shared_ptr<const SettingsConstraints> getSettingsConstraints() const { assert_context_exists(); return settings_constraints; }
const EmbeddedDictionaries & getEmbeddedDictionaries() const;
const ExternalDictionariesLoader & getExternalDictionariesLoader() const;
@ -384,25 +411,25 @@ public:
const Context & getQueryContext() const;
Context & getQueryContext();
bool hasQueryContext() const { return query_context != nullptr; }
bool hasQueryContext() const { assert_context_exists(); return query_context != nullptr; }
const Context & getSessionContext() const;
Context & getSessionContext();
bool hasSessionContext() const { return session_context != nullptr; }
bool hasSessionContext() const { assert_context_exists(); return session_context != nullptr; }
const Context & getGlobalContext() const;
Context & getGlobalContext();
bool hasGlobalContext() const { return global_context != nullptr; }
bool hasGlobalContext() const { assert_context_exists(); return global_context != nullptr; }
void setQueryContext(Context & context_) { query_context = &context_; }
void setSessionContext(Context & context_) { session_context = &context_; }
void setQueryContext(Context & context_) { assert_context_exists(); query_context = &context_; }
void setSessionContext(Context & context_) { assert_context_exists(); session_context = &context_; }
void makeQueryContext() { query_context = this; }
void makeSessionContext() { session_context = this; }
void makeGlobalContext() { global_context = this; }
void makeQueryContext() { assert_context_exists(); query_context = this; }
void makeSessionContext() { assert_context_exists(); session_context = this; }
void makeGlobalContext() { assert_context_exists(); global_context = this; }
const Settings & getSettingsRef() const { return settings; }
Settings & getSettingsRef() { return settings; }
const Settings & getSettingsRef() const { assert_context_exists(); return settings; }
Settings & getSettingsRef() { assert_context_exists(); return settings; }
void setProgressCallback(ProgressCallback callback);
@ -544,7 +571,7 @@ public:
bool hasQueryParameters() const;
const NameToNameMap & getQueryParameters() const;
void setQueryParameter(const String & name, const String & value);
void setQueryParameters(const NameToNameMap & parameters) { query_parameters = parameters; }
void setQueryParameters(const NameToNameMap & parameters) { assert_context_exists(); query_parameters = parameters; }
#if USE_EMBEDDED_COMPILER
std::shared_ptr<CompiledExpressionCache> getCompiledExpressionCache() const;