ClickHouse/dbms/src/Interpreters/Context.cpp

538 lines
15 KiB
C++
Raw Normal View History

2012-08-17 19:53:11 +00:00
#include <Poco/File.h>
2013-12-07 16:51:29 +00:00
#include <Poco/Net/NetworkInterface.h>
2012-08-17 19:53:11 +00:00
#include <DB/Common/escapeForFileName.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/copyData.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ParserCreateQuery.h>
2012-08-02 17:33:31 +00:00
#include <DB/Interpreters/Context.h>
2013-12-07 16:51:29 +00:00
#include <DB/Client/ConnectionPoolWithFailover.h>
2012-08-02 17:33:31 +00:00
namespace DB
{
String Context::getPath() const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return shared->path;
}
void Context::setPath(const String & path)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
shared->path = path;
}
void Context::setUsersConfig(ConfigurationPtr config)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
shared->users_config = config;
shared->users.loadFromConfig(*shared->users_config);
shared->quotas.loadFromConfig(*shared->users_config);
}
ConfigurationPtr Context::getUsersConfig()
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return shared->users_config;
}
void Context::setUser(const String & name, const String & password, const Poco::Net::IPAddress & address, const String & quota_key)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
const User & user_props = shared->users.get(name, password, address);
setSetting("profile", user_props.profile);
setQuota(user_props.quota, quota_key, name, address);
user = name;
ip_address = address;
}
void Context::setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
quota = shared->quotas.get(name, quota_key, user_name, address);
}
QuotaForIntervals & Context::getQuota()
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return *quota;
}
void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
shared->view_dependencies[from].insert(where);
}
void Context::removeDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
shared->view_dependencies[from].erase(where);
}
Dependencies Context::getDependencies(const DatabaseAndTableName & from) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
ViewDependencies::const_iterator iter = shared->view_dependencies.find(from);
if (iter == shared->view_dependencies.end())
return Dependencies();
const std::set<DatabaseAndTableName> &buf = iter->second;
Dependencies res(buf.begin(), buf.end());
return res;
}
2012-08-02 17:33:31 +00:00
bool Context::isTableExist(const String & database_name, const String & table_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
Databases::const_iterator it;
return shared->databases.end() != (it = shared->databases.find(db))
&& it->second.end() != it->second.find(table_name);
}
bool Context::isDatabaseExist(const String & database_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
return shared->databases.end() != shared->databases.find(db);
}
void Context::assertTableExists(const String & database_name, const String & table_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
Databases::const_iterator it;
if (shared->databases.end() == (it = shared->databases.find(db)))
throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
if (it->second.end() == it->second.find(table_name))
throw Exception("Table " + db + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
}
void Context::assertTableDoesntExist(const String & database_name, const String & table_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
Databases::const_iterator it;
if (shared->databases.end() != (it = shared->databases.find(db))
&& it->second.end() != it->second.find(table_name))
throw Exception("Table " + db + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
void Context::assertDatabaseExists(const String & database_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
if (shared->databases.end() == shared->databases.find(db))
throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
}
void Context::assertDatabaseDoesntExist(const String & database_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
if (shared->databases.end() != shared->databases.find(db))
throw Exception("Database " + db + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS);
}
StoragePtr Context::tryGetTemporaryTable(const String & table_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
Tables::const_iterator jt;
if (temporary_tables.end() == (jt = temporary_tables.find(table_name)))
return StoragePtr();
return jt->second;
}
2012-08-02 17:33:31 +00:00
StoragePtr Context::getTable(const String & database_name, const String & table_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
Databases::const_iterator it;
Tables::const_iterator jt;
if (database_name.empty())
{
StoragePtr res;
if (res = tryGetTemporaryTable(table_name))
return res;
if (session_context && (res = session_context->tryGetTemporaryTable(table_name)))
return res;
if (global_context && (res = global_context->tryGetTemporaryTable(table_name)))
return res;
}
2012-08-02 17:33:31 +00:00
String db = database_name.empty() ? current_database : database_name;
if (shared->databases.end() == (it = shared->databases.find(db)))
throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
if (it->second.end() == (jt = it->second.find(table_name)))
throw Exception("Table " + db + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
return jt->second;
}
StoragePtr Context::tryGetTable(const String & database_name, const String & table_name) const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (database_name.empty())
{
StoragePtr res;
if (res = tryGetTemporaryTable(table_name))
return res;
if (session_context && (res = session_context->tryGetTemporaryTable(table_name)))
return res;
if (global_context && (res = global_context->tryGetTemporaryTable(table_name)))
return res;
}
String db = database_name.empty() ? current_database : database_name;
Databases::const_iterator it;
if (shared->databases.end() == (it = shared->databases.find(db)))
return StoragePtr();
Tables::const_iterator jt;
if (it->second.end() == (jt = it->second.find(table_name)))
return StoragePtr();
return jt->second;
}
2012-08-02 17:33:31 +00:00
void Context::addTemporaryTable(const String & table_name, StoragePtr storage)
{
if (temporary_tables.end() != temporary_tables.find(table_name))
throw Exception("Temporary table " + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
temporary_tables[table_name] = storage;
}
2012-08-02 17:33:31 +00:00
void Context::addTable(const String & database_name, const String & table_name, StoragePtr table)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
2012-08-02 17:33:31 +00:00
String db = database_name.empty() ? current_database : database_name;
assertDatabaseExists(db);
assertTableDoesntExist(db, table_name);
shared->databases[db][table_name] = table;
}
void Context::addDatabase(const String & database_name)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
assertDatabaseDoesntExist(db);
shared->databases[db];
}
2013-09-30 01:29:19 +00:00
StoragePtr Context::detachTable(const String & database_name, const String & table_name)
2012-08-02 17:33:31 +00:00
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
assertTableExists(db, table_name);
2013-09-30 01:29:19 +00:00
Tables::iterator it = shared->databases[db].find(table_name);
StoragePtr res = it->second;
shared->databases[db].erase(it);
return res;
2012-08-02 17:33:31 +00:00
}
void Context::detachDatabase(const String & database_name)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
String db = database_name.empty() ? current_database : database_name;
assertDatabaseExists(db);
shared->databases.erase(db);
2012-08-02 17:33:31 +00:00
}
2012-08-17 19:53:11 +00:00
ASTPtr Context::getCreateQuery(const String & database_name, const String & table_name) const
{
StoragePtr table;
String db;
2012-08-17 19:53:11 +00:00
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
db = database_name.empty() ? current_database : database_name;
table = getTable(db, table_name);
}
2012-08-17 19:53:11 +00:00
auto table_lock = table->lockStructure(false);
2012-08-17 19:53:11 +00:00
/// Здесь хранится определение таблицы
String metadata_path = shared->path + "metadata/" + escapeForFileName(db) + "/" + escapeForFileName(table_name) + ".sql";
2012-08-17 19:53:11 +00:00
if (!Poco::File(metadata_path).exists())
{
try
{
/// Если файл .sql не предусмотрен (например, для таблиц типа ChunkRef), то движок может сам предоставить запрос CREATE.
return table->getCustomCreateQuery(*this);
}
catch (...)
{
throw Exception("Metadata file " + metadata_path + " for table " + db + "." + table_name + " doesn't exist.",
ErrorCodes::TABLE_METADATA_DOESNT_EXIST);
}
}
2012-08-17 19:53:11 +00:00
2013-02-11 11:22:15 +00:00
StringPtr query = new String();
2012-08-17 19:53:11 +00:00
{
ReadBufferFromFile in(metadata_path);
2013-02-11 11:22:15 +00:00
WriteBufferFromString out(*query);
2012-08-17 19:53:11 +00:00
copyData(in, out);
}
2013-02-11 11:22:15 +00:00
const char * begin = query->data();
const char * end = begin + query->size();
2012-08-17 19:53:11 +00:00
const char * pos = begin;
ParserCreateQuery parser;
ASTPtr ast;
const char * expected = "";
2012-08-17 19:53:11 +00:00
bool parse_res = parser.parse(pos, end, ast, expected);
/// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой.
if (!parse_res || (pos != end && *pos != ';'))
throw Exception("Syntax error while parsing query from file " + metadata_path + ": failed at position "
2013-06-21 20:34:19 +00:00
+ toString(pos - begin) + ": "
2012-08-17 19:53:11 +00:00
+ std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos))
+ ", expected " + (parse_res ? "end of query" : expected) + ".",
DB::ErrorCodes::SYNTAX_ERROR);
ASTCreateQuery & ast_create_query = dynamic_cast<ASTCreateQuery &>(*ast);
ast_create_query.attach = false;
ast_create_query.database = db;
2013-02-11 11:22:15 +00:00
ast_create_query.query_string = query;
2012-08-17 19:53:11 +00:00
return ast;
}
2012-08-02 17:33:31 +00:00
Settings Context::getSettings() const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return settings;
}
Limits Context::getLimits() const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return settings.limits;
}
2012-08-02 17:33:31 +00:00
void Context::setSettings(const Settings & settings_)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
settings = settings_;
}
2012-08-02 19:03:32 +00:00
void Context::setSetting(const String & name, const Field & value)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (name == "profile")
settings.setProfile(value.safeGet<String>(), *shared->users_config);
else
settings.set(name, value);
}
void Context::setSetting(const String & name, const std::string & value)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (name == "profile")
settings.setProfile(value, *shared->users_config);
else
settings.set(name, value);
2012-08-02 19:03:32 +00:00
}
2012-08-02 17:33:31 +00:00
String Context::getCurrentDatabase() const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return current_database;
}
String Context::getCurrentQueryId() const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return current_query_id;
}
2012-08-02 17:33:31 +00:00
void Context::setCurrentDatabase(const String & name)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
assertDatabaseExists(name);
current_database = name;
}
void Context::setCurrentQueryId(const String & query_id)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
current_query_id = query_id;
}
String Context::getDefaultFormat() const
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
return default_format.empty() ? "TabSeparated" : default_format;
}
void Context::setDefaultFormat(const String & name)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
default_format = name;
}
2012-08-02 17:33:31 +00:00
Context & Context::getSessionContext()
{
if (!session_context)
throw Exception("There is no session", ErrorCodes::THERE_IS_NO_SESSION);
return *session_context;
}
Context & Context::getGlobalContext()
{
if (!global_context)
throw Exception("Logical error: there is no global context", ErrorCodes::LOGICAL_ERROR);
return *global_context;
}
2012-12-19 20:15:15 +00:00
const Dictionaries & Context::getDictionaries() const
2012-12-19 20:15:15 +00:00
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (!shared->dictionaries)
shared->dictionaries = new Dictionaries;
return *shared->dictionaries;
}
void Context::setProgressCallback(ProgressCallback callback)
{
/// Колбек устанавливается на сессию или на запрос. В сессии одновременно обрабатывается только один запрос. Поэтому блокировка не нужна.
progress_callback = callback;
}
ProgressCallback Context::getProgressCallback() const
{
return progress_callback;
}
void Context::setProcessListElement(ProcessList::Element * elem)
{
/// Устанавливается на сессию или на запрос. В сессии одновременно обрабатывается только один запрос. Поэтому блокировка не нужна.
process_list_elem = elem;
}
ProcessList::Element * Context::getProcessListElement()
{
return process_list_elem;
}
void Context::setUncompressedCache(size_t cache_size_in_cells)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (shared->uncompressed_cache)
throw Exception("Uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->uncompressed_cache = new UncompressedCache(cache_size_in_cells);
}
UncompressedCachePtr Context::getUncompressedCache() const
{
/// Исходим из допущения, что функция setUncompressedCache, если вызывалась, то раньше. Иначе поставьте mutex.
return shared->uncompressed_cache;
}
2014-02-11 13:30:42 +00:00
void Context::setMarkCache(size_t cache_size_in_bytes)
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (shared->mark_cache)
throw Exception("Uncompressed cache has been already created.", ErrorCodes::LOGICAL_ERROR);
shared->mark_cache = new MarkCache(cache_size_in_bytes);
}
MarkCachePtr Context::getMarkCache() const
{
return shared->mark_cache;
}
void Context::initClusters()
2013-12-07 16:51:29 +00:00
{
Poco::ScopedLock<Poco::Mutex> lock(shared->mutex);
if (!shared->clusters)
shared->clusters = new Clusters(settings, shared->data_type_factory);
}
2013-12-07 16:51:29 +00:00
Cluster & Context::getCluster(const std::string & cluster_name)
{
if (!shared->clusters)
throw Poco::Exception("Clusters have not been initialized yet.");
Clusters::Impl::iterator it = shared->clusters->impl.find(cluster_name);
if (it != shared->clusters->impl.end())
2013-12-07 16:51:29 +00:00
return it->second;
else
throw Poco::Exception("Failed to find cluster with name = " + cluster_name);
}
2012-08-02 17:33:31 +00:00
}