提交 4972daea 编写于 作者: A Alexander Tokmakov

fixes

上级 8b3a245a
......@@ -86,7 +86,6 @@ namespace ErrorCodes
extern const int UNKNOWN_DATABASE;
extern const int UNKNOWN_TABLE;
extern const int TABLE_ALREADY_EXISTS;
extern const int DATABASE_ALREADY_EXISTS;
extern const int THERE_IS_NO_SESSION;
extern const int THERE_IS_NO_QUERY;
extern const int NO_ELEMENTS_IN_CONFIG;
......@@ -818,6 +817,7 @@ void Context::setProfile(const String & profile)
bool Context::isExternalTableExist(const String & table_name) const
{
assert(global_context != this);
auto lock = getLock();
return external_tables_mapping.count(table_name);
}
......@@ -861,6 +861,7 @@ Tables Context::getExternalTables() const
void Context::addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table)
{
assert(global_context != this);
auto lock = getLock();
if (external_tables_mapping.end() != external_tables_mapping.find(table_name))
throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
......@@ -870,9 +871,9 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder &
bool Context::removeExternalTable(const String & table_name)
{
assert(global_context != this);
std::shared_ptr<TemporaryTableHolder> holder;
{
auto lock = getLock();
auto iter = external_tables_mapping.find(table_name);
if (iter == external_tables_mapping.end())
return false;
......@@ -885,12 +886,14 @@ bool Context::removeExternalTable(const String & table_name)
void Context::addScalar(const String & name, const Block & block)
{
assert(global_context != this);
scalars[name] = block;
}
bool Context::hasScalar(const String & name) const
{
assert(global_context != this);
return scalars.count(name);
}
......@@ -2087,19 +2090,30 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w
/// Global context should not contain temporary tables
assert(global_context != this);
/// Firstly look for temporary table in current context
auto it = external_tables_mapping.find(storage_id.getTableName());
if (it != external_tables_mapping.end())
return it->second->getGlobalTableID();
/// If not found and current context was created from some session context, look for temporay table in session context
if (session_context && session_context != this)
auto resolved_id = StorageID::createEmpty();
auto try_resolve = [&](const Context & context) -> bool
{
const auto & external_tables = session_context->external_tables_mapping;
it = external_tables.find(storage_id.getTableName());
if (it != external_tables.end())
return it->second->getGlobalTableID();
}
const auto & tables = context.external_tables_mapping;
auto it = tables.find(storage_id.getTableName());
if (it == tables.end())
return false;
resolved_id = it->second->getGlobalTableID();
return true;
};
/// Firstly look for temporary table in current context
if (try_resolve(*this))
return resolved_id;
/// If not found and current context was created from some query context, look for temporary table in query context
bool is_local_context = query_context && query_context != this;
if (is_local_context && try_resolve(*query_context))
return resolved_id;
/// If not found and current context was created from some session context, look for temporary table in session context
bool is_local_or_query_context = session_context && session_context != this;
if (is_local_or_query_context && try_resolve(*session_context))
return resolved_id;
}
if (in_current_database)
......
......@@ -17,7 +17,6 @@ namespace ErrorCodes
extern const int UNKNOWN_TABLE;
extern const int TABLE_ALREADY_EXISTS;
extern const int DATABASE_ALREADY_EXISTS;
extern const int DDL_GUARD_IS_ACTIVE;
extern const int DATABASE_NOT_EMPTY;
extern const int DATABASE_ACCESS_DENIED;
}
......
......@@ -138,7 +138,7 @@ public:
Dependencies getDependencies(const StorageID & from) const;
/// For Materialized and Live View
void updateDependency(const StorageID & old_from, const StorageID & old_where,const StorageID & new_from, const StorageID & new_where);
void updateDependency(const StorageID & old_from, const StorageID & old_where,const StorageID & new_from, const StorageID & new_where);
private:
DatabaseCatalog(const Context * global_context_);
......
......@@ -78,7 +78,10 @@ public:
if (is_table)
{
/// If this is already an external table, you do not need to add anything. Just remember its presence.
if (external_tables.end() != external_tables.find(getIdentifierName(subquery_or_table_name)))
auto temporary_table_name = getIdentifierName(subquery_or_table_name);
bool exists_in_local_map = external_tables.end() != external_tables.find(temporary_table_name);
bool exists_in_context = !context.tryResolveStorageID(StorageID("", temporary_table_name), Context::ResolveExternal).empty();
if (exists_in_local_map || exists_in_context)
return;
}
......
......@@ -18,7 +18,6 @@ namespace DB
namespace ErrorCodes
{
extern const int DATABASE_NOT_EMPTY;
extern const int LOGICAL_ERROR;
extern const int SYNTAX_ERROR;
extern const int UNKNOWN_TABLE;
......
......@@ -154,8 +154,6 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type,
}
InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, Context & context_)
: query_ptr(query_ptr_->clone()), context(context_), log(&Poco::Logger::get("InterpreterSystemQuery"))
{
......
......@@ -4,11 +4,16 @@
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Common/quoteString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_DATABASE;
}
StorageID::StorageID(const ASTQueryWithTableAndOutput & query, const Context & local_context)
{
database_name = local_context.resolveDatabase(query.database);
......
......@@ -8,12 +8,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_DATABASE;
}
static constexpr char const * TABLE_WITH_UUID_NAME_PLACEHOLDER = "_";
class ASTQueryWithTableAndOutput;
......@@ -70,7 +64,7 @@ struct StorageID
/// Avoid implicit construction of empty StorageID. However, it's needed for deferred initialization.
static StorageID createEmpty() { return {}; }
QualifiedTableName getQualifiedName() const { return {getDatabaseName(), getTableName()}; }
QualifiedTableName getQualifiedName() const { return {database_name, getTableName()}; }
private:
StorageID() = default;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册