mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
enable UUIDs for temporary tables
This commit is contained in:
parent
e98d4f4e5e
commit
158cde68b5
@ -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.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.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;
|
||||
|
||||
|
@ -1,6 +1,7 @@
|
||||
#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;
|
||||
|
@ -1,4 +1,4 @@
|
||||
CREATE DATABASE test_00800;
|
||||
CREATE DATABASE IF NOT EXISTS test_00800;
|
||||
|
||||
USE test_00800;
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user