提交 158cde68 编写于 作者: A Alexander Tokmakov

enable UUIDs for temporary tables

上级 e98d4f4e
......@@ -79,6 +79,14 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
res = it->second;
tables.erase(it);
auto table_id = res->getStorageID();
if (table_id.hasUUID())
{
/// For now it's the only database, which contains storages with UUID
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE);
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
}
return res;
}
......@@ -92,6 +100,13 @@ void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, c
{
if (!tables.emplace(table_name, table).second)
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
auto table_id = table->getStorageID();
if (table_id.hasUUID())
{
/// For now it's the only database, which contains storages with UUID
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE);
DatabaseCatalog::instance().addUUIDMapping(table_id.uuid, shared_from_this(), table);
}
}
void DatabaseWithOwnTablesBase::shutdown()
......
......@@ -480,13 +480,10 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
}
else if (identifier && node.name == "joinGet" && arg == 0)
{
String database_name;
String table_name;
std::tie(database_name, table_name) = IdentifierSemantic::extractDatabaseAndTable(*identifier);
if (database_name.empty())
database_name = data.context.getCurrentDatabase();
auto table_id = IdentifierSemantic::extractDatabaseAndTable(*identifier);
table_id = data.context.resolveStorageID(table_id, Context::ResolveOrdinary);
auto column_string = ColumnString::create();
column_string->insert(database_name + "." + table_name);
column_string->insert(table_id.getDatabaseName() + "." + table_id.getTableName());
ColumnWithTypeAndName column(
ColumnConst::create(std::move(column_string), 1),
std::make_shared<DataTypeString>(),
......
......@@ -832,7 +832,7 @@ const Block & Context::getScalar(const String & name) const
Tables Context::getExternalTables() const
{
assert(global_context != this);
assert(global_context != this || getApplicationType() == ApplicationType::LOCAL);
auto lock = getLock();
Tables res;
......@@ -855,7 +855,7 @@ Tables Context::getExternalTables() const
void Context::addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table)
{
assert(global_context != this);
assert(global_context != this || getApplicationType() == ApplicationType::LOCAL);
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);
......@@ -865,7 +865,7 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder &
bool Context::removeExternalTable(const String & table_name)
{
assert(global_context != this);
assert(global_context != this || getApplicationType() == ApplicationType::LOCAL);
std::shared_ptr<TemporaryTableHolder> holder;
{
auto iter = external_tables_mapping.find(table_name);
......@@ -880,14 +880,14 @@ bool Context::removeExternalTable(const String & table_name)
void Context::addScalar(const String & name, const Block & block)
{
assert(global_context != this);
assert(global_context != this || getApplicationType() == ApplicationType::LOCAL);
scalars[name] = block;
}
bool Context::hasScalar(const String & name) const
{
assert(global_context != this);
assert(global_context != this || getApplicationType() == ApplicationType::LOCAL);
return scalars.count(name);
}
......@@ -2082,7 +2082,7 @@ StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace w
if (look_for_external_table)
{
/// Global context should not contain temporary tables
assert(global_context != this);
assert(global_context != this || getApplicationType() == ApplicationType::LOCAL);
auto resolved_id = StorageID::createEmpty();
auto try_resolve = [&](const Context & context) -> bool
......
......@@ -22,7 +22,8 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & ident
{
alias = identifier.tryGetAlias();
std::tie(database, table) = IdentifierSemantic::extractDatabaseAndTable(identifier);
auto table_id = IdentifierSemantic::extractDatabaseAndTable(identifier);
std::tie(database, table, uuid) = std::tie(table_id.database_name, table_id.table_name, table_id.uuid);
if (database.empty())
database = current_database;
}
......
......@@ -7,6 +7,7 @@
#include <memory>
#include <optional>
#include <Core/UUID.h>
namespace DB
......@@ -24,6 +25,7 @@ struct DatabaseAndTableWithAlias
String database;
String table;
String alias;
UUID uuid = UUIDHelpers::Nil;
DatabaseAndTableWithAlias() = default;
DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
......@@ -39,7 +41,7 @@ struct DatabaseAndTableWithAlias
/// Exactly the same table name
bool same(const DatabaseAndTableWithAlias & db_table) const
{
return database == db_table.database && table == db_table.table && alias == db_table.alias;
return database == db_table.database && table == db_table.table && alias == db_table.alias && uuid == db_table.uuid;
}
};
......
......@@ -156,26 +156,25 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(const StorageID & table_id, cons
return {};
}
//if (table_id.database_name == TEMPORARY_DATABASE && !table_id.hasUUID())
//{
// if (exception)
// exception->emplace("Direct access to `" + String(TEMPORARY_DATABASE) + "` database is not allowed.", ErrorCodes::DATABASE_ACCESS_DENIED);
// return {};
//}
//if (table_id.hasUUID())
//{
// auto db_and_table = tryGetByUUID(table_id.uuid);
// if (!db_and_table.first || !db_and_table.second)
// {
// assert(!db_and_table.first && !db_and_table.second);
// if (exception)
// exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
// return {};
//
// }
// return db_and_table.second;
//}
if (table_id.database_name == TEMPORARY_DATABASE && !table_id.hasUUID())
{
if (exception)
exception->emplace("Direct access to `" + String(TEMPORARY_DATABASE) + "` database is not allowed.", ErrorCodes::DATABASE_ACCESS_DENIED);
return {};
}
if (table_id.hasUUID())
{
auto db_and_table = tryGetByUUID(table_id.uuid);
if (!db_and_table.first || !db_and_table.second)
{
assert(!db_and_table.first && !db_and_table.second);
if (exception)
exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
return {};
}
return db_and_table;
}
DatabasePtr database;
{
......@@ -293,19 +292,17 @@ Databases DatabaseCatalog::getDatabases() const
bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id, const DB::Context & context) const
{
//if (table_id.hasUUID())
// return tryGetByUUID(table_id.uuid).second != nullptr;
//else
//{
DatabasePtr db;
{
std::lock_guard lock{databases_mutex};
auto iter = databases.find(table_id.database_name);
if (iter != databases.end())
db = iter->second;
}
return db && db->isTableExist(context, table_id.table_name);
//}
if (table_id.hasUUID())
return tryGetByUUID(table_id.uuid).second != nullptr;
DatabasePtr db;
{
std::lock_guard lock{databases_mutex};
auto iter = databases.find(table_id.database_name);
if (iter != databases.end())
db = iter->second;
}
return db && db->isTableExist(context, table_id.table_name);
}
void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id, const Context & context) const
......
......@@ -98,7 +98,6 @@ public:
/// Get an object that protects the table from concurrently executing multiple DDL operations.
std::unique_ptr<DDLGuard> getDDLGuard(const String & database, const String & table);
//static String resolveDatabase(const String & database_name, const String & current_database);
void assertDatabaseExists(const String & database_name) const;
void assertDatabaseDoesntExist(const String & database_name) const;
......@@ -148,7 +147,7 @@ private:
mutable std::mutex mutex;
};
static constexpr UInt64 bits_for_first_level = 8;
static constexpr UInt64 bits_for_first_level = 4;
using UUIDToStorageMap = std::array<UUIDToStorageMapPart, 1ull << bits_for_first_level>;
inline size_t getFirstLevelIdx(const UUID & uuid) const
......@@ -162,7 +161,6 @@ private:
ViewDependencies view_dependencies; /// Current dependencies
//const String default_database;
Databases databases;
UUIDToStorageMap uuid_map;
......
#include <Common/typeid_cast.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Storages/StorageID.h>
namespace DB
{
......@@ -136,14 +137,14 @@ std::optional<size_t> IdentifierSemantic::chooseTable(const ASTIdentifier & iden
return tryChooseTable<TableWithColumnNamesAndTypes>(identifier, tables, ambiguous);
}
std::pair<String, String> IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
StorageID IdentifierSemantic::extractDatabaseAndTable(const ASTIdentifier & identifier)
{
if (identifier.name_parts.size() > 2)
throw Exception("Logical error: more than two components in table expression", ErrorCodes::LOGICAL_ERROR);
if (identifier.name_parts.size() == 2)
return { identifier.name_parts[0], identifier.name_parts[1] };
return { "", identifier.name };
return { identifier.name_parts[0], identifier.name_parts[1], identifier.uuid };
return { "", identifier.name, identifier.uuid };
}
std::optional<String> IdentifierSemantic::extractNestedName(const ASTIdentifier & identifier, const String & table_name)
......
......@@ -37,7 +37,7 @@ struct IdentifierSemantic
/// @returns name for 'not a column' identifiers
static std::optional<String> getTableName(const ASTIdentifier & node);
static std::optional<String> getTableName(const ASTPtr & ast);
static std::pair<String, String> extractDatabaseAndTable(const ASTIdentifier & identifier);
static StorageID extractDatabaseAndTable(const ASTIdentifier & identifier);
static std::optional<String> extractNestedName(const ASTIdentifier & identifier, const String & table_name);
static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table);
......
......@@ -86,8 +86,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
{
const auto & identifier = table_expression.database_and_table_name->as<ASTIdentifier &>();
StorageID table_id = StorageID::createEmpty();
std::tie(table_id.database_name, table_id.table_name) = IdentifierSemantic::extractDatabaseAndTable(identifier);
StorageID table_id = IdentifierSemantic::extractDatabaseAndTable(identifier);
table_id = context.resolveStorageID(table_id);
context.checkAccess(AccessType::SHOW, table_id);
......
......@@ -74,8 +74,8 @@ namespace
{
if (const auto * identifier = expression.database_and_table_name->as<ASTIdentifier>())
{
const auto & [database, table] = IdentifierSemantic::extractDatabaseAndTable(*identifier);
auto table_id = data.context.resolveStorageID({database, table});
auto table_id = IdentifierSemantic::extractDatabaseAndTable(*identifier);
table_id = data.context.resolveStorageID(table_id);
const auto & storage = DatabaseCatalog::instance().getTable(table_id);
if (auto * storage_view = dynamic_cast<StorageView *>(storage.get()))
......
......@@ -68,7 +68,7 @@ StoragePtr JoinedTables::getLeftTableStorage()
if (left_db_and_table)
{
table_id = context.resolveStorageID(StorageID(left_db_and_table->database, left_db_and_table->table));
table_id = context.resolveStorageID(StorageID(left_db_and_table->database, left_db_and_table->table, left_db_and_table->uuid));
}
else /// If the table is not specified - use the table `system.one`.
{
......
......@@ -97,7 +97,7 @@ std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
auto table_id = StorageID::resolveFromAST(table_expression, context);
const auto & storage = DatabaseCatalog::instance().getTable(table_id);
columns = storage->getColumns().getOrdinary();
select_query->replaceDatabaseAndTable(table_id.database_name, table_id.table_name);
select_query->replaceDatabaseAndTable(table_id);
}
select_expression_list->children.reserve(columns.size());
......
......@@ -4,6 +4,7 @@
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Storages/StorageID.h>
namespace DB
......@@ -102,11 +103,19 @@ void ASTIdentifier::appendColumnNameImpl(WriteBuffer & ostr) const
ASTPtr createTableIdentifier(const String & database_name, const String & table_name)
{
if (database_name.empty())
return ASTIdentifier::createSpecial(table_name);
assert(database_name != "_temporary_and_external_tables");
return createTableIdentifier(StorageID(database_name, table_name));
}
ASTPtr database_and_table = ASTIdentifier::createSpecial(database_name + "." + table_name, {database_name, table_name});
return database_and_table;
ASTPtr createTableIdentifier(const StorageID & table_id)
{
std::shared_ptr<ASTIdentifier> res;
if (table_id.database_name.empty())
res = ASTIdentifier::createSpecial(table_id.table_name);
else
res = ASTIdentifier::createSpecial(table_id.database_name + "." + table_id.table_name, {table_id.database_name, table_id.table_name});
res->uuid = table_id.uuid;
return res;
}
String getIdentifierName(const IAST * ast)
......
......@@ -3,6 +3,7 @@
#include <optional>
#include <Parsers/ASTWithAlias.h>
#include <Core/UUID.h>
namespace DB
......@@ -11,6 +12,7 @@ namespace DB
struct IdentifierSemantic;
struct IdentifierSemanticImpl;
struct DatabaseAndTableWithAlias;
struct StorageID;
/// Identifier (column, table or alias)
......@@ -20,6 +22,7 @@ public:
/// The composite identifier will have a concatenated name (of the form a.b.c),
/// and individual components will be available inside the name_parts.
String name;
UUID uuid = UUIDHelpers::Nil;
ASTIdentifier(const String & name_, std::vector<String> && name_parts_ = {});
ASTIdentifier(std::vector<String> && name_parts_);
......@@ -60,7 +63,7 @@ private:
static std::shared_ptr<ASTIdentifier> createSpecial(const String & name, std::vector<String> && name_parts = {});
friend struct IdentifierSemantic;
friend ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
friend ASTPtr createTableIdentifier(const StorageID & table_id);
friend void setIdentifierSpecial(ASTPtr & ast);
};
......@@ -68,6 +71,7 @@ private:
/// ASTIdentifier Helpers: hide casts and semantic.
ASTPtr createTableIdentifier(const String & database_name, const String & table_name);
ASTPtr createTableIdentifier(const StorageID & table_id);
void setIdentifierSpecial(ASTPtr & ast);
String getIdentifierName(const IAST * ast);
......
......@@ -5,6 +5,7 @@
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Storages/StorageID.h>
namespace DB
......@@ -326,6 +327,12 @@ static String getTableExpressionAlias(const ASTTableExpression * table_expressio
}
void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const String & table_name)
{
assert(database_name != "_temporary_and_external_tables");
replaceDatabaseAndTable(StorageID(database_name, table_name));
}
void ASTSelectQuery::replaceDatabaseAndTable(const StorageID & table_id)
{
ASTTableExpression * table_expression = getFirstTableExpression(*this);
......@@ -341,7 +348,7 @@ void ASTSelectQuery::replaceDatabaseAndTable(const String & database_name, const
}
String table_alias = getTableExpressionAlias(table_expression);
table_expression->database_and_table_name = createTableIdentifier(database_name, table_name);
table_expression->database_and_table_name = createTableIdentifier(table_id);
if (!table_alias.empty())
table_expression->database_and_table_name->setAlias(table_alias);
......
......@@ -8,6 +8,7 @@ namespace DB
{
struct ASTTablesInSelectQueryElement;
struct StorageID;
/** SELECT query
......@@ -85,6 +86,7 @@ public:
bool final() const;
bool withFill() const;
void replaceDatabaseAndTable(const String & database_name, const String & table_name);
void replaceDatabaseAndTable(const StorageID & table_id);
void addTableFunction(ASTPtr & table_function_ptr);
protected:
......
......@@ -65,7 +65,7 @@ void StorageID::assertNotEmpty() const
StorageID StorageID::resolveFromAST(const ASTPtr & table_identifier_node, const Context & context)
{
DatabaseAndTableWithAlias database_table(table_identifier_node);
return context.tryResolveStorageID({database_table.database, database_table.table});
return context.tryResolveStorageID({database_table.database, database_table.table, database_table.uuid});
}
String StorageID::getFullTableName() const
......
......@@ -3,9 +3,9 @@ CREATE TEMPORARY TABLE temp_tab (number UInt64);
INSERT INTO temp_tab SELECT number FROM system.numbers LIMIT 1;
SELECT number FROM temp_tab;
SET send_logs_level = 'none';
EXISTS temp_tab;
EXISTS TEMPORARY TABLE temp_tab;
DROP TABLE temp_tab;
EXISTS temp_tab;
EXISTS TEMPORARY TABLE temp_tab;
SET send_logs_level = 'warning';
CREATE TEMPORARY TABLE temp_tab (number UInt64);
SELECT number FROM temp_tab;
......
CREATE DATABASE test_00800;
CREATE DATABASE IF NOT EXISTS test_00800;
USE test_00800;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册