mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-17 13:13:36 +00:00
add uuid to atomic databases
This commit is contained in:
parent
f4869eca41
commit
383c583bdd
@ -20,6 +20,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_ASSIGN_ALTER;
|
||||
extern const int DATABASE_NOT_EMPTY;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
|
||||
@ -31,13 +32,34 @@ public:
|
||||
};
|
||||
|
||||
|
||||
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_)
|
||||
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, UUID uuid, Context & context_)
|
||||
: DatabaseOrdinary(name_, std::move(metadata_path_), "store/", "DatabaseAtomic (" + name_ + ")", context_)
|
||||
, path_to_table_symlinks(context_.getPath() + "data/" + escapeForFileName(name_) + "/")
|
||||
, path_to_metadata_symlink(global_context.getPath() + "metadata/" + escapeForFileName(name_))
|
||||
, db_uuid(uuid)
|
||||
{
|
||||
/// Symlinks in data/db_name/ directory are not used by ClickHouse,
|
||||
assert(db_uuid != UUIDHelpers::Nil);
|
||||
/// Symlinks in data/db_name/ directory and metadata/db_name/ are not used by ClickHouse,
|
||||
/// it's needed only for convenient introspection.
|
||||
Poco::File(path_to_table_symlinks).createDirectories();
|
||||
assert(path_to_metadata_symlink != metadata_path);
|
||||
Poco::File metadata_symlink(path_to_metadata_symlink);
|
||||
if (metadata_symlink.exists())
|
||||
{
|
||||
if (!metadata_symlink.isLink())
|
||||
throw Exception(ErrorCodes::FILE_ALREADY_EXISTS, "Directory {} exists", path_to_metadata_symlink);
|
||||
}
|
||||
else
|
||||
{
|
||||
try
|
||||
{
|
||||
Poco::File{metadata_path}.linkTo(path_to_metadata_symlink, Poco::File::LINK_SYMBOLIC);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String DatabaseAtomic::getTableDataPath(const String & table_name) const
|
||||
@ -59,7 +81,15 @@ String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const
|
||||
|
||||
void DatabaseAtomic::drop(const Context &)
|
||||
{
|
||||
Poco::File(path_to_table_symlinks).remove(true);
|
||||
try
|
||||
{
|
||||
Poco::File(path_to_metadata_symlink).remove();
|
||||
Poco::File(path_to_table_symlinks).remove(true);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
Poco::File(getMetadataPath()).remove(true);
|
||||
}
|
||||
|
||||
@ -72,7 +102,7 @@ void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table,
|
||||
assertDetachedTableNotInUse(table->getStorageID().uuid);
|
||||
DatabaseWithDictionaries::attachTableUnlocked(name, table, lock);
|
||||
table_name_to_path.emplace(std::make_pair(name, relative_table_path));
|
||||
tryCreateSymlink(name, relative_table_path);
|
||||
//tryCreateSymlink(name, relative_table_path);
|
||||
}
|
||||
|
||||
StoragePtr DatabaseAtomic::detachTable(const String & name)
|
||||
@ -83,7 +113,7 @@ StoragePtr DatabaseAtomic::detachTable(const String & name)
|
||||
table_name_to_path.erase(name);
|
||||
detached_tables.emplace(table->getStorageID().uuid, table);
|
||||
not_in_use = cleenupDetachedTables();
|
||||
tryRemoveSymlink(name);
|
||||
//tryRemoveSymlink(name);
|
||||
return table;
|
||||
}
|
||||
|
||||
|
@ -21,9 +21,10 @@ class DatabaseAtomic : public DatabaseOrdinary
|
||||
{
|
||||
public:
|
||||
|
||||
DatabaseAtomic(String name_, String metadata_path_, Context & context_);
|
||||
DatabaseAtomic(String name_, String metadata_path_, UUID uuid, Context & context_);
|
||||
|
||||
String getEngineName() const override { return "Atomic"; }
|
||||
UUID getUUID() const override { return db_uuid; }
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
@ -69,6 +70,8 @@ private:
|
||||
|
||||
DetachedTables detached_tables;
|
||||
const String path_to_table_symlinks;
|
||||
const String path_to_metadata_symlink;
|
||||
const UUID db_uuid;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -35,14 +35,15 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
DatabasePtr DatabaseFactory::get(
|
||||
const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context)
|
||||
const String & database_name, const String & metadata_path, const ASTStorage * engine_define, UUID uuid, Context & context)
|
||||
{
|
||||
bool created = false;
|
||||
|
||||
try
|
||||
{
|
||||
Poco::File(Poco::Path(metadata_path).makeParent()).createDirectories();
|
||||
created = Poco::File(metadata_path).createDirectory();
|
||||
return getImpl(database_name, metadata_path, engine_define, context);
|
||||
return getImpl(database_name, metadata_path, engine_define, uuid, context);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -65,7 +66,7 @@ static inline ValueType safeGetLiteralValue(const ASTPtr &ast, const String &eng
|
||||
}
|
||||
|
||||
DatabasePtr DatabaseFactory::getImpl(
|
||||
const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context)
|
||||
const String & database_name, const String & metadata_path, const ASTStorage * engine_define, UUID uuid, Context & context)
|
||||
{
|
||||
String engine_name = engine_define->engine->name;
|
||||
|
||||
@ -80,7 +81,7 @@ DatabasePtr DatabaseFactory::getImpl(
|
||||
if (engine_name == "Ordinary")
|
||||
return std::make_shared<DatabaseOrdinary>(database_name, metadata_path, context);
|
||||
else if (engine_name == "Atomic")
|
||||
return std::make_shared<DatabaseAtomic>(database_name, metadata_path, context);
|
||||
return std::make_shared<DatabaseAtomic>(database_name, metadata_path, uuid, context);
|
||||
else if (engine_name == "Memory")
|
||||
return std::make_shared<DatabaseMemory>(database_name, context);
|
||||
else if (engine_name == "Dictionary")
|
||||
|
@ -11,9 +11,9 @@ class ASTStorage;
|
||||
class DatabaseFactory
|
||||
{
|
||||
public:
|
||||
static DatabasePtr get(const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context);
|
||||
static DatabasePtr get(const String & database_name, const String & metadata_path, const ASTStorage * engine_define, UUID uuid, Context & context);
|
||||
|
||||
static DatabasePtr getImpl(const String & database_name, const String & metadata_path, const ASTStorage * engine_define, Context & context);
|
||||
static DatabasePtr getImpl(const String & database_name, const String & metadata_path, const ASTStorage * engine_define, UUID uuid, Context & context);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -332,8 +332,7 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
|
||||
ASTPtr ast;
|
||||
|
||||
auto settings = global_context.getSettingsRef();
|
||||
auto metadata_dir_path = getMetadataPath();
|
||||
auto database_metadata_path = metadata_dir_path.substr(0, metadata_dir_path.size() - 1) + ".sql";
|
||||
auto database_metadata_path = global_context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql";
|
||||
ast = getCreateQueryFromMetadata(database_metadata_path, true);
|
||||
if (!ast)
|
||||
{
|
||||
@ -463,7 +462,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * logger, const Conte
|
||||
return nullptr;
|
||||
|
||||
auto & create = ast->as<ASTCreateQuery &>();
|
||||
if (create.uuid != UUIDHelpers::Nil)
|
||||
if (!create.table.empty() && create.uuid != UUIDHelpers::Nil)
|
||||
{
|
||||
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
|
||||
table_name = unescapeForFileName(table_name);
|
||||
|
@ -283,6 +283,9 @@ public:
|
||||
|
||||
/// Get name of database.
|
||||
String getDatabaseName() const { return database_name; }
|
||||
/// Get UUID of database.
|
||||
virtual UUID getUUID() const { return UUIDHelpers::Nil; }
|
||||
|
||||
/// Returns path for persistent data storage if the database supports it, empty string otherwise
|
||||
virtual String getDataPath() const { return {}; }
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#include <memory>
|
||||
|
||||
#include <Poco/File.h>
|
||||
#include <filesystem>
|
||||
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
@ -76,6 +76,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
namespace fs = std::filesystem;
|
||||
|
||||
InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, Context & context_)
|
||||
: query_ptr(query_ptr_), context(context_)
|
||||
@ -119,20 +120,41 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||
}
|
||||
|
||||
if (create.storage->engine->name == "Atomic" && !context.getSettingsRef().allow_experimental_database_atomic && !internal)
|
||||
throw Exception("Atomic is an experimental database engine. Enable allow_experimental_database_atomic to use it.",
|
||||
ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||
|
||||
String database_name_escaped = escapeForFileName(database_name);
|
||||
String path = context.getPath();
|
||||
String metadata_path = path + "metadata/" + database_name_escaped + "/";
|
||||
DatabasePtr database = DatabaseFactory::get(database_name, metadata_path, create.storage, context);
|
||||
|
||||
/// Will write file with database metadata, if needed.
|
||||
String metadata_file_tmp_path = path + "metadata/" + database_name_escaped + ".sql.tmp";
|
||||
String metadata_file_path = path + "metadata/" + database_name_escaped + ".sql";
|
||||
String database_name_escaped = escapeForFileName(database_name);
|
||||
fs::path metadata_path = fs::canonical(context.getPath());
|
||||
fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp");
|
||||
fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql");
|
||||
|
||||
bool need_write_metadata = !create.attach;
|
||||
if (create.storage->engine->name == "Atomic")
|
||||
{
|
||||
if (!context.getSettingsRef().allow_experimental_database_atomic && !internal)
|
||||
throw Exception("Atomic is an experimental database engine. "
|
||||
"Enable allow_experimental_database_atomic to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||
|
||||
if (create.attach && create.uuid == UUIDHelpers::Nil)
|
||||
throw Exception("UUID must be specified for ATTACH", ErrorCodes::INCORRECT_QUERY);
|
||||
else if (create.uuid == UUIDHelpers::Nil)
|
||||
create.uuid = UUIDHelpers::generateV4();
|
||||
|
||||
metadata_path = metadata_path / "store" / DatabaseCatalog::getPathForUUID(create.uuid);
|
||||
|
||||
if (!create.attach && fs::exists(metadata_path))
|
||||
throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Metadata directory {} already exists", metadata_path);
|
||||
|
||||
create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (create.uuid != UUIDHelpers::Nil)
|
||||
throw Exception("Ordinary database engine does not support UUID", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
metadata_path = metadata_path / "metadata" / database_name_escaped;
|
||||
}
|
||||
|
||||
DatabasePtr database = DatabaseFactory::get(database_name, metadata_path, create.storage, create.uuid, context);
|
||||
|
||||
bool need_write_metadata = !create.attach || !fs::exists(metadata_file_path);
|
||||
|
||||
if (need_write_metadata)
|
||||
{
|
||||
@ -164,7 +186,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
|
||||
if (need_write_metadata)
|
||||
{
|
||||
Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path);
|
||||
fs::rename(metadata_file_tmp_path, metadata_file_path);
|
||||
renamed = true;
|
||||
}
|
||||
|
||||
@ -173,7 +195,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
catch (...)
|
||||
{
|
||||
if (renamed)
|
||||
Poco::File(metadata_file_tmp_path).remove();
|
||||
{
|
||||
[[maybe_unused]] bool removed = fs::remove(metadata_file_tmp_path);
|
||||
assert(removed);
|
||||
}
|
||||
if (added)
|
||||
DatabaseCatalog::instance().detachDatabase(database_name, false, false);
|
||||
|
||||
@ -663,7 +688,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
}
|
||||
|
||||
data_path = database->getTableDataPath(create);
|
||||
if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists())
|
||||
if (!create.attach && !data_path.empty() && fs::exists(fs::path{context.getPath()} / data_path))
|
||||
throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
else
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -95,7 +96,15 @@ void loadMetadata(Context & context, const String & default_database_name)
|
||||
for (Poco::DirectoryIterator it(path); it != dir_end; ++it)
|
||||
{
|
||||
if (!it->isDirectory())
|
||||
{
|
||||
if (endsWith(it.name(), ".sql"))
|
||||
{
|
||||
String db_name = it.name().substr(0, it.name().size() - 4);
|
||||
if (db_name != SYSTEM_DATABASE)
|
||||
databases.emplace(unescapeForFileName(db_name), path + "/" + db_name);
|
||||
}
|
||||
continue;
|
||||
}
|
||||
|
||||
/// For '.svn', '.gitignore' directory and similar.
|
||||
if (it.name().at(0) == '.')
|
||||
|
@ -205,6 +205,13 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
<< (if_not_exists ? "IF NOT EXISTS " : "")
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
<< backQuoteIfNeed(database);
|
||||
|
||||
if (uuid != UUIDHelpers::Nil)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
|
||||
<< quoteString(toString(uuid));
|
||||
}
|
||||
|
||||
formatOnCluster(settings);
|
||||
|
||||
if (storage)
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
#include <Parsers/ParserDictionary.h>
|
||||
#include <Parsers/ParserDictionaryAttributeDeclaration.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -595,6 +596,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr storage;
|
||||
UUID uuid = UUIDHelpers::Nil;
|
||||
|
||||
String cluster_str;
|
||||
bool attach = false;
|
||||
@ -617,6 +619,15 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
if (!name_p.parse(pos, database, expected))
|
||||
return false;
|
||||
|
||||
if (ParserKeyword("UUID").ignore(pos, expected))
|
||||
{
|
||||
ParserStringLiteral uuid_p;
|
||||
ASTPtr ast_uuid;
|
||||
if (!uuid_p.parse(pos, ast_uuid, expected))
|
||||
return false;
|
||||
uuid = parseFromString<UUID>(ast_uuid->as<ASTLiteral>()->value.get<String>());
|
||||
}
|
||||
|
||||
if (ParserKeyword{"ON"}.ignore(pos, expected))
|
||||
{
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
@ -633,6 +644,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
query->if_not_exists = if_not_exists;
|
||||
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
query->uuid = uuid;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
query->set(query->storage, storage);
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Storages/System/StorageSystemDatabases.h>
|
||||
@ -15,6 +16,7 @@ NamesAndTypesList StorageSystemDatabases::getNamesAndTypes()
|
||||
{"engine", std::make_shared<DataTypeString>()},
|
||||
{"data_path", std::make_shared<DataTypeString>()},
|
||||
{"metadata_path", std::make_shared<DataTypeString>()},
|
||||
{"uuid", std::make_shared<DataTypeUUID>()},
|
||||
};
|
||||
}
|
||||
|
||||
@ -33,6 +35,7 @@ void StorageSystemDatabases::fillData(MutableColumns & res_columns, const Contex
|
||||
res_columns[1]->insert(database.second->getEngineName());
|
||||
res_columns[2]->insert(context.getPath() + database.second->getDataPath());
|
||||
res_columns[3]->insert(database.second->getMetadataPath());
|
||||
res_columns[4]->insert(database.second->getUUID());
|
||||
}
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user