remove Context::getTable()

This commit is contained in:
Alexander Tokmakov 2020-03-06 23:38:19 +03:00
parent 2a18ada890
commit 367b358822
15 changed files with 43 additions and 51 deletions

View File

@ -42,7 +42,7 @@ static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & co
++dot;
}
String table_name = join_name.substr(dot);
auto table = context.getTable(database_name, table_name);
auto table = DatabaseCatalog::instance().getTable({database_name, table_name});
auto storage_join = std::dynamic_pointer_cast<StorageJoin>(table);
if (!storage_join)
throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};

View File

@ -113,7 +113,7 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers &
bool has_column;
if (host_name.empty())
{
const StoragePtr & table = global_context.getTable(database_name, table_name);
const StoragePtr & table = DatabaseCatalog::instance().getTable({database_name, table_name});
has_column = table->hasColumn(column_name);
}
else
@ -128,7 +128,7 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers &
global_context.getTCPPort(),
false);
auto remote_columns = getStructureOfRemoteTable(*cluster, database_name, table_name, global_context);
auto remote_columns = getStructureOfRemoteTable(*cluster, {database_name, table_name}, global_context);
has_column = remote_columns.hasPhysical(column_name);
}

View File

@ -834,7 +834,6 @@ const Block & Context::getScalar(const String & name) const
Tables Context::getExternalTables() const
{
//FIXME getTable() may acquire some locks. Better not to call it while holding context lock
auto lock = getLock();
Tables res;
@ -855,17 +854,6 @@ Tables Context::getExternalTables() const
}
StoragePtr Context::getTable(const String & database_name, const String & table_name) const
{
auto resolved_id = resolveStorageID(StorageID(database_name, table_name));
std::optional<Exception> exc;
auto res = DatabaseCatalog::instance().getTableImpl(resolved_id, *this, &exc).second;
if (!res)
throw *exc;
return res;
}
void Context::addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast)
{
auto external_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables();

View File

@ -312,7 +312,6 @@ public:
const Scalars & getScalars() const;
const Block & getScalar(const String & name) const;
Tables getExternalTables() const;
StoragePtr getTable(const String & database_name, const String & table_name) const;
void addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast = {});
void addScalar(const String & name, const Block & block);
bool hasScalar(const String & name) const;

View File

@ -322,8 +322,8 @@ SetPtr SelectQueryExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr &
const auto * table = subquery_or_table_name->as<ASTIdentifier>();
if (!table)
return nullptr;
const DatabaseAndTableWithAlias database_table(*table);
const auto storage = context.getTable(database_table.database, database_table.table);
auto table_id = StorageID::resolveFromAST(subquery_or_table_name, context);
const auto storage = DatabaseCatalog::instance().getTable(table_id);
if (storage->getName() != "Set")
return nullptr;
const auto storage_set = std::dynamic_pointer_cast<StorageSet>(storage);

View File

@ -86,14 +86,13 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
{
const auto & identifier = table_expression.database_and_table_name->as<ASTIdentifier &>();
String database_name;
String table_name;
std::tie(database_name, table_name) = IdentifierSemantic::extractDatabaseAndTable(identifier);
StorageID table_id = StorageID::createEmpty();
std::tie(table_id.database_name, table_id.table_name) = IdentifierSemantic::extractDatabaseAndTable(identifier);
if (!database_name.empty() || !context.isExternalTableExist(table_name))
context.checkAccess(AccessType::SHOW, database_name, table_name);
table_id = context.resolveStorageID(table_id);
context.checkAccess(AccessType::SHOW, table_id);
table = context.getTable(database_name, table_name);
table = DatabaseCatalog::instance().getTable(table_id);
}
auto table_lock = table->lockStructureForShare(false, context.getInitialQueryId());

View File

@ -75,7 +75,8 @@ namespace
if (const auto * identifier = expression.database_and_table_name->as<ASTIdentifier>())
{
const auto & [database, table] = IdentifierSemantic::extractDatabaseAndTable(*identifier);
const auto & storage = data.context.getTable(database.empty() ? data.context.getCurrentDatabase() : database, table);
auto table_id = data.context.resolveStorageID({database, table});
const auto & storage = DatabaseCatalog::instance().getTable(table_id);
if (auto * storage_view = dynamic_cast<StorageView *>(storage.get()))
storage_view->getRuntimeViewQuery(&select_query, data.context, true);

View File

@ -82,7 +82,8 @@ StoragePtr JoinedTables::getLeftTableStorage(Context & context)
if (!storage)
{
/// Read from table. Even without table expression (implicit SELECT ... FROM system.one).
storage = context.getTable(database_name, table_name);
auto table_id = context.resolveStorageID({database_name, table_name});
storage = DatabaseCatalog::instance().getTable(table_id);
}
return storage;

View File

@ -94,8 +94,8 @@ static NamesAndTypesList getColumnsFromTableExpression(const ASTTableExpression
}
else if (table_expression.database_and_table_name)
{
DatabaseAndTableWithAlias database_table(table_expression.database_and_table_name);
const auto & table = context.getTable(database_table.database, database_table.table);
auto table_id = StorageID::resolveFromAST(table_expression.database_and_table_name, context);
const auto & table = DatabaseCatalog::instance().getTable(table_id);
auto & columns = table->getColumns();
names_and_type_list = columns.getOrdinary();
materialized = columns.getMaterialized();

View File

@ -84,7 +84,6 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
const auto select_expression_list = select_query->select();
NamesAndTypesList columns;
/// get columns list for target table
if (function)
{
@ -95,10 +94,10 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
}
else
{
DatabaseAndTableWithAlias database_table(*table);
const auto & storage = context.getTable(database_table.database, database_table.table);
auto table_id = StorageID::resolveFromAST(table_expression, context);
const auto & storage = DatabaseCatalog::instance().getTable(table_id);
columns = storage->getColumns().getOrdinary();
select_query->replaceDatabaseAndTable(database_table.database, database_table.table);
select_query->replaceDatabaseAndTable(table_id.database_name, table_id.table_name);
}
select_expression_list->children.reserve(columns.size());

View File

@ -4,6 +4,7 @@
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Common/quoteString.h>
namespace DB
{
@ -16,6 +17,14 @@ StorageID::StorageID(const ASTQueryWithTableAndOutput & query, const Context & l
assertNotEmpty();
}
String StorageID::getDatabaseName() const
{
assertNotEmpty();
if (database_name.empty())
throw Exception("Database name is empty", ErrorCodes::UNKNOWN_DATABASE);
return database_name;
}
String StorageID::getNameForLogs() const
{
assertNotEmpty();
@ -54,4 +63,9 @@ StorageID StorageID::resolveFromAST(const ASTPtr & table_identifier_node, const
return context.tryResolveStorageID({database_table.database, database_table.table});
}
String StorageID::getFullTableName() const
{
return backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name);
}
}

View File

@ -3,7 +3,6 @@
#include <Core/UUID.h>
#include <tuple>
#include <Parsers/IAST_fwd.h>
#include <Core/QualifiedTableName.h>
namespace DB
@ -12,6 +11,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_DATABASE;
}
static constexpr char const * TABLE_WITH_UUID_NAME_PLACEHOLDER = "_";
@ -36,11 +36,7 @@ struct StorageID
static StorageID resolveFromAST(const ASTPtr & table_identifier_node, const Context & context);
String getDatabaseName() const
{
assertNotEmpty();
return database_name;
}
String getDatabaseName() const;
String getTableName() const
{
@ -48,11 +44,7 @@ struct StorageID
return table_name;
}
String getFullTableName() const
{
assertNotEmpty();
return (database_name.empty() ? "" : database_name + ".") + table_name;
}
String getFullTableName() const;
String getNameForLogs() const;

View File

@ -25,8 +25,7 @@ namespace ErrorCodes
ColumnsDescription getStructureOfRemoteTable(
const Cluster & cluster,
const std::string & database,
const std::string & table,
const StorageID & table_id,
const Context & context,
const ASTPtr & table_func_ptr)
{
@ -50,10 +49,10 @@ ColumnsDescription getStructureOfRemoteTable(
else
{
if (shard_info.isLocal())
return context.getTable(database, table)->getColumns();
return DatabaseCatalog::instance().getTable(table_id)->getColumns();
/// Request for a table description
query = "DESC TABLE " + backQuoteIfNeed(database) + "." + backQuoteIfNeed(table);
query = "DESC TABLE " + table_id.getFullTableName();
}
ColumnsDescription res;
@ -73,7 +72,7 @@ ColumnsDescription getStructureOfRemoteTable(
auto input = std::make_shared<RemoteBlockInputStream>(shard_info.pool, query, sample_block, new_context);
input->setPoolMode(PoolMode::GET_ONE);
if (!table_func_ptr)
input->setMainTable(StorageID{database, table});
input->setMainTable(table_id);
input->readPrefix();
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();

View File

@ -10,13 +10,13 @@ namespace DB
class Cluster;
class Context;
struct StorageID;
/// Find the names and types of the table columns on any server in the cluster.
/// Used to implement the `remote` table function and others.
ColumnsDescription getStructureOfRemoteTable(
const Cluster & cluster,
const std::string & database,
const std::string & table,
const StorageID & table_id,
const Context & context,
const ASTPtr & table_func_ptr = nullptr);

View File

@ -188,7 +188,7 @@ StoragePtr TableFunctionRemote::executeImpl(const ASTPtr & ast_function, const C
secure);
}
auto structure_remote_table = getStructureOfRemoteTable(*cluster, remote_database, remote_table, context, remote_table_function_ptr);
auto structure_remote_table = getStructureOfRemoteTable(*cluster, {remote_database, remote_table}, context, remote_table_function_ptr);
StoragePtr res = remote_table_function_ptr
? StorageDistributed::createWithOwnCluster(