mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #7512 from ClickHouse/database_atomic
DatabaseAtomic
This commit is contained in:
commit
7928f2eaa1
@ -349,7 +349,7 @@ create view addresses_src as select *
|
||||
from file('$version-addresses.tsv', TSVWithNamesAndTypes,
|
||||
'$(cat "$version-addresses.tsv.columns")');
|
||||
|
||||
create table addresses_join engine Join(any, left, address) as
|
||||
create table addresses_join_$version engine Join(any, left, address) as
|
||||
select addr address, name from addresses_src;
|
||||
|
||||
create table unstable_query_runs engine File(TSVWithNamesAndTypes,
|
||||
@ -388,7 +388,7 @@ create table unstable_run_traces engine File(TSVWithNamesAndTypes,
|
||||
'unstable-run-traces.$version.rep') as
|
||||
select
|
||||
count() value,
|
||||
joinGet(addresses_join, 'name', arrayJoin(trace)) metric,
|
||||
joinGet(addresses_join_$version, 'name', arrayJoin(trace)) metric,
|
||||
unstable_query_runs.query_id,
|
||||
any(unstable_query_runs.query) query
|
||||
from unstable_query_runs
|
||||
@ -414,7 +414,7 @@ create table stacks engine File(TSV, 'stacks.$version.rep') as
|
||||
select
|
||||
query,
|
||||
arrayStringConcat(
|
||||
arrayMap(x -> joinGet(addresses_join, 'name', x),
|
||||
arrayMap(x -> joinGet(addresses_join_$version, 'name', x),
|
||||
arrayReverse(trace)
|
||||
),
|
||||
';'
|
||||
|
@ -44,7 +44,7 @@ void ReplicasStatusHandler::handleRequest(Poco::Net::HTTPServerRequest & request
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
auto & table = iterator->table();
|
||||
StorageReplicatedMergeTree * table_replicated = dynamic_cast<StorageReplicatedMergeTree *>(table.get());
|
||||
|
@ -310,10 +310,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
|
||||
global_context->setPath(path);
|
||||
|
||||
/// Create directories for 'path' and for default database, if not exist.
|
||||
Poco::File(path + "data/" + default_database).createDirectories();
|
||||
Poco::File(path + "metadata/" + default_database).createDirectories();
|
||||
|
||||
StatusFile status{path + "status"};
|
||||
|
||||
SCOPE_EXIT({
|
||||
@ -404,6 +400,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
Poco::File(dictionaries_lib_path).createDirectories();
|
||||
}
|
||||
|
||||
{
|
||||
/// Directory with metadata of tables, which was marked as dropped by Atomic database
|
||||
Poco::File(path + "metadata_dropped/").createDirectories();
|
||||
}
|
||||
|
||||
if (config().has("interserver_http_port") && config().has("interserver_https_port"))
|
||||
throw Exception("Both http and https interserver ports are specified", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG);
|
||||
|
||||
@ -593,7 +594,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
/// After the system database is created, attach virtual system tables (in addition to query_log and part_log)
|
||||
attachSystemTablesServer(*DatabaseCatalog::instance().getSystemDatabase(), has_zookeeper);
|
||||
/// Then, load remaining databases
|
||||
loadMetadata(*global_context);
|
||||
loadMetadata(*global_context, default_database);
|
||||
DatabaseCatalog::instance().loadDatabases();
|
||||
}
|
||||
catch (...)
|
||||
|
@ -492,6 +492,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_COMMIT_OFFSET = 518;
|
||||
extern const int NO_REMOTE_SHARD_AVAILABLE = 519;
|
||||
extern const int CANNOT_DETACH_DICTIONARY_AS_TABLE = 520;
|
||||
extern const int ATOMIC_RENAME_FAIL = 521;
|
||||
|
||||
extern const int KEEPER_EXCEPTION = 999;
|
||||
extern const int POCO_EXCEPTION = 1000;
|
||||
|
@ -79,6 +79,7 @@ using Exceptions = std::vector<std::exception_ptr>;
|
||||
|
||||
std::string errnoToString(int code, int the_errno = errno);
|
||||
[[noreturn]] void throwFromErrno(const std::string & s, int code, int the_errno = errno);
|
||||
/// Useful to produce some extra information about available space and inodes on device
|
||||
[[noreturn]] void throwFromErrnoWithPath(const std::string & s, const std::string & path, int code,
|
||||
int the_errno = errno);
|
||||
|
||||
|
122
src/Common/renameat2.cpp
Normal file
122
src/Common/renameat2.cpp
Normal file
@ -0,0 +1,122 @@
|
||||
#include <Common/renameat2.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
#if defined(linux) || defined(__linux) || defined(__linux__)
|
||||
#include <unistd.h>
|
||||
#include <sys/syscall.h>
|
||||
#include <linux/fs.h>
|
||||
#include <sys/utsname.h>
|
||||
#endif
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ATOMIC_RENAME_FAIL;
|
||||
extern const int SYSTEM_ERROR;
|
||||
extern const int UNSUPPORTED_METHOD;
|
||||
extern const int FILE_ALREADY_EXISTS;
|
||||
}
|
||||
|
||||
static bool supportsRenameat2Impl()
|
||||
{
|
||||
#if defined(__NR_renameat2)
|
||||
/// renameat2 is available in linux since 3.15
|
||||
struct utsname sysinfo;
|
||||
if (uname(&sysinfo))
|
||||
return false;
|
||||
char * point = nullptr;
|
||||
auto v_major = strtol(sysinfo.release, &point, 10);
|
||||
|
||||
errno = 0;
|
||||
if (errno || *point != '.' || v_major < 3)
|
||||
return false;
|
||||
if (3 < v_major)
|
||||
return true;
|
||||
|
||||
errno = 0;
|
||||
auto v_minor = strtol(point + 1, nullptr, 10);
|
||||
return !errno && 15 <= v_minor;
|
||||
#else
|
||||
return false;
|
||||
#endif
|
||||
}
|
||||
|
||||
#if defined(__NR_renameat2)
|
||||
|
||||
static void renameat2(const std::string & old_path, const std::string & new_path, int flags)
|
||||
{
|
||||
if (old_path.empty() || new_path.empty())
|
||||
throw Exception("Cannot rename " + old_path + " to " + new_path + ": path is empty", ErrorCodes::LOGICAL_ERROR);
|
||||
if (old_path[0] != '/' || new_path[0] != '/')
|
||||
throw Exception("Cannot rename " + old_path + " to " + new_path + ": path is relative", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// int olddirfd (ignored for absolute oldpath), const char *oldpath,
|
||||
/// int newdirfd (ignored for absolute newpath), const char *newpath,
|
||||
/// unsigned int flags
|
||||
if (0 == syscall(__NR_renameat2, 0, old_path.c_str(), 0, new_path.c_str(), flags))
|
||||
return;
|
||||
|
||||
if (errno == EEXIST)
|
||||
throwFromErrno("Cannot rename " + old_path + " to " + new_path + " because the second path already exists", ErrorCodes::ATOMIC_RENAME_FAIL);
|
||||
if (errno == ENOENT)
|
||||
throwFromErrno("Paths cannot be exchanged because " + old_path + " or " + new_path + " does not exist", ErrorCodes::ATOMIC_RENAME_FAIL);
|
||||
throwFromErrnoWithPath("Cannot rename " + old_path + " to " + new_path, new_path, ErrorCodes::SYSTEM_ERROR);
|
||||
}
|
||||
|
||||
#else
|
||||
#define RENAME_NOREPLACE -1
|
||||
#define RENAME_EXCHANGE -1
|
||||
|
||||
[[noreturn]]
|
||||
static void renameat2(const std::string &, const std::string &, int)
|
||||
{
|
||||
throw Exception("Compiled without renameat2() support", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
static void renameNoReplaceFallback(const std::string & old_path, const std::string & new_path)
|
||||
{
|
||||
/// NOTE it's unsafe
|
||||
if (Poco::File{new_path}.exists())
|
||||
throw Exception("File " + new_path + " exists", ErrorCodes::FILE_ALREADY_EXISTS);
|
||||
Poco::File{old_path}.renameTo(new_path);
|
||||
}
|
||||
|
||||
/// Do not use [[noreturn]] to avoid warnings like "code will never be executed" in other places
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wmissing-noreturn"
|
||||
static void renameExchangeFallback(const std::string &, const std::string &)
|
||||
{
|
||||
throw Exception("System call renameat2() is not supported", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
#pragma GCC diagnostic pop
|
||||
|
||||
|
||||
bool supportsRenameat2()
|
||||
{
|
||||
static bool supports = supportsRenameat2Impl();
|
||||
return supports;
|
||||
}
|
||||
|
||||
void renameNoReplace(const std::string & old_path, const std::string & new_path)
|
||||
{
|
||||
if (supportsRenameat2())
|
||||
renameat2(old_path, new_path, RENAME_NOREPLACE);
|
||||
else
|
||||
renameNoReplaceFallback(old_path, new_path);
|
||||
}
|
||||
|
||||
void renameExchange(const std::string & old_path, const std::string & new_path)
|
||||
{
|
||||
if (supportsRenameat2())
|
||||
renameat2(old_path, new_path, RENAME_EXCHANGE);
|
||||
else
|
||||
renameExchangeFallback(old_path, new_path);
|
||||
}
|
||||
|
||||
}
|
17
src/Common/renameat2.h
Normal file
17
src/Common/renameat2.h
Normal file
@ -0,0 +1,17 @@
|
||||
#pragma once
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Returns true, if the following functions supported by the system
|
||||
bool supportsRenameat2();
|
||||
|
||||
/// Atomically rename old_path to new_path. If new_path exists, do not overwrite it and throw exception
|
||||
void renameNoReplace(const std::string & old_path, const std::string & new_path);
|
||||
|
||||
/// Atomically exchange oldpath and newpath. Throw exception if some of them does not exist
|
||||
void renameExchange(const std::string & old_path, const std::string & new_path);
|
||||
|
||||
}
|
@ -78,6 +78,7 @@ SRCS(
|
||||
quoteString.cpp
|
||||
randomSeed.cpp
|
||||
RemoteHostFilter.cpp
|
||||
renameat2.cpp
|
||||
RWLock.cpp
|
||||
SensitiveDataMasker.cpp
|
||||
setThreadName.cpp
|
||||
|
@ -394,6 +394,8 @@ struct Settings : public SettingsCollection<Settings>
|
||||
M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \
|
||||
M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \
|
||||
\
|
||||
M(SettingDefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \
|
||||
M(SettingBool, allow_experimental_database_atomic, false, "Allow to create database with Engine=Atomic.", 0) \
|
||||
M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \
|
||||
M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
|
||||
M(SettingUInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
|
||||
|
@ -550,6 +550,12 @@ IMPLEMENT_SETTING_ENUM(LogsLevel, LOGS_LEVEL_LIST_OF_NAMES, ErrorCodes::BAD_ARGU
|
||||
IMPLEMENT_SETTING_ENUM(QueryLogElementType, LOG_QUERIES_TYPE_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS)
|
||||
|
||||
|
||||
#define DEFAULT_DATABASE_ENGINE_LIST_OF_NAMES(M) \
|
||||
M(Ordinary, "Ordinary") \
|
||||
M(Atomic, "Atomic")
|
||||
IMPLEMENT_SETTING_ENUM(DefaultDatabaseEngine , DEFAULT_DATABASE_ENGINE_LIST_OF_NAMES, ErrorCodes::BAD_ARGUMENTS)
|
||||
|
||||
|
||||
namespace details
|
||||
{
|
||||
void SettingsCollectionUtils::serializeName(const StringRef & name, WriteBuffer & buf)
|
||||
|
@ -298,6 +298,13 @@ enum class LogsLevel
|
||||
};
|
||||
using SettingLogsLevel = SettingEnum<LogsLevel>;
|
||||
|
||||
enum class DefaultDatabaseEngine
|
||||
{
|
||||
Ordinary,
|
||||
Atomic,
|
||||
};
|
||||
using SettingDefaultDatabaseEngine = SettingEnum<DefaultDatabaseEngine>;
|
||||
|
||||
// Make it signed for compatibility with DataTypeEnum8
|
||||
enum QueryLogElementType : int8_t
|
||||
{
|
||||
|
349
src/Databases/DatabaseAtomic.cpp
Normal file
349
src/Databases/DatabaseAtomic.cpp
Normal file
@ -0,0 +1,349 @@
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
#include <Databases/DatabaseOnDisk.h>
|
||||
#include <Poco/File.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <Common/renameat2.h>
|
||||
#include <Storages/StorageMaterializedView.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int CANNOT_ASSIGN_ALTER;
|
||||
extern const int DATABASE_NOT_EMPTY;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
|
||||
{
|
||||
public:
|
||||
explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base)
|
||||
: DatabaseTablesSnapshotIterator(std::move(base)) {}
|
||||
UUID uuid() const override { return table()->getStorageID().uuid; }
|
||||
};
|
||||
|
||||
|
||||
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_)
|
||||
: DatabaseOrdinary(name_, std::move(metadata_path_), "store/", "DatabaseAtomic (" + name_ + ")", context_)
|
||||
, path_to_table_symlinks(context_.getPath() + "data/" + escapeForFileName(name_) + "/")
|
||||
{
|
||||
/// Symlinks in data/db_name/ directory are not used by ClickHouse,
|
||||
/// it's needed only for convenient introspection.
|
||||
Poco::File(path_to_table_symlinks).createDirectories();
|
||||
}
|
||||
|
||||
String DatabaseAtomic::getTableDataPath(const String & table_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = table_name_to_path.find(table_name);
|
||||
if (it == table_name_to_path.end())
|
||||
throw Exception("Table " + table_name + " not found in database " + getDatabaseName(), ErrorCodes::UNKNOWN_TABLE);
|
||||
assert(it->second != data_path && !it->second.empty());
|
||||
return it->second;
|
||||
}
|
||||
|
||||
String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const
|
||||
{
|
||||
auto tmp = data_path + DatabaseCatalog::getPathForUUID(query.uuid);
|
||||
assert(tmp != data_path && !tmp.empty());
|
||||
return tmp;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::drop(const Context &)
|
||||
{
|
||||
Poco::File(path_to_table_symlinks).remove(true);
|
||||
Poco::File(getMetadataPath()).remove(true);
|
||||
}
|
||||
|
||||
void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path)
|
||||
{
|
||||
assert(relative_table_path != data_path && !relative_table_path.empty());
|
||||
DetachedTables not_in_use;
|
||||
std::unique_lock lock(mutex);
|
||||
not_in_use = cleenupDetachedTables();
|
||||
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);
|
||||
}
|
||||
|
||||
StoragePtr DatabaseAtomic::detachTable(const String & name)
|
||||
{
|
||||
DetachedTables not_in_use;
|
||||
std::unique_lock lock(mutex);
|
||||
auto table = DatabaseWithDictionaries::detachTableUnlocked(name, lock);
|
||||
table_name_to_path.erase(name);
|
||||
detached_tables.emplace(table->getStorageID().uuid, table);
|
||||
not_in_use = cleenupDetachedTables();
|
||||
tryRemoveSymlink(name);
|
||||
return table;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool no_delay)
|
||||
{
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
String table_metadata_path_drop;
|
||||
StoragePtr table;
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
table = getTableUnlocked(table_name, lock);
|
||||
table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
|
||||
Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); /// Mark table as dropped
|
||||
DatabaseWithDictionaries::detachTableUnlocked(table_name, lock); /// Should never throw
|
||||
table_name_to_path.erase(table_name);
|
||||
}
|
||||
tryRemoveSymlink(table_name);
|
||||
/// Notify DatabaseCatalog that table was dropped. It will remove table data in background.
|
||||
/// Cleanup is performed outside of database to allow easily DROP DATABASE without waiting for cleanup to complete.
|
||||
DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay);
|
||||
}
|
||||
|
||||
void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
|
||||
const String & to_table_name, bool exchange)
|
||||
{
|
||||
if (typeid(*this) != typeid(to_database))
|
||||
{
|
||||
if (!typeid_cast<DatabaseOrdinary *>(&to_database))
|
||||
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
/// Allow moving tables between Atomic and Ordinary (with table lock)
|
||||
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange);
|
||||
return;
|
||||
}
|
||||
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);
|
||||
bool inside_database = this == &other_db;
|
||||
|
||||
String old_metadata_path = getObjectMetadataPath(table_name);
|
||||
String new_metadata_path = to_database.getObjectMetadataPath(to_table_name);
|
||||
|
||||
auto detach = [](DatabaseAtomic & db, const String & table_name_)
|
||||
{
|
||||
auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second;
|
||||
db.tables.erase(table_name_);
|
||||
db.table_name_to_path.erase(table_name_);
|
||||
db.tryRemoveSymlink(table_name_);
|
||||
return table_data_path_saved;
|
||||
};
|
||||
|
||||
auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
|
||||
{
|
||||
db.tables.emplace(table_name_, table_);
|
||||
db.table_name_to_path.emplace(table_name_, table_data_path_);
|
||||
db.tryCreateSymlink(table_name_, table_data_path_);
|
||||
};
|
||||
|
||||
auto assert_can_move_mat_view = [inside_database](const StoragePtr & table_)
|
||||
{
|
||||
if (inside_database)
|
||||
return;
|
||||
if (const auto * mv = dynamic_cast<const StorageMaterializedView *>(table_.get()))
|
||||
if (mv->hasInnerTable())
|
||||
throw Exception("Cannot move MaterializedView with inner table to other database", ErrorCodes::NOT_IMPLEMENTED);
|
||||
};
|
||||
|
||||
String table_data_path;
|
||||
String other_table_data_path;
|
||||
|
||||
if (inside_database && table_name == to_table_name)
|
||||
return;
|
||||
|
||||
std::unique_lock<std::mutex> db_lock;
|
||||
std::unique_lock<std::mutex> other_db_lock;
|
||||
if (inside_database)
|
||||
db_lock = std::unique_lock{mutex};
|
||||
else if (this < &other_db)
|
||||
{
|
||||
db_lock = std::unique_lock{mutex};
|
||||
other_db_lock = std::unique_lock{other_db.mutex};
|
||||
}
|
||||
else
|
||||
{
|
||||
other_db_lock = std::unique_lock{other_db.mutex};
|
||||
db_lock = std::unique_lock{mutex};
|
||||
}
|
||||
|
||||
StoragePtr table = getTableUnlocked(table_name, db_lock);
|
||||
assert_can_move_mat_view(table);
|
||||
StoragePtr other_table;
|
||||
if (exchange)
|
||||
{
|
||||
other_table = other_db.getTableUnlocked(to_table_name, other_db_lock);
|
||||
assert_can_move_mat_view(other_table);
|
||||
}
|
||||
|
||||
/// Table renaming actually begins here
|
||||
if (exchange)
|
||||
renameExchange(old_metadata_path, new_metadata_path);
|
||||
else
|
||||
renameNoReplace(old_metadata_path, new_metadata_path);
|
||||
|
||||
/// After metadata was successfully moved, the following methods should not throw (if them do, it's a logical error)
|
||||
table_data_path = detach(*this, table_name);
|
||||
if (exchange)
|
||||
other_table_data_path = detach(other_db, to_table_name);
|
||||
|
||||
table->renameInMemory({other_db.getDatabaseName(), to_table_name, table->getStorageID().uuid});
|
||||
if (exchange)
|
||||
other_table->renameInMemory({getDatabaseName(), table_name, other_table->getStorageID().uuid});
|
||||
|
||||
if (!inside_database)
|
||||
{
|
||||
DatabaseCatalog::instance().updateUUIDMapping(table->getStorageID().uuid, other_db.shared_from_this(), table);
|
||||
if (exchange)
|
||||
DatabaseCatalog::instance().updateUUIDMapping(other_table->getStorageID().uuid, shared_from_this(), other_table);
|
||||
}
|
||||
|
||||
attach(other_db, to_table_name, table_data_path, table);
|
||||
if (exchange)
|
||||
attach(*this, table_name, other_table_data_path, other_table);
|
||||
}
|
||||
|
||||
void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path)
|
||||
{
|
||||
DetachedTables not_in_use;
|
||||
auto table_data_path = getTableDataPath(query);
|
||||
try
|
||||
{
|
||||
std::unique_lock lock{mutex};
|
||||
not_in_use = cleenupDetachedTables();
|
||||
assertDetachedTableNotInUse(query.uuid);
|
||||
renameNoReplace(table_metadata_tmp_path, table_metadata_path);
|
||||
attachTableUnlocked(query.table, table, lock); /// Should never throw
|
||||
table_name_to_path.emplace(query.table, table_data_path);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
Poco::File(table_metadata_tmp_path).remove();
|
||||
throw;
|
||||
}
|
||||
tryCreateSymlink(query.table, table_data_path);
|
||||
}
|
||||
|
||||
void DatabaseAtomic::commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path)
|
||||
{
|
||||
SCOPE_EXIT({ Poco::File(table_metadata_tmp_path).remove(); });
|
||||
|
||||
std::unique_lock lock{mutex};
|
||||
auto actual_table_id = getTableUnlocked(table_id.table_name, lock)->getStorageID();
|
||||
|
||||
if (table_id.uuid != actual_table_id.uuid)
|
||||
throw Exception("Cannot alter table because it was renamed", ErrorCodes::CANNOT_ASSIGN_ALTER);
|
||||
|
||||
renameExchange(table_metadata_tmp_path, table_metadata_path);
|
||||
}
|
||||
|
||||
void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid)
|
||||
{
|
||||
/// Without this check the following race is possible since table RWLocks are not used:
|
||||
/// 1. INSERT INTO table ...;
|
||||
/// 2. DETACH TABLE table; (INSERT still in progress, it holds StoragePtr)
|
||||
/// 3. ATTACH TABLE table; (new instance of Storage with the same UUID is created, instances share data on disk)
|
||||
/// 4. INSERT INTO table ...; (both Storage instances writes data without any synchronization)
|
||||
/// To avoid it, we remember UUIDs of detached tables and does not allow ATTACH table with such UUID until detached instance still in use.
|
||||
if (detached_tables.count(uuid))
|
||||
throw Exception("Cannot attach table with UUID " + toString(uuid) +
|
||||
", because it was detached but still used by come query. Retry later.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
DatabaseAtomic::DetachedTables DatabaseAtomic::cleenupDetachedTables()
|
||||
{
|
||||
DetachedTables not_in_use;
|
||||
auto it = detached_tables.begin();
|
||||
while (it != detached_tables.end())
|
||||
{
|
||||
if (it->second.unique())
|
||||
{
|
||||
not_in_use.emplace(it->first, it->second);
|
||||
it = detached_tables.erase(it);
|
||||
}
|
||||
else
|
||||
++it;
|
||||
}
|
||||
/// It should be destroyed in caller with released database mutex
|
||||
return not_in_use;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::assertCanBeDetached(bool cleenup)
|
||||
{
|
||||
if (cleenup)
|
||||
{
|
||||
DetachedTables not_in_use;
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
not_in_use = cleenupDetachedTables();
|
||||
}
|
||||
}
|
||||
std::lock_guard lock(mutex);
|
||||
if (!detached_tables.empty())
|
||||
throw Exception("Database " + backQuoteIfNeed(database_name) + " cannot be detached, "
|
||||
"because some tables are still in use. Retry later.", ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseAtomic::getTablesIterator(const IDatabase::FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
auto base_iter = DatabaseWithOwnTablesBase::getTablesIterator(filter_by_table_name);
|
||||
return std::make_unique<AtomicDatabaseTablesSnapshotIterator>(std::move(typeid_cast<DatabaseTablesSnapshotIterator &>(*base_iter)));
|
||||
}
|
||||
|
||||
UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const
|
||||
{
|
||||
if (auto table = tryGetTable(table_name))
|
||||
return table->getStorageID().uuid;
|
||||
return UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
void DatabaseAtomic::loadStoredObjects(Context & context, bool has_force_restore_data_flag)
|
||||
{
|
||||
/// Recreate symlinks to table data dirs in case of force restore, because some of them may be broken
|
||||
if (has_force_restore_data_flag)
|
||||
Poco::File(path_to_table_symlinks).remove(true);
|
||||
|
||||
DatabaseOrdinary::loadStoredObjects(context, has_force_restore_data_flag);
|
||||
|
||||
if (has_force_restore_data_flag)
|
||||
{
|
||||
NameToPathMap table_names;
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
table_names = table_name_to_path;
|
||||
}
|
||||
for (const auto & table : table_names)
|
||||
tryCreateSymlink(table.first, table.second);
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseAtomic::tryCreateSymlink(const String & table_name, const String & actual_data_path)
|
||||
{
|
||||
try
|
||||
{
|
||||
String link = path_to_table_symlinks + escapeForFileName(table_name);
|
||||
String data = global_context.getPath() + actual_data_path;
|
||||
Poco::File{data}.linkTo(link, Poco::File::LINK_SYMBOLIC);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseAtomic::tryRemoveSymlink(const String & table_name)
|
||||
{
|
||||
try
|
||||
{
|
||||
String path = path_to_table_symlinks + escapeForFileName(table_name);
|
||||
Poco::File{path}.remove();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
74
src/Databases/DatabaseAtomic.h
Normal file
74
src/Databases/DatabaseAtomic.h
Normal file
@ -0,0 +1,74 @@
|
||||
#pragma once
|
||||
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// All tables in DatabaseAtomic have persistent UUID and store data in
|
||||
/// /clickhouse_path/store/xxx/xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy/
|
||||
/// where xxxyyyyy-yyyy-yyyy-yyyy-yyyyyyyyyyyy is UUID of the table.
|
||||
/// RENAMEs are performed without changing UUID and moving table data.
|
||||
/// Tables in Atomic databases can be accessed by UUID through DatabaseCatalog.
|
||||
/// On DROP TABLE no data is removed, DatabaseAtomic just marks table as dropped
|
||||
/// by moving metadata to /clickhouse_path/metadata_dropped/ and notifies DatabaseCatalog.
|
||||
/// Running queries still may use dropped table. Table will be actually removed when it's not in use.
|
||||
/// Allows to execute RENAME and DROP without IStorage-level RWLocks
|
||||
class DatabaseAtomic : public DatabaseOrdinary
|
||||
{
|
||||
public:
|
||||
|
||||
DatabaseAtomic(String name_, String metadata_path_, Context & context_);
|
||||
|
||||
String getEngineName() const override { return "Atomic"; }
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
bool exchange) override;
|
||||
|
||||
void dropTable(const Context & context, const String & table_name, bool no_delay) override;
|
||||
|
||||
void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override;
|
||||
StoragePtr detachTable(const String & name) override;
|
||||
|
||||
String getTableDataPath(const String & table_name) const override;
|
||||
String getTableDataPath(const ASTCreateQuery & query) const override;
|
||||
|
||||
void drop(const Context & /*context*/) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
void loadStoredObjects(Context & context, bool has_force_restore_data_flag) override;
|
||||
|
||||
/// Atomic database cannot be detached if there is detached table which still in use
|
||||
void assertCanBeDetached(bool cleenup);
|
||||
|
||||
UUID tryGetTableUUID(const String & table_name) const override;
|
||||
|
||||
private:
|
||||
void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path) override;
|
||||
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path) override;
|
||||
|
||||
void assertDetachedTableNotInUse(const UUID & uuid);
|
||||
typedef std::unordered_map<UUID, StoragePtr> DetachedTables;
|
||||
[[nodiscard]] DetachedTables cleenupDetachedTables();
|
||||
|
||||
void tryCreateSymlink(const String & table_name, const String & actual_data_path);
|
||||
void tryRemoveSymlink(const String & table_name);
|
||||
|
||||
//TODO store path in DatabaseWithOwnTables::tables
|
||||
typedef std::unordered_map<String, String> NameToPathMap;
|
||||
NameToPathMap table_name_to_path;
|
||||
|
||||
DetachedTables detached_tables;
|
||||
const String path_to_table_symlinks;
|
||||
};
|
||||
|
||||
}
|
@ -30,16 +30,17 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_)
|
||||
: IDatabase(name_),
|
||||
log(&Logger::get("DatabaseDictionary(" + database_name + ")"))
|
||||
DatabaseDictionary::DatabaseDictionary(const String & name_, const Context & global_context_)
|
||||
: IDatabase(name_)
|
||||
, log(&Logger::get("DatabaseDictionary(" + database_name + ")"))
|
||||
, global_context(global_context_.getGlobalContext())
|
||||
{
|
||||
}
|
||||
|
||||
Tables DatabaseDictionary::listTables(const Context & context, const FilterByNameFunction & filter_by_name)
|
||||
Tables DatabaseDictionary::listTables(const FilterByNameFunction & filter_by_name)
|
||||
{
|
||||
Tables tables;
|
||||
auto load_results = context.getExternalDictionariesLoader().getLoadResults(filter_by_name);
|
||||
auto load_results = global_context.getExternalDictionariesLoader().getLoadResults(filter_by_name);
|
||||
for (auto & load_result : load_results)
|
||||
{
|
||||
auto storage = createStorageDictionary(getDatabaseName(), load_result);
|
||||
@ -49,39 +50,34 @@ Tables DatabaseDictionary::listTables(const Context & context, const FilterByNam
|
||||
return tables;
|
||||
}
|
||||
|
||||
bool DatabaseDictionary::isTableExist(
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
bool DatabaseDictionary::isTableExist(const String & table_name) const
|
||||
{
|
||||
return context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
|
||||
return global_context.getExternalDictionariesLoader().getCurrentStatus(table_name) != ExternalLoader::Status::NOT_EXIST;
|
||||
}
|
||||
|
||||
StoragePtr DatabaseDictionary::tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
StoragePtr DatabaseDictionary::tryGetTable(const String & table_name) const
|
||||
{
|
||||
auto load_result = context.getExternalDictionariesLoader().getLoadResult(table_name);
|
||||
auto load_result = global_context.getExternalDictionariesLoader().getLoadResult(table_name);
|
||||
return createStorageDictionary(getDatabaseName(), load_result);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseDictionary::getTablesIterator(const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(context, filter_by_table_name));
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(listTables(filter_by_table_name));
|
||||
}
|
||||
|
||||
bool DatabaseDictionary::empty(const Context & context) const
|
||||
bool DatabaseDictionary::empty() const
|
||||
{
|
||||
return !context.getExternalDictionariesLoader().hasObjects();
|
||||
return !global_context.getExternalDictionariesLoader().hasObjects();
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context,
|
||||
const String & table_name, bool throw_on_error) const
|
||||
ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const
|
||||
{
|
||||
String query;
|
||||
{
|
||||
WriteBufferFromString buffer(query);
|
||||
|
||||
auto load_result = context.getExternalDictionariesLoader().getLoadResult(table_name);
|
||||
auto load_result = global_context.getExternalDictionariesLoader().getLoadResult(table_name);
|
||||
if (!load_result.config)
|
||||
{
|
||||
if (throw_on_error)
|
||||
@ -95,7 +91,7 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context,
|
||||
buffer << ") Engine = Dictionary(" << backQuoteIfNeed(table_name) << ")";
|
||||
}
|
||||
|
||||
auto settings = context.getSettingsRef();
|
||||
auto settings = global_context.getSettingsRef();
|
||||
ParserCreateQuery parser;
|
||||
const char * pos = query.data();
|
||||
std::string error_message;
|
||||
@ -108,14 +104,14 @@ ASTPtr DatabaseDictionary::getCreateTableQueryImpl(const Context & context,
|
||||
return ast;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseDictionary::getCreateDatabaseQuery(const Context & context) const
|
||||
ASTPtr DatabaseDictionary::getCreateDatabaseQuery() const
|
||||
{
|
||||
String query;
|
||||
{
|
||||
WriteBufferFromString buffer(query);
|
||||
buffer << "CREATE DATABASE " << backQuoteIfNeed(database_name) << " ENGINE = Dictionary";
|
||||
}
|
||||
auto settings = context.getSettingsRef();
|
||||
auto settings = global_context.getSettingsRef();
|
||||
ParserCreateQuery parser;
|
||||
return parseQuery(parser, query.data(), query.data() + query.size(), "", 0, settings.max_parser_depth);
|
||||
}
|
||||
|
@ -22,40 +22,37 @@ namespace DB
|
||||
class DatabaseDictionary final : public IDatabase
|
||||
{
|
||||
public:
|
||||
DatabaseDictionary(const String & name_);
|
||||
DatabaseDictionary(const String & name_, const Context & global_context);
|
||||
|
||||
String getEngineName() const override
|
||||
{
|
||||
return "Dictionary";
|
||||
}
|
||||
|
||||
bool isTableExist(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
bool isTableExist(const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
StoragePtr tryGetTable(const String & table_name) const override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
bool empty() const override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
ASTPtr getCreateDatabaseQuery() const override;
|
||||
|
||||
bool shouldBeEmptyOnDetach() const override { return false; }
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
protected:
|
||||
ASTPtr getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const override;
|
||||
ASTPtr getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const override;
|
||||
|
||||
private:
|
||||
mutable std::mutex mutex;
|
||||
|
||||
Poco::Logger * log;
|
||||
const Context & global_context;
|
||||
|
||||
Tables listTables(const Context & context, const FilterByNameFunction & filter_by_name);
|
||||
Tables listTables(const FilterByNameFunction & filter_by_name);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
#include <Databases/DatabaseDictionary.h>
|
||||
#include <Databases/DatabaseFactory.h>
|
||||
#include <Databases/DatabaseLazy.h>
|
||||
@ -78,10 +79,12 @@ 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);
|
||||
else if (engine_name == "Memory")
|
||||
return std::make_shared<DatabaseMemory>(database_name);
|
||||
else if (engine_name == "Dictionary")
|
||||
return std::make_shared<DatabaseDictionary>(database_name);
|
||||
return std::make_shared<DatabaseDictionary>(database_name, context);
|
||||
|
||||
#if USE_MYSQL
|
||||
|
||||
@ -110,7 +113,7 @@ DatabasePtr DatabaseFactory::getImpl(
|
||||
auto mysql_database = std::make_shared<DatabaseMySQL>(
|
||||
context, database_name, metadata_path, engine_define, database_name_in_mysql, std::move(mysql_pool));
|
||||
|
||||
mysql_database->empty(context); /// test database is works fine.
|
||||
mysql_database->empty(); /// test database is works fine.
|
||||
return mysql_database;
|
||||
}
|
||||
catch (...)
|
||||
|
@ -28,10 +28,9 @@ namespace ErrorCodes
|
||||
|
||||
|
||||
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_)
|
||||
: DatabaseOnDisk(name_, metadata_path_, "DatabaseLazy (" + name_ + ")")
|
||||
: DatabaseOnDisk(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseLazy (" + name_ + ")", context_)
|
||||
, expiration_time(expiration_time_)
|
||||
{
|
||||
Poco::File(context_.getPath() + getDataPath()).createDirectories();
|
||||
}
|
||||
|
||||
|
||||
@ -42,7 +41,7 @@ void DatabaseLazy::loadStoredObjects(
|
||||
iterateMetadataFiles(context, [this](const String & file_name)
|
||||
{
|
||||
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
||||
attachTable(table_name, nullptr);
|
||||
attachTable(table_name, nullptr, {});
|
||||
});
|
||||
}
|
||||
|
||||
@ -65,12 +64,13 @@ void DatabaseLazy::createTable(
|
||||
it->second.metadata_modification_time = DatabaseOnDisk::getObjectMetadataModificationTime(table_name);
|
||||
}
|
||||
|
||||
void DatabaseLazy::removeTable(
|
||||
void DatabaseLazy::dropTable(
|
||||
const Context & context,
|
||||
const String & table_name)
|
||||
const String & table_name,
|
||||
bool no_delay)
|
||||
{
|
||||
SCOPE_EXIT({ clearExpiredTables(); });
|
||||
DatabaseOnDisk::removeTable(context, table_name);
|
||||
DatabaseOnDisk::dropTable(context, table_name, no_delay);
|
||||
}
|
||||
|
||||
void DatabaseLazy::renameTable(
|
||||
@ -78,10 +78,10 @@ void DatabaseLazy::renameTable(
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
TableStructureWriteLockHolder & lock)
|
||||
bool exchange)
|
||||
{
|
||||
SCOPE_EXIT({ clearExpiredTables(); });
|
||||
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, lock);
|
||||
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange);
|
||||
}
|
||||
|
||||
|
||||
@ -96,25 +96,21 @@ time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name
|
||||
|
||||
void DatabaseLazy::alterTable(
|
||||
const Context & /* context */,
|
||||
const String & /* table_name */,
|
||||
const StorageID & /*table_id*/,
|
||||
const StorageInMemoryMetadata & /* metadata */)
|
||||
{
|
||||
clearExpiredTables();
|
||||
throw Exception("ALTER query is not supported for Lazy database.", ErrorCodes::UNSUPPORTED_METHOD);
|
||||
}
|
||||
|
||||
bool DatabaseLazy::isTableExist(
|
||||
const Context & /* context */,
|
||||
const String & table_name) const
|
||||
bool DatabaseLazy::isTableExist(const String & table_name) const
|
||||
{
|
||||
SCOPE_EXIT({ clearExpiredTables(); });
|
||||
std::lock_guard lock(mutex);
|
||||
return tables_cache.find(table_name) != tables_cache.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseLazy::tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const
|
||||
StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const
|
||||
{
|
||||
SCOPE_EXIT({ clearExpiredTables(); });
|
||||
{
|
||||
@ -133,10 +129,10 @@ StoragePtr DatabaseLazy::tryGetTable(
|
||||
}
|
||||
}
|
||||
|
||||
return loadTable(context, table_name);
|
||||
return loadTable(table_name);
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
Strings filtered_tables;
|
||||
@ -146,15 +142,15 @@ DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(const Context & contex
|
||||
filtered_tables.push_back(table_name);
|
||||
}
|
||||
std::sort(filtered_tables.begin(), filtered_tables.end());
|
||||
return std::make_unique<DatabaseLazyIterator>(*this, context, std::move(filtered_tables));
|
||||
return std::make_unique<DatabaseLazyIterator>(*this, std::move(filtered_tables));
|
||||
}
|
||||
|
||||
bool DatabaseLazy::empty(const Context & /* context */) const
|
||||
bool DatabaseLazy::empty() const
|
||||
{
|
||||
return tables_cache.empty();
|
||||
}
|
||||
|
||||
void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table)
|
||||
void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table, const String &)
|
||||
{
|
||||
LOG_DEBUG(log, "Attach table " << backQuote(table_name) << ".");
|
||||
std::lock_guard lock(mutex);
|
||||
@ -216,7 +212,7 @@ DatabaseLazy::~DatabaseLazy()
|
||||
}
|
||||
}
|
||||
|
||||
StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table_name) const
|
||||
StoragePtr DatabaseLazy::loadTable(const String & table_name) const
|
||||
{
|
||||
SCOPE_EXIT({ clearExpiredTables(); });
|
||||
|
||||
@ -227,9 +223,9 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table
|
||||
try
|
||||
{
|
||||
StoragePtr table;
|
||||
Context context_copy(context); /// some tables can change context, but not LogTables
|
||||
Context context_copy(global_context); /// some tables can change context, but not LogTables
|
||||
|
||||
auto ast = parseQueryFromMetadata(context, table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false);
|
||||
auto ast = parseQueryFromMetadata(log, global_context, table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false);
|
||||
if (ast)
|
||||
{
|
||||
const auto & ast_create = ast->as<const ASTCreateQuery &>();
|
||||
@ -297,10 +293,9 @@ void DatabaseLazy::clearExpiredTables() const
|
||||
}
|
||||
|
||||
|
||||
DatabaseLazyIterator::DatabaseLazyIterator(DatabaseLazy & database_, const Context & context_, Strings && table_names_)
|
||||
DatabaseLazyIterator::DatabaseLazyIterator(DatabaseLazy & database_, Strings && table_names_)
|
||||
: database(database_)
|
||||
, table_names(std::move(table_names_))
|
||||
, context(context_)
|
||||
, iterator(table_names.begin())
|
||||
, current_storage(nullptr)
|
||||
{
|
||||
@ -310,7 +305,7 @@ void DatabaseLazyIterator::next()
|
||||
{
|
||||
current_storage.reset();
|
||||
++iterator;
|
||||
while (isValid() && !database.isTableExist(context, *iterator))
|
||||
while (isValid() && !database.isTableExist(*iterator))
|
||||
++iterator;
|
||||
}
|
||||
|
||||
@ -327,7 +322,7 @@ const String & DatabaseLazyIterator::name() const
|
||||
const StoragePtr & DatabaseLazyIterator::table() const
|
||||
{
|
||||
if (!current_storage)
|
||||
current_storage = database.tryGetTable(context, *iterator);
|
||||
current_storage = database.tryGetTable(*iterator);
|
||||
return current_storage;
|
||||
}
|
||||
|
||||
|
@ -32,37 +32,34 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void removeTable(
|
||||
void dropTable(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
const String & table_name,
|
||||
bool no_delay) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
TableStructureWriteLockHolder &) override;
|
||||
bool exchange) override;
|
||||
|
||||
void alterTable(
|
||||
const Context & context,
|
||||
const String & name,
|
||||
const StorageID & table_id,
|
||||
const StorageInMemoryMetadata & metadata) override;
|
||||
|
||||
time_t getObjectMetadataModificationTime(const String & table_name) const override;
|
||||
|
||||
bool isTableExist(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
bool isTableExist(const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
StoragePtr tryGetTable(const String & table_name) const override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
bool empty() const override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
@ -90,7 +87,7 @@ private:
|
||||
time_t metadata_modification_time;
|
||||
CacheExpirationQueue::iterator expiration_iterator;
|
||||
|
||||
CachedTable() {}
|
||||
CachedTable() = delete;
|
||||
CachedTable(const StoragePtr & table_, time_t last_touched_, time_t metadata_modification_time_)
|
||||
: table(table_), last_touched(last_touched_), metadata_modification_time(metadata_modification_time_) {}
|
||||
};
|
||||
@ -103,7 +100,7 @@ private:
|
||||
mutable TablesCache tables_cache;
|
||||
mutable CacheExpirationQueue cache_expiration_queue;
|
||||
|
||||
StoragePtr loadTable(const Context & context, const String & table_name) const;
|
||||
StoragePtr loadTable(const String & table_name) const;
|
||||
|
||||
void clearExpiredTables() const;
|
||||
|
||||
@ -116,7 +113,6 @@ class DatabaseLazyIterator final : public IDatabaseTablesIterator
|
||||
public:
|
||||
DatabaseLazyIterator(
|
||||
DatabaseLazy & database_,
|
||||
const Context & context_,
|
||||
Strings && table_names_);
|
||||
|
||||
void next() override;
|
||||
@ -127,7 +123,6 @@ public:
|
||||
private:
|
||||
const DatabaseLazy & database;
|
||||
const Strings table_names;
|
||||
const Context context;
|
||||
Strings::const_iterator iterator;
|
||||
mutable StoragePtr current_storage;
|
||||
};
|
||||
|
@ -1,7 +1,10 @@
|
||||
#include <common/logger_useful.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Poco/File.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -23,21 +26,35 @@ void DatabaseMemory::createTable(
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
attachTableUnlocked(table_name, table);
|
||||
std::unique_lock lock{mutex};
|
||||
attachTableUnlocked(table_name, table, lock);
|
||||
create_queries.emplace(table_name, query);
|
||||
}
|
||||
|
||||
void DatabaseMemory::removeTable(
|
||||
void DatabaseMemory::dropTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name)
|
||||
const String & table_name,
|
||||
bool /*no_delay*/)
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
detachTableUnlocked(table_name);
|
||||
std::unique_lock lock{mutex};
|
||||
auto table = detachTableUnlocked(table_name, lock);
|
||||
try
|
||||
{
|
||||
table->drop();
|
||||
Poco::File table_data_dir{getTableDataPath(table_name)};
|
||||
if (table_data_dir.exists())
|
||||
table_data_dir.remove(true);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
attachTableUnlocked(table_name, table, lock);
|
||||
throw;
|
||||
}
|
||||
table->is_dropped = true;
|
||||
create_queries.erase(table_name);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateDatabaseQuery(const Context & /*context*/) const
|
||||
ASTPtr DatabaseMemory::getCreateDatabaseQuery() const
|
||||
{
|
||||
auto create_query = std::make_shared<ASTCreateQuery>();
|
||||
create_query->database = database_name;
|
||||
@ -46,7 +63,7 @@ ASTPtr DatabaseMemory::getCreateDatabaseQuery(const Context & /*context*/) const
|
||||
return create_query;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseMemory::getCreateTableQueryImpl(const Context &, const String & table_name, bool throw_on_error) const
|
||||
ASTPtr DatabaseMemory::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
auto it = create_queries.find(table_name);
|
||||
@ -60,4 +77,11 @@ ASTPtr DatabaseMemory::getCreateTableQueryImpl(const Context &, const String & t
|
||||
return it->second;
|
||||
}
|
||||
|
||||
UUID DatabaseMemory::tryGetTableUUID(const String & table_name) const
|
||||
{
|
||||
if (auto table = tryGetTable(table_name))
|
||||
return table->getStorageID().uuid;
|
||||
return UUIDHelpers::Nil;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -29,12 +29,13 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void removeTable(
|
||||
void dropTable(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
const String & table_name,
|
||||
bool no_delay) override;
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(const Context & /*context*/, const String & name, bool throw_on_error) const override;
|
||||
ASTPtr getCreateDatabaseQuery(const Context & /*context*/) const override;
|
||||
ASTPtr getCreateTableQueryImpl(const String & name, bool throw_on_error) const override;
|
||||
ASTPtr getCreateDatabaseQuery() const override;
|
||||
|
||||
/// DatabaseMemory allows to create tables, which store data on disk.
|
||||
/// It's needed to create such tables in default database of clickhouse-local.
|
||||
@ -43,6 +44,8 @@ public:
|
||||
String getTableDataPath(const String & table_name) const override { return data_path + escapeForFileName(table_name) + "/"; }
|
||||
String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.table); }
|
||||
|
||||
UUID tryGetTableUUID(const String & table_name) const override;
|
||||
|
||||
private:
|
||||
String data_path;
|
||||
using NameToASTCreate = std::unordered_map<String, ASTPtr>;
|
||||
|
@ -65,7 +65,7 @@ DatabaseMySQL::DatabaseMySQL(
|
||||
const Context & global_context_, const String & database_name_, const String & metadata_path_,
|
||||
const ASTStorage * database_engine_define_, const String & database_name_in_mysql_, mysqlxx::Pool && pool)
|
||||
: IDatabase(database_name_)
|
||||
, global_context(global_context_)
|
||||
, global_context(global_context_.getGlobalContext())
|
||||
, metadata_path(metadata_path_)
|
||||
, database_engine_define(database_engine_define_->clone())
|
||||
, database_name_in_mysql(database_name_in_mysql_)
|
||||
@ -73,7 +73,7 @@ DatabaseMySQL::DatabaseMySQL(
|
||||
{
|
||||
}
|
||||
|
||||
bool DatabaseMySQL::empty(const Context &) const
|
||||
bool DatabaseMySQL::empty() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
@ -89,7 +89,7 @@ bool DatabaseMySQL::empty(const Context &) const
|
||||
return true;
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
Tables tables;
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
@ -103,12 +103,12 @@ DatabaseTablesIteratorPtr DatabaseMySQL::getTablesIterator(const Context &, cons
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(tables);
|
||||
}
|
||||
|
||||
bool DatabaseMySQL::isTableExist(const Context & context, const String & name) const
|
||||
bool DatabaseMySQL::isTableExist(const String & name) const
|
||||
{
|
||||
return bool(tryGetTable(context, name));
|
||||
return bool(tryGetTable(name));
|
||||
}
|
||||
|
||||
StoragePtr DatabaseMySQL::tryGetTable(const Context &, const String & mysql_table_name) const
|
||||
StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
@ -155,7 +155,7 @@ static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr
|
||||
return create_table_query;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const Context &, const String & table_name, bool throw_on_error) const
|
||||
ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
@ -184,7 +184,7 @@ time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_nam
|
||||
return time_t(local_tables_cache[table_name].first);
|
||||
}
|
||||
|
||||
ASTPtr DatabaseMySQL::getCreateDatabaseQuery(const Context & /*context*/) const
|
||||
ASTPtr DatabaseMySQL::getCreateDatabaseQuery() const
|
||||
{
|
||||
const auto & create_query = std::make_shared<ASTCreateQuery>();
|
||||
create_query->database = database_name;
|
||||
@ -373,7 +373,7 @@ void DatabaseMySQL::cleanOutdatedTables()
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseMySQL::attachTable(const String & table_name, const StoragePtr & storage)
|
||||
void DatabaseMySQL::attachTable(const String & table_name, const StoragePtr & storage, const String &)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
|
||||
@ -434,7 +434,7 @@ void DatabaseMySQL::loadStoredObjects(Context &, bool)
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseMySQL::removeTable(const Context &, const String & table_name)
|
||||
void DatabaseMySQL::dropTable(const Context &, const String & table_name, bool /*no_delay*/)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock{mutex};
|
||||
|
||||
@ -448,7 +448,8 @@ void DatabaseMySQL::removeTable(const Context &, const String & table_name)
|
||||
throw Exception("The remove flag file already exists but the " + backQuoteIfNeed(getDatabaseName()) +
|
||||
"." + backQuoteIfNeed(table_name) + " does not exists remove tables, it is bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (!local_tables_cache.count(table_name))
|
||||
auto table_iter = local_tables_cache.find(table_name);
|
||||
if (table_iter == local_tables_cache.end())
|
||||
throw Exception("Table " + backQuoteIfNeed(getDatabaseName()) + "." + backQuoteIfNeed(table_name) + " doesn't exist.",
|
||||
ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
@ -456,6 +457,7 @@ void DatabaseMySQL::removeTable(const Context &, const String & table_name)
|
||||
|
||||
try
|
||||
{
|
||||
table_iter->second.second->drop();
|
||||
remove_flag.createFile();
|
||||
}
|
||||
catch (...)
|
||||
@ -463,6 +465,7 @@ void DatabaseMySQL::removeTable(const Context &, const String & table_name)
|
||||
remove_or_detach_tables.erase(table_name);
|
||||
throw;
|
||||
}
|
||||
table_iter->second.second->is_dropped = true;
|
||||
}
|
||||
|
||||
DatabaseMySQL::~DatabaseMySQL()
|
||||
@ -487,7 +490,7 @@ DatabaseMySQL::~DatabaseMySQL()
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseMySQL::createTable(const Context & context, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query)
|
||||
void DatabaseMySQL::createTable(const Context &, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query)
|
||||
{
|
||||
const auto & create = create_query->as<ASTCreateQuery>();
|
||||
|
||||
@ -498,14 +501,14 @@ void DatabaseMySQL::createTable(const Context & context, const String & table_na
|
||||
/// XXX: hack
|
||||
/// In order to prevent users from broken the table structure by executing attach table database_name.table_name (...)
|
||||
/// we should compare the old and new create_query to make them completely consistent
|
||||
const auto & origin_create_query = getCreateTableQuery(context, table_name);
|
||||
const auto & origin_create_query = getCreateTableQuery(table_name);
|
||||
origin_create_query->as<ASTCreateQuery>()->attach = true;
|
||||
|
||||
if (queryToString(origin_create_query) != queryToString(create_query))
|
||||
throw Exception("The MySQL database engine can only execute attach statements of type attach table database_name.table_name",
|
||||
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
|
||||
attachTable(table_name, storage);
|
||||
attachTable(table_name, storage, {});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -28,15 +28,15 @@ public:
|
||||
|
||||
String getEngineName() const override { return "MySQL"; }
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
bool empty() const override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & /*context*/) const override;
|
||||
ASTPtr getCreateDatabaseQuery() const override;
|
||||
|
||||
bool isTableExist(const Context & context, const String & name) const override;
|
||||
bool isTableExist(const String & name) const override;
|
||||
|
||||
StoragePtr tryGetTable(const Context & context, const String & name) const override;
|
||||
StoragePtr tryGetTable(const String & name) const override;
|
||||
|
||||
time_t getObjectMetadataModificationTime(const String & name) const override;
|
||||
|
||||
@ -52,16 +52,15 @@ public:
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
void removeTable(const Context &, const String & table_name) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & storage) override;
|
||||
void dropTable(const Context &, const String & table_name, bool no_delay) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override;
|
||||
|
||||
protected:
|
||||
ASTPtr getCreateTableQueryImpl(const Context & context, const String & name, bool throw_on_error) const override;
|
||||
ASTPtr getCreateTableQueryImpl(const String & name, bool throw_on_error) const override;
|
||||
|
||||
private:
|
||||
Context global_context;
|
||||
const Context & global_context;
|
||||
String metadata_path;
|
||||
ASTPtr database_engine_define;
|
||||
String database_name_in_mysql;
|
||||
|
@ -18,6 +18,9 @@
|
||||
#include <common/logger_useful.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -33,7 +36,6 @@ namespace ErrorCodes
|
||||
extern const int INCORRECT_FILE_NAME;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int DICTIONARY_ALREADY_EXISTS;
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
}
|
||||
@ -53,8 +55,8 @@ std::pair<String, StoragePtr> createTableFromAST(
|
||||
{
|
||||
const auto & table_function = ast_create_query.as_table_function->as<ASTFunction &>();
|
||||
const auto & factory = TableFunctionFactory::instance();
|
||||
//FIXME storage will have wrong database name
|
||||
StoragePtr storage = factory.get(table_function.name, context)->execute(ast_create_query.as_table_function, context, ast_create_query.table);
|
||||
storage->renameInMemory(ast_create_query);
|
||||
return {ast_create_query.table, storage};
|
||||
}
|
||||
/// We do not directly use `InterpreterCreateQuery::execute`, because
|
||||
@ -111,12 +113,26 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query)
|
||||
create->format = nullptr;
|
||||
create->out_file = nullptr;
|
||||
|
||||
if (create->uuid != UUIDHelpers::Nil)
|
||||
create->table = TABLE_WITH_UUID_NAME_PLACEHOLDER;
|
||||
|
||||
std::ostringstream statement_stream;
|
||||
formatAST(*create, statement_stream, false);
|
||||
statement_stream << '\n';
|
||||
return statement_stream.str();
|
||||
}
|
||||
|
||||
DatabaseOnDisk::DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context)
|
||||
: DatabaseWithOwnTablesBase(name, logger)
|
||||
, metadata_path(metadata_path_)
|
||||
, data_path(data_path_)
|
||||
, global_context(context.getGlobalContext())
|
||||
{
|
||||
Poco::File(context.getPath() + data_path).createDirectories();
|
||||
Poco::File(metadata_path).createDirectories();
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOnDisk::createTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -124,6 +140,8 @@ void DatabaseOnDisk::createTable(
|
||||
const ASTPtr & query)
|
||||
{
|
||||
const auto & settings = context.getSettingsRef();
|
||||
const auto & create = query->as<ASTCreateQuery &>();
|
||||
assert(getDatabaseName() == create.database && table_name == create.table);
|
||||
|
||||
/// Create a file with metadata if necessary - if the query is not ATTACH.
|
||||
/// Write the query of `ATTACH table` to it.
|
||||
@ -137,15 +155,23 @@ void DatabaseOnDisk::createTable(
|
||||
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
|
||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||
|
||||
if (isDictionaryExist(context, table_name))
|
||||
|
||||
if (isDictionaryExist(table_name))
|
||||
throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.",
|
||||
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
|
||||
if (isTableExist(context, table_name))
|
||||
if (isTableExist(table_name))
|
||||
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
|
||||
if (create.attach_short_syntax)
|
||||
{
|
||||
/// Metadata already exists, table was detached
|
||||
attachTable(table_name, table, getTableDataPath(create));
|
||||
return;
|
||||
}
|
||||
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
String table_metadata_tmp_path = table_metadata_path + ".tmp";
|
||||
String table_metadata_tmp_path = table_metadata_path + create_suffix;
|
||||
String statement;
|
||||
|
||||
{
|
||||
@ -160,10 +186,16 @@ void DatabaseOnDisk::createTable(
|
||||
out.close();
|
||||
}
|
||||
|
||||
commitCreateTable(create, table, table_metadata_tmp_path, table_metadata_path);
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path)
|
||||
{
|
||||
try
|
||||
{
|
||||
/// Add a table to the map of known tables.
|
||||
attachTable(table_name, table);
|
||||
attachTable(query.table, table, getTableDataPath(query));
|
||||
|
||||
/// If it was ATTACH query and file with table metadata already exist
|
||||
/// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one.
|
||||
@ -176,30 +208,36 @@ void DatabaseOnDisk::createTable(
|
||||
}
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::removeTable(const Context & /* context */, const String & table_name)
|
||||
void DatabaseOnDisk::dropTable(const Context & context, const String & table_name, bool /*no_delay*/)
|
||||
{
|
||||
StoragePtr res = detachTable(table_name);
|
||||
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
String table_metadata_path_drop = table_metadata_path + drop_suffix;
|
||||
String table_data_path_relative = getTableDataPath(table_name);
|
||||
assert(!table_data_path_relative.empty());
|
||||
|
||||
StoragePtr table = detachTable(table_name);
|
||||
bool renamed = false;
|
||||
try
|
||||
{
|
||||
Poco::File(table_metadata_path).remove();
|
||||
Poco::File(table_metadata_path).renameTo(table_metadata_path_drop);
|
||||
renamed = true;
|
||||
table->drop();
|
||||
table->is_dropped = true;
|
||||
|
||||
Poco::File table_data_dir{context.getPath() + table_data_path_relative};
|
||||
if (table_data_dir.exists())
|
||||
table_data_dir.remove(true);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
try
|
||||
{
|
||||
Poco::File(table_metadata_path + ".tmp_drop").remove();
|
||||
return;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__));
|
||||
}
|
||||
attachTable(table_name, res);
|
||||
LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__));
|
||||
attachTable(table_name, table, table_data_path_relative);
|
||||
if (renamed)
|
||||
Poco::File(table_metadata_path_drop).renameTo(table_metadata_path);
|
||||
throw;
|
||||
}
|
||||
|
||||
Poco::File(table_metadata_path_drop).remove();
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::renameTable(
|
||||
@ -207,52 +245,73 @@ void DatabaseOnDisk::renameTable(
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
TableStructureWriteLockHolder & lock)
|
||||
bool exchange)
|
||||
{
|
||||
if (exchange)
|
||||
throw Exception("Tables can be exchanged only in Atomic databases", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
bool from_ordinary_to_atomic = false;
|
||||
bool from_atomic_to_ordinary = false;
|
||||
if (typeid(*this) != typeid(to_database))
|
||||
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
{
|
||||
if (typeid_cast<DatabaseOrdinary *>(this) && typeid_cast<DatabaseAtomic *>(&to_database))
|
||||
from_ordinary_to_atomic = true;
|
||||
else if (typeid_cast<DatabaseAtomic *>(this) && typeid_cast<DatabaseOrdinary *>(&to_database))
|
||||
from_atomic_to_ordinary = true;
|
||||
else
|
||||
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
StoragePtr table = tryGetTable(context, table_name);
|
||||
|
||||
if (!table)
|
||||
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
ASTPtr ast = parseQueryFromMetadata(context, getObjectMetadataPath(table_name));
|
||||
if (!ast)
|
||||
throw Exception("There is no metadata file for table " + backQuote(table_name) + ".", ErrorCodes::FILE_DOESNT_EXIST);
|
||||
auto & create = ast->as<ASTCreateQuery &>();
|
||||
create.table = to_table_name;
|
||||
|
||||
/// Notify the table that it is renamed. If the table does not support renaming, exception is thrown.
|
||||
auto table_data_relative_path = getTableDataPath(table_name);
|
||||
TableStructureWriteLockHolder table_lock;
|
||||
String table_metadata_path;
|
||||
ASTPtr attach_query;
|
||||
/// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case.
|
||||
StoragePtr table = tryGetTable(table_name);
|
||||
detachTable(table_name);
|
||||
try
|
||||
{
|
||||
table->rename(to_database.getTableDataPath(create),
|
||||
to_database.getDatabaseName(),
|
||||
to_table_name, lock);
|
||||
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
|
||||
table_metadata_path = getObjectMetadataPath(table_name);
|
||||
attach_query = parseQueryFromMetadata(log, context, table_metadata_path);
|
||||
auto & create = attach_query->as<ASTCreateQuery &>();
|
||||
create.database = to_database.getDatabaseName();
|
||||
create.table = to_table_name;
|
||||
if (from_ordinary_to_atomic)
|
||||
create.uuid = UUIDHelpers::generateV4();
|
||||
if (from_atomic_to_ordinary)
|
||||
create.uuid = UUIDHelpers::Nil;
|
||||
|
||||
/// Notify the table that it is renamed. It will move data to new path (if it stores data on disk) and update StorageID
|
||||
table->rename(to_database.getTableDataPath(create), StorageID(create));
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
attachTable(table_name, table, table_data_relative_path);
|
||||
throw;
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
attachTable(table_name, table, table_data_relative_path);
|
||||
/// Better diagnostics.
|
||||
throw Exception{Exception::CreateFromPoco, e};
|
||||
}
|
||||
|
||||
/// NOTE Non-atomic.
|
||||
to_database.createTable(context, to_table_name, table, ast);
|
||||
removeTable(context, table_name);
|
||||
/// Now table data are moved to new database, so we must add metadata and attach table to new database
|
||||
to_database.createTable(context, to_table_name, table, attach_query);
|
||||
|
||||
Poco::File(table_metadata_path).remove();
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const Context & context, const String & table_name, bool throw_on_error) const
|
||||
ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, bool throw_on_error) const
|
||||
{
|
||||
ASTPtr ast;
|
||||
bool has_table = tryGetTable(context, table_name) != nullptr;
|
||||
bool has_table = tryGetTable(table_name) != nullptr;
|
||||
auto table_metadata_path = getObjectMetadataPath(table_name);
|
||||
try
|
||||
{
|
||||
ast = getCreateQueryFromMetadata(context, table_metadata_path, throw_on_error);
|
||||
ast = getCreateQueryFromMetadata(table_metadata_path, throw_on_error);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
@ -265,14 +324,14 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const Context & context, const St
|
||||
return ast;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const Context & context) const
|
||||
ASTPtr DatabaseOnDisk::getCreateDatabaseQuery() const
|
||||
{
|
||||
ASTPtr ast;
|
||||
|
||||
auto settings = context.getSettingsRef();
|
||||
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";
|
||||
ast = getCreateQueryFromMetadata(context, database_metadata_path, true);
|
||||
ast = getCreateQueryFromMetadata(database_metadata_path, true);
|
||||
if (!ast)
|
||||
{
|
||||
/// Handle databases (such as default) for which there are no database.sql files.
|
||||
@ -307,8 +366,26 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(const String & object_n
|
||||
return static_cast<time_t>(0);
|
||||
}
|
||||
|
||||
void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & iterating_function) const
|
||||
void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const
|
||||
{
|
||||
auto process_tmp_drop_metadata_file = [&](const String & file_name)
|
||||
{
|
||||
assert(getEngineName() != "Atomic");
|
||||
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
||||
const std::string object_name = file_name.substr(0, file_name.size() - strlen(tmp_drop_ext));
|
||||
if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists())
|
||||
{
|
||||
Poco::File(getMetadataPath() + file_name).renameTo(getMetadataPath() + object_name + ".sql");
|
||||
LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously and will be restored");
|
||||
process_metadata_file(object_name + ".sql");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Removing file " << getMetadataPath() + file_name);
|
||||
Poco::File(getMetadataPath() + file_name).remove();
|
||||
}
|
||||
};
|
||||
|
||||
Poco::DirectoryIterator dir_end;
|
||||
for (Poco::DirectoryIterator dir_it(getMetadataPath()); dir_it != dir_end; ++dir_it)
|
||||
{
|
||||
@ -320,38 +397,22 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
|
||||
if (endsWith(dir_it.name(), ".sql.bak"))
|
||||
continue;
|
||||
|
||||
// There are files that we tried to delete previously
|
||||
static const char * tmp_drop_ext = ".sql.tmp_drop";
|
||||
if (endsWith(dir_it.name(), tmp_drop_ext))
|
||||
{
|
||||
const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
|
||||
if (Poco::File(context.getPath() + getDataPath() + '/' + object_name).exists())
|
||||
{
|
||||
/// TODO maybe complete table drop and remove all table data (including data on other volumes and metadata in ZK)
|
||||
Poco::File(dir_it->path()).renameTo(getMetadataPath() + object_name + ".sql");
|
||||
LOG_WARNING(log, "Object " << backQuote(object_name) << " was not dropped previously and will be restored");
|
||||
iterating_function(object_name + ".sql");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_INFO(log, "Removing file " << dir_it->path());
|
||||
Poco::File(dir_it->path()).remove();
|
||||
}
|
||||
continue;
|
||||
/// There are files that we tried to delete previously
|
||||
process_tmp_drop_metadata_file(dir_it.name());
|
||||
}
|
||||
|
||||
/// There are files .sql.tmp - delete
|
||||
if (endsWith(dir_it.name(), ".sql.tmp"))
|
||||
else if (endsWith(dir_it.name(), ".sql.tmp"))
|
||||
{
|
||||
/// There are files .sql.tmp - delete
|
||||
LOG_INFO(log, "Removing file " << dir_it->path());
|
||||
Poco::File(dir_it->path()).remove();
|
||||
continue;
|
||||
}
|
||||
|
||||
/// The required files have names like `table_name.sql`
|
||||
if (endsWith(dir_it.name(), ".sql"))
|
||||
else if (endsWith(dir_it.name(), ".sql"))
|
||||
{
|
||||
iterating_function(dir_it.name());
|
||||
/// The required files have names like `table_name.sql`
|
||||
process_metadata_file(dir_it.name());
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + getMetadataPath(),
|
||||
@ -359,7 +420,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const Context & context, const Iterati
|
||||
}
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const String & metadata_file_path, bool throw_on_error /*= true*/, bool remove_empty /*= false*/) const
|
||||
ASTPtr DatabaseOnDisk::parseQueryFromMetadata(Poco::Logger * loger, const Context & context, const String & metadata_file_path, bool throw_on_error /*= true*/, bool remove_empty /*= false*/)
|
||||
{
|
||||
String query;
|
||||
|
||||
@ -381,7 +442,7 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str
|
||||
*/
|
||||
if (remove_empty && query.empty())
|
||||
{
|
||||
LOG_ERROR(log, "File " << metadata_file_path << " is empty. Removing.");
|
||||
LOG_ERROR(loger, "File " << metadata_file_path << " is empty. Removing.");
|
||||
Poco::File(metadata_file_path).remove();
|
||||
return nullptr;
|
||||
}
|
||||
@ -391,19 +452,31 @@ ASTPtr DatabaseOnDisk::parseQueryFromMetadata(const Context & context, const Str
|
||||
const char * pos = query.data();
|
||||
std::string error_message;
|
||||
auto ast = tryParseQuery(parser, pos, pos + query.size(), error_message, /* hilite = */ false,
|
||||
"in file " + getMetadataPath(), /* allow_multi_statements = */ false, 0, settings.max_parser_depth);
|
||||
"in file " + metadata_file_path, /* allow_multi_statements = */ false, 0, settings.max_parser_depth);
|
||||
|
||||
if (!ast && throw_on_error)
|
||||
throw Exception(error_message, ErrorCodes::SYNTAX_ERROR);
|
||||
else if (!ast)
|
||||
return nullptr;
|
||||
|
||||
auto & create = ast->as<ASTCreateQuery &>();
|
||||
if (create.uuid != UUIDHelpers::Nil)
|
||||
{
|
||||
String table_name = Poco::Path(metadata_file_path).makeFile().getBaseName();
|
||||
table_name = unescapeForFileName(table_name);
|
||||
|
||||
if (create.table != TABLE_WITH_UUID_NAME_PLACEHOLDER)
|
||||
LOG_WARNING(loger, "File " << metadata_file_path << " contains both UUID and table name. "
|
||||
"Will use name `" << table_name << "` instead of `" << create.table << "`");
|
||||
create.table = table_name;
|
||||
}
|
||||
|
||||
return ast;
|
||||
}
|
||||
|
||||
ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const Context & context, const String & database_metadata_path, bool throw_on_error) const
|
||||
ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metadata_path, bool throw_on_error) const
|
||||
{
|
||||
ASTPtr ast = parseQueryFromMetadata(context, database_metadata_path, throw_on_error);
|
||||
ASTPtr ast = parseQueryFromMetadata(log, global_context, database_metadata_path, throw_on_error);
|
||||
|
||||
if (ast)
|
||||
{
|
||||
|
@ -30,10 +30,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query);
|
||||
class DatabaseOnDisk : public DatabaseWithOwnTablesBase
|
||||
{
|
||||
public:
|
||||
DatabaseOnDisk(const String & name, const String & metadata_path_, const String & logger)
|
||||
: DatabaseWithOwnTablesBase(name, logger)
|
||||
, metadata_path(metadata_path_)
|
||||
, data_path("data/" + escapeForFileName(database_name) + "/") {}
|
||||
DatabaseOnDisk(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context);
|
||||
|
||||
void createTable(
|
||||
const Context & context,
|
||||
@ -41,18 +38,19 @@ public:
|
||||
const StoragePtr & table,
|
||||
const ASTPtr & query) override;
|
||||
|
||||
void removeTable(
|
||||
void dropTable(
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
const String & table_name,
|
||||
bool no_delay) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
IDatabase & to_database,
|
||||
const String & to_table_name,
|
||||
TableStructureWriteLockHolder & lock) override;
|
||||
bool exchange) override;
|
||||
|
||||
ASTPtr getCreateDatabaseQuery(const Context & context) const override;
|
||||
ASTPtr getCreateDatabaseQuery() const override;
|
||||
|
||||
void drop(const Context & context) override;
|
||||
|
||||
@ -65,21 +63,28 @@ public:
|
||||
String getTableDataPath(const ASTCreateQuery & query) const override { return getTableDataPath(query.table); }
|
||||
String getMetadataPath() const override { return metadata_path; }
|
||||
|
||||
static ASTPtr parseQueryFromMetadata(Poco::Logger * log, const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false);
|
||||
|
||||
protected:
|
||||
static constexpr const char * create_suffix = ".tmp";
|
||||
static constexpr const char * drop_suffix = ".tmp_drop";
|
||||
|
||||
using IteratingFunction = std::function<void(const String &)>;
|
||||
void iterateMetadataFiles(const Context & context, const IteratingFunction & iterating_function) const;
|
||||
|
||||
void iterateMetadataFiles(const Context & context, const IteratingFunction & process_metadata_file) const;
|
||||
|
||||
ASTPtr getCreateTableQueryImpl(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
bool throw_on_error) const override;
|
||||
|
||||
ASTPtr parseQueryFromMetadata(const Context & context, const String & metadata_file_path, bool throw_on_error = true, bool remove_empty = false) const;
|
||||
ASTPtr getCreateQueryFromMetadata(const Context & context, const String & metadata_path, bool throw_on_error) const;
|
||||
ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const;
|
||||
|
||||
virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
||||
const String & table_metadata_tmp_path, const String & table_metadata_path);
|
||||
|
||||
const String metadata_path;
|
||||
const String data_path;
|
||||
const Context & global_context;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -35,16 +35,10 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
static constexpr size_t PRINT_MESSAGE_EACH_N_OBJECTS = 256;
|
||||
static constexpr size_t PRINT_MESSAGE_EACH_N_SECONDS = 5;
|
||||
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
void tryAttachTable(
|
||||
@ -62,7 +56,7 @@ namespace
|
||||
StoragePtr table;
|
||||
std::tie(table_name, table)
|
||||
= createTableFromAST(query, database_name, database.getTableDataPath(query), context, has_force_restore_data_flag);
|
||||
database.attachTable(table_name, table);
|
||||
database.attachTable(table_name, table, database.getTableDataPath(query));
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -110,11 +104,14 @@ namespace
|
||||
|
||||
|
||||
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context_)
|
||||
: DatabaseWithDictionaries(name_, metadata_path_, "DatabaseOrdinary (" + name_ + ")")
|
||||
: DatabaseOrdinary(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseOrdinary (" + name_ + ")", context_)
|
||||
{
|
||||
Poco::File(context_.getPath() + getDataPath()).createDirectories();
|
||||
}
|
||||
|
||||
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context_)
|
||||
: DatabaseWithDictionaries(name_, metadata_path_, data_path_, logger, context_)
|
||||
{
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::loadStoredObjects(
|
||||
Context & context,
|
||||
@ -128,12 +125,13 @@ void DatabaseOrdinary::loadStoredObjects(
|
||||
FileNames file_names;
|
||||
|
||||
size_t total_dictionaries = 0;
|
||||
iterateMetadataFiles(context, [&context, &file_names, &total_dictionaries, this](const String & file_name)
|
||||
|
||||
auto process_metadata = [&context, &file_names, &total_dictionaries, this](const String & file_name)
|
||||
{
|
||||
String full_path = getMetadataPath() + file_name;
|
||||
try
|
||||
{
|
||||
auto ast = parseQueryFromMetadata(context, full_path, /*throw_on_error*/ true, /*remove_empty*/ false);
|
||||
auto ast = parseQueryFromMetadata(log, context, full_path, /*throw_on_error*/ true, /*remove_empty*/ false);
|
||||
if (ast)
|
||||
{
|
||||
auto * create_query = ast->as<ASTCreateQuery>();
|
||||
@ -147,7 +145,9 @@ void DatabaseOrdinary::loadStoredObjects(
|
||||
throw;
|
||||
}
|
||||
|
||||
});
|
||||
};
|
||||
|
||||
iterateMetadataFiles(context, process_metadata);
|
||||
|
||||
size_t total_tables = file_names.size() - total_dictionaries;
|
||||
|
||||
@ -226,9 +226,10 @@ void DatabaseOrdinary::startupTables(ThreadPool & thread_pool)
|
||||
|
||||
void DatabaseOrdinary::alterTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
const StorageID & table_id,
|
||||
const StorageInMemoryMetadata & metadata)
|
||||
{
|
||||
String table_name = table_id.table_name;
|
||||
/// Read the definition of the table and replace the necessary parts with new ones.
|
||||
String table_metadata_path = getObjectMetadataPath(table_name);
|
||||
String table_metadata_tmp_path = table_metadata_path + ".tmp";
|
||||
@ -287,6 +288,11 @@ void DatabaseOrdinary::alterTable(
|
||||
out.close();
|
||||
}
|
||||
|
||||
commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path);
|
||||
}
|
||||
|
||||
void DatabaseOrdinary::commitAlterTable(const StorageID &, const String & table_metadata_tmp_path, const String & table_metadata_path)
|
||||
{
|
||||
try
|
||||
{
|
||||
/// rename atomically replaces the old file with the new one.
|
||||
|
@ -11,10 +11,11 @@ namespace DB
|
||||
* It stores tables list in filesystem using list of .sql files,
|
||||
* that contain declaration of table represented by SQL ATTACH TABLE query.
|
||||
*/
|
||||
class DatabaseOrdinary final : public DatabaseWithDictionaries
|
||||
class DatabaseOrdinary : public DatabaseWithDictionaries
|
||||
{
|
||||
public:
|
||||
DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context);
|
||||
DatabaseOrdinary(const String & name_, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context_);
|
||||
|
||||
String getEngineName() const override { return "Ordinary"; }
|
||||
|
||||
@ -24,10 +25,11 @@ public:
|
||||
|
||||
void alterTable(
|
||||
const Context & context,
|
||||
const String & name,
|
||||
const StorageID & table_id,
|
||||
const StorageInMemoryMetadata & metadata) override;
|
||||
|
||||
private:
|
||||
protected:
|
||||
virtual void commitAlterTable(const StorageID & table_id, const String & table_metadata_tmp_path, const String & table_metadata_path);
|
||||
|
||||
void startupTables(ThreadPool & thread_pool);
|
||||
};
|
||||
|
@ -36,7 +36,7 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name,
|
||||
{
|
||||
String full_name = getDatabaseName() + "." + dictionary_name;
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
auto [it, inserted] = dictionaries.emplace(dictionary_name, attach_info);
|
||||
if (!inserted)
|
||||
throw Exception("Dictionary " + full_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
@ -49,7 +49,8 @@ void DatabaseWithDictionaries::attachDictionary(const String & dictionary_name,
|
||||
StorageDictionary::create(
|
||||
StorageID(getDatabaseName(), dictionary_name),
|
||||
full_name,
|
||||
ExternalDictionariesLoader::getDictionaryStructure(*attach_info.config)));
|
||||
ExternalDictionariesLoader::getDictionaryStructure(*attach_info.config)),
|
||||
lock);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -76,7 +77,7 @@ void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_na
|
||||
String full_name = getDatabaseName() + "." + dictionary_name;
|
||||
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
std::unique_lock lock(mutex);
|
||||
auto it = dictionaries.find(dictionary_name);
|
||||
if (it == dictionaries.end())
|
||||
throw Exception("Dictionary " + full_name + " doesn't exist.", ErrorCodes::UNKNOWN_DICTIONARY);
|
||||
@ -86,7 +87,7 @@ void DatabaseWithDictionaries::detachDictionaryImpl(const String & dictionary_na
|
||||
/// Detach the dictionary as table too.
|
||||
try
|
||||
{
|
||||
detachTableUnlocked(dictionary_name);
|
||||
detachTableUnlocked(dictionary_name, lock);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -116,7 +117,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
|
||||
|
||||
/// A race condition would be possible if a dictionary with the same name is simultaneously created using CREATE and using ATTACH.
|
||||
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
|
||||
if (isDictionaryExist(context, dictionary_name))
|
||||
if (isDictionaryExist(dictionary_name))
|
||||
throw Exception("Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
|
||||
/// A dictionary with the same full name could be defined in *.xml config files.
|
||||
@ -126,7 +127,7 @@ void DatabaseWithDictionaries::createDictionary(const Context & context, const S
|
||||
"Dictionary " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.",
|
||||
ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
||||
|
||||
if (isTableExist(context, dictionary_name))
|
||||
if (isTableExist(dictionary_name))
|
||||
throw Exception("Table " + backQuote(getDatabaseName()) + "." + backQuote(dictionary_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
|
||||
|
||||
@ -201,7 +202,7 @@ void DatabaseWithDictionaries::removeDictionary(const Context &, const String &
|
||||
}
|
||||
}
|
||||
|
||||
DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_dictionary_name)
|
||||
DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterator(const FilterByNameFunction & filter_by_dictionary_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!filter_by_dictionary_name)
|
||||
@ -214,14 +215,13 @@ DatabaseDictionariesIteratorPtr DatabaseWithDictionaries::getDictionariesIterato
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>(std::move(filtered_dictionaries));
|
||||
}
|
||||
|
||||
bool DatabaseWithDictionaries::isDictionaryExist(const Context & /*context*/, const String & dictionary_name) const
|
||||
bool DatabaseWithDictionaries::isDictionaryExist(const String & dictionary_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return dictionaries.find(dictionary_name) != dictionaries.end();
|
||||
}
|
||||
|
||||
ASTPtr DatabaseWithDictionaries::getCreateDictionaryQueryImpl(
|
||||
const Context & context,
|
||||
const String & dictionary_name,
|
||||
bool throw_on_error) const
|
||||
{
|
||||
@ -244,7 +244,7 @@ ASTPtr DatabaseWithDictionaries::getCreateDictionaryQueryImpl(
|
||||
try
|
||||
{
|
||||
auto dictionary_metadata_path = getObjectMetadataPath(dictionary_name);
|
||||
ast = getCreateQueryFromMetadata(context, dictionary_metadata_path, throw_on_error);
|
||||
ast = getCreateQueryFromMetadata(dictionary_metadata_path, throw_on_error);
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
@ -264,7 +264,6 @@ ASTPtr DatabaseWithDictionaries::getCreateDictionaryQueryImpl(
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
Poco::AutoPtr<Poco::Util::AbstractConfiguration> DatabaseWithDictionaries::getDictionaryConfiguration(const String & dictionary_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
@ -286,7 +285,7 @@ time_t DatabaseWithDictionaries::getObjectMetadataModificationTime(const String
|
||||
}
|
||||
|
||||
|
||||
bool DatabaseWithDictionaries::empty(const Context &) const
|
||||
bool DatabaseWithDictionaries::empty() const
|
||||
{
|
||||
std::lock_guard lock{mutex};
|
||||
return tables.empty() && dictionaries.empty();
|
||||
|
@ -18,31 +18,30 @@ public:
|
||||
|
||||
void removeDictionary(const Context & context, const String & dictionary_name) override;
|
||||
|
||||
bool isDictionaryExist(const Context & context, const String & dictionary_name) const override;
|
||||
bool isDictionaryExist(const String & dictionary_name) const override;
|
||||
|
||||
DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & context, const FilterByNameFunction & filter_by_dictionary_name) override;
|
||||
DatabaseDictionariesIteratorPtr getDictionariesIterator(const FilterByNameFunction & filter_by_dictionary_name) override;
|
||||
|
||||
Poco::AutoPtr<Poco::Util::AbstractConfiguration> getDictionaryConfiguration(const String & /*name*/) const override;
|
||||
|
||||
time_t getObjectMetadataModificationTime(const String & object_name) const override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
bool empty() const override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
~DatabaseWithDictionaries() override;
|
||||
|
||||
protected:
|
||||
DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & logger)
|
||||
: DatabaseOnDisk(name, metadata_path_, logger) {}
|
||||
DatabaseWithDictionaries(const String & name, const String & metadata_path_, const String & data_path_, const String & logger, const Context & context)
|
||||
: DatabaseOnDisk(name, metadata_path_, data_path_, logger, context) {}
|
||||
|
||||
void attachToExternalDictionariesLoader(Context & context);
|
||||
void detachFromExternalDictionariesLoader();
|
||||
|
||||
void detachDictionaryImpl(const String & dictionary_name, DictionaryAttachInfo & attach_info);
|
||||
|
||||
ASTPtr getCreateDictionaryQueryImpl(const Context & context,
|
||||
const String & dictionary_name,
|
||||
ASTPtr getCreateDictionaryQueryImpl(const String & dictionary_name,
|
||||
bool throw_on_error) const override;
|
||||
|
||||
std::unordered_map<String, DictionaryAttachInfo> dictionaries;
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Storages/StorageDictionary.h>
|
||||
#include <Storages/StorageFactory.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <TableFunctions/TableFunctionFactory.h>
|
||||
|
||||
|
||||
@ -22,17 +23,13 @@ DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const
|
||||
{
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::isTableExist(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
bool DatabaseWithOwnTablesBase::isTableExist(const String & table_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(const String & table_name) const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
@ -41,7 +38,7 @@ StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
|
||||
return {};
|
||||
}
|
||||
|
||||
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Context & /*context*/, const FilterByNameFunction & filter_by_table_name)
|
||||
DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const FilterByNameFunction & filter_by_table_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
if (!filter_by_table_name)
|
||||
@ -55,7 +52,7 @@ DatabaseTablesIteratorPtr DatabaseWithOwnTablesBase::getTablesIterator(const Con
|
||||
return std::make_unique<DatabaseTablesSnapshotIterator>(std::move(filtered_tables));
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const
|
||||
bool DatabaseWithOwnTablesBase::empty() const
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return tables.empty();
|
||||
@ -63,11 +60,11 @@ bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
return detachTableUnlocked(table_name);
|
||||
std::unique_lock lock(mutex);
|
||||
return detachTableUnlocked(table_name, lock);
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_name)
|
||||
StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_name, std::unique_lock<std::mutex> &)
|
||||
{
|
||||
StoragePtr res;
|
||||
|
||||
@ -80,29 +77,27 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n
|
||||
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);
|
||||
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
|
||||
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table)
|
||||
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table, const String &)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
attachTableUnlocked(table_name, table);
|
||||
std::unique_lock lock(mutex);
|
||||
attachTableUnlocked(table_name, table, lock);
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table)
|
||||
void DatabaseWithOwnTablesBase::attachTableUnlocked(const String & table_name, const StoragePtr & table, std::unique_lock<std::mutex> &)
|
||||
{
|
||||
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);
|
||||
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
|
||||
DatabaseCatalog::instance().addUUIDMapping(table_id.uuid, shared_from_this(), table);
|
||||
}
|
||||
}
|
||||
@ -120,7 +115,13 @@ void DatabaseWithOwnTablesBase::shutdown()
|
||||
|
||||
for (const auto & kv : tables_snapshot)
|
||||
{
|
||||
auto table_id = kv.second->getStorageID();
|
||||
kv.second->shutdown();
|
||||
if (table_id.hasUUID())
|
||||
{
|
||||
assert(getDatabaseName() == DatabaseCatalog::TEMPORARY_DATABASE || getEngineName() == "Atomic");
|
||||
DatabaseCatalog::instance().removeUUIDMapping(table_id.uuid);
|
||||
}
|
||||
}
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
@ -139,4 +140,12 @@ DatabaseWithOwnTablesBase::~DatabaseWithOwnTablesBase()
|
||||
}
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::getTableUnlocked(const String & table_name, std::unique_lock<std::mutex> &) const
|
||||
{
|
||||
auto it = tables.find(table_name);
|
||||
if (it != tables.end())
|
||||
return it->second;
|
||||
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -19,21 +19,17 @@ class Context;
|
||||
class DatabaseWithOwnTablesBase : public IDatabase
|
||||
{
|
||||
public:
|
||||
bool isTableExist(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
bool isTableExist(const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
StoragePtr tryGetTable(const String & table_name) const override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
bool empty() const override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name) override;
|
||||
DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
@ -46,8 +42,9 @@ protected:
|
||||
|
||||
DatabaseWithOwnTablesBase(const String & name_, const String & logger);
|
||||
|
||||
void attachTableUnlocked(const String & table_name, const StoragePtr & table);
|
||||
StoragePtr detachTableUnlocked(const String & table_name);
|
||||
void attachTableUnlocked(const String & table_name, const StoragePtr & table, std::unique_lock<std::mutex> & lock);
|
||||
StoragePtr detachTableUnlocked(const String & table_name, std::unique_lock<std::mutex> & lock);
|
||||
StoragePtr getTableUnlocked(const String & table_name, std::unique_lock<std::mutex> & lock) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -43,15 +43,27 @@ public:
|
||||
virtual const StoragePtr & table() const = 0;
|
||||
|
||||
virtual ~IDatabaseTablesIterator() = default;
|
||||
|
||||
virtual UUID uuid() const { return UUIDHelpers::Nil; }
|
||||
};
|
||||
|
||||
/// Copies list of tables and iterates through such snapshot.
|
||||
class DatabaseTablesSnapshotIterator final : public IDatabaseTablesIterator
|
||||
class DatabaseTablesSnapshotIterator : public IDatabaseTablesIterator
|
||||
{
|
||||
private:
|
||||
Tables tables;
|
||||
Tables::iterator it;
|
||||
|
||||
protected:
|
||||
DatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && other)
|
||||
{
|
||||
size_t idx = std::distance(other.tables.begin(), other.it);
|
||||
std::swap(tables, other.tables);
|
||||
other.it = other.tables.end();
|
||||
it = tables.begin();
|
||||
std::advance(it, idx);
|
||||
}
|
||||
|
||||
public:
|
||||
DatabaseTablesSnapshotIterator(Tables & tables_) : tables(tables_), it(tables.begin()) {}
|
||||
|
||||
@ -118,37 +130,33 @@ public:
|
||||
virtual void loadStoredObjects(Context & /*context*/, bool /*has_force_restore_data_flag*/) {}
|
||||
|
||||
/// Check the existence of the table.
|
||||
virtual bool isTableExist(
|
||||
const Context & context,
|
||||
const String & name) const = 0;
|
||||
virtual bool isTableExist(const String & name) const = 0;
|
||||
|
||||
/// Check the existence of the dictionary
|
||||
virtual bool isDictionaryExist(
|
||||
const Context & /*context*/,
|
||||
const String & /*name*/) const
|
||||
virtual bool isDictionaryExist(const String & /*name*/) const
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Get the table for work. Return nullptr if there is no table.
|
||||
virtual StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & name) const = 0;
|
||||
virtual StoragePtr tryGetTable(const String & name) const = 0;
|
||||
|
||||
virtual UUID tryGetTableUUID(const String & /*table_name*/) const { return UUIDHelpers::Nil; }
|
||||
|
||||
using FilterByNameFunction = std::function<bool(const String &)>;
|
||||
|
||||
/// Get an iterator that allows you to pass through all the tables.
|
||||
/// It is possible to have "hidden" tables that are not visible when passing through, but are visible if you get them by name using the functions above.
|
||||
virtual DatabaseTablesIteratorPtr getTablesIterator(const Context & context, const FilterByNameFunction & filter_by_table_name = {}) = 0;
|
||||
virtual DatabaseTablesIteratorPtr getTablesIterator(const FilterByNameFunction & filter_by_table_name = {}) = 0;
|
||||
|
||||
/// Get an iterator to pass through all the dictionaries.
|
||||
virtual DatabaseDictionariesIteratorPtr getDictionariesIterator(const Context & /*context*/, [[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {})
|
||||
virtual DatabaseDictionariesIteratorPtr getDictionariesIterator([[maybe_unused]] const FilterByNameFunction & filter_by_dictionary_name = {})
|
||||
{
|
||||
return std::make_unique<DatabaseDictionariesSnapshotIterator>();
|
||||
}
|
||||
|
||||
/// Is the database empty.
|
||||
virtual bool empty(const Context & context) const = 0;
|
||||
virtual bool empty() const = 0;
|
||||
|
||||
/// Add the table to the database. Record its presence in the metadata.
|
||||
virtual void createTable(
|
||||
@ -169,10 +177,11 @@ public:
|
||||
throw Exception("There is no CREATE DICTIONARY query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// Delete the table from the database. Delete the metadata.
|
||||
virtual void removeTable(
|
||||
/// Delete the table from the database, drop table and delete the metadata.
|
||||
virtual void dropTable(
|
||||
const Context & /*context*/,
|
||||
const String & /*name*/)
|
||||
const String & /*name*/,
|
||||
[[maybe_unused]] bool no_delay = false)
|
||||
{
|
||||
throw Exception("There is no DROP TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
@ -186,7 +195,7 @@ public:
|
||||
}
|
||||
|
||||
/// Add a table to the database, but do not add it to the metadata. The database may not support this method.
|
||||
virtual void attachTable(const String & /*name*/, const StoragePtr & /*table*/)
|
||||
virtual void attachTable(const String & /*name*/, const StoragePtr & /*table*/, [[maybe_unused]] const String & relative_table_path = {})
|
||||
{
|
||||
throw Exception("There is no ATTACH TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
@ -216,7 +225,7 @@ public:
|
||||
const String & /*name*/,
|
||||
IDatabase & /*to_database*/,
|
||||
const String & /*to_name*/,
|
||||
TableStructureWriteLockHolder &)
|
||||
bool /*exchange*/)
|
||||
{
|
||||
throw Exception(getEngineName() + ": renameTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
@ -227,7 +236,7 @@ public:
|
||||
/// You must call under the TableStructureLock of the corresponding table . If engine_modifier is empty, then engine does not change.
|
||||
virtual void alterTable(
|
||||
const Context & /*context*/,
|
||||
const String & /*name*/,
|
||||
const StorageID & /*table_id*/,
|
||||
const StorageInMemoryMetadata & /*metadata*/)
|
||||
{
|
||||
throw Exception(getEngineName() + ": alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -240,25 +249,25 @@ public:
|
||||
}
|
||||
|
||||
/// Get the CREATE TABLE query for the table. It can also provide information for detached tables for which there is metadata.
|
||||
ASTPtr tryGetCreateTableQuery(const Context & context, const String & name) const noexcept
|
||||
ASTPtr tryGetCreateTableQuery(const String & name) const noexcept
|
||||
{
|
||||
return getCreateTableQueryImpl(context, name, false);
|
||||
return getCreateTableQueryImpl(name, false);
|
||||
}
|
||||
|
||||
ASTPtr getCreateTableQuery(const Context & context, const String & name) const
|
||||
ASTPtr getCreateTableQuery(const String & name) const
|
||||
{
|
||||
return getCreateTableQueryImpl(context, name, true);
|
||||
return getCreateTableQueryImpl(name, true);
|
||||
}
|
||||
|
||||
/// Get the CREATE DICTIONARY query for the dictionary. Returns nullptr if dictionary doesn't exists.
|
||||
ASTPtr tryGetCreateDictionaryQuery(const Context & context, const String & name) const noexcept
|
||||
ASTPtr tryGetCreateDictionaryQuery(const String & name) const noexcept
|
||||
{
|
||||
return getCreateDictionaryQueryImpl(context, name, false);
|
||||
return getCreateDictionaryQueryImpl(name, false);
|
||||
}
|
||||
|
||||
ASTPtr getCreateDictionaryQuery(const Context & context, const String & name) const
|
||||
ASTPtr getCreateDictionaryQuery(const String & name) const
|
||||
{
|
||||
return getCreateDictionaryQueryImpl(context, name, true);
|
||||
return getCreateDictionaryQueryImpl(name, true);
|
||||
}
|
||||
|
||||
virtual Poco::AutoPtr<Poco::Util::AbstractConfiguration> getDictionaryConfiguration(const String & /*name*/) const
|
||||
@ -267,12 +276,13 @@ public:
|
||||
}
|
||||
|
||||
/// Get the CREATE DATABASE query for current database.
|
||||
virtual ASTPtr getCreateDatabaseQuery(const Context & /*context*/) const = 0;
|
||||
virtual ASTPtr getCreateDatabaseQuery() const = 0;
|
||||
|
||||
/// Get name of database.
|
||||
String getDatabaseName() const { return database_name; }
|
||||
/// Returns path for persistent data storage if the database supports it, empty string otherwise
|
||||
virtual String getDataPath() const { return {}; }
|
||||
|
||||
/// Returns path for persistent data storage for table if the database supports it, empty string otherwise. Table must exist
|
||||
virtual String getTableDataPath(const String & /*table_name*/) const { return {}; }
|
||||
/// Returns path for persistent data storage for CREATE/ATTACH query if the database supports it, empty string otherwise
|
||||
@ -294,14 +304,14 @@ public:
|
||||
virtual ~IDatabase() {}
|
||||
|
||||
protected:
|
||||
virtual ASTPtr getCreateTableQueryImpl(const Context & /*context*/, const String & /*name*/, bool throw_on_error) const
|
||||
virtual ASTPtr getCreateTableQueryImpl(const String & /*name*/, bool throw_on_error) const
|
||||
{
|
||||
if (throw_on_error)
|
||||
throw Exception("There is no SHOW CREATE TABLE query for Database" + getEngineName(), ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
virtual ASTPtr getCreateDictionaryQueryImpl(const Context & /*context*/, const String & /*name*/, bool throw_on_error) const
|
||||
virtual ASTPtr getCreateDictionaryQueryImpl(const String & /*name*/, bool throw_on_error) const
|
||||
{
|
||||
if (throw_on_error)
|
||||
throw Exception("There is no SHOW CREATE DICTIONARY query for Database" + getEngineName(), ErrorCodes::CANNOT_GET_CREATE_DICTIONARY_QUERY);
|
||||
|
@ -5,6 +5,7 @@ PEERDIR(
|
||||
)
|
||||
|
||||
SRCS(
|
||||
DatabaseAtomic.cpp
|
||||
DatabaseDictionary.cpp
|
||||
DatabaseFactory.cpp
|
||||
DatabaseLazy.cpp
|
||||
|
@ -20,17 +20,17 @@ namespace ActionLocks
|
||||
|
||||
|
||||
template <typename F>
|
||||
inline void forEachTable(Context & context, F && f)
|
||||
inline void forEachTable(F && f)
|
||||
{
|
||||
for (auto & elem : DatabaseCatalog::instance().getDatabases())
|
||||
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = elem.second->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
f(iterator->table());
|
||||
|
||||
}
|
||||
|
||||
void ActionLocksManager::add(StorageActionBlockType action_type)
|
||||
{
|
||||
forEachTable(global_context, [&](const StoragePtr & table) { add(table, action_type); });
|
||||
forEachTable([&](const StoragePtr & table) { add(table, action_type); });
|
||||
}
|
||||
|
||||
void ActionLocksManager::add(const StorageID & table_id, StorageActionBlockType action_type)
|
||||
|
@ -19,8 +19,6 @@ class Context;
|
||||
class ActionLocksManager
|
||||
{
|
||||
public:
|
||||
explicit ActionLocksManager(Context & global_context_) : global_context(global_context_) {}
|
||||
|
||||
/// Adds new locks for each table
|
||||
void add(StorageActionBlockType action_type);
|
||||
/// Add new lock for a table if it has not been already added
|
||||
@ -37,8 +35,6 @@ public:
|
||||
void cleanExpired();
|
||||
|
||||
private:
|
||||
Context & global_context;
|
||||
|
||||
using StorageRawPtr = const IStorage *;
|
||||
using Locks = std::unordered_map<size_t, ActionLock>;
|
||||
using StorageLocks = std::unordered_map<StorageRawPtr, Locks>;
|
||||
|
@ -173,7 +173,7 @@ void AsynchronousMetrics::update()
|
||||
/// Lazy database can not contain MergeTree tables
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
++total_number_of_tables;
|
||||
const auto & table = iterator->table();
|
||||
|
@ -406,7 +406,7 @@ struct ContextShared
|
||||
if (system_logs)
|
||||
system_logs->shutdown();
|
||||
|
||||
DatabaseCatalog::instance().shutdown();
|
||||
DatabaseCatalog::shutdown();
|
||||
|
||||
/// Preemptive destruction is important, because these objects may have a refcount to ContextShared (cyclic reference).
|
||||
/// TODO: Get rid of this.
|
||||
@ -1988,7 +1988,7 @@ std::shared_ptr<ActionLocksManager> Context::getActionLocksManager()
|
||||
auto lock = getLock();
|
||||
|
||||
if (!shared->action_locks_manager)
|
||||
shared->action_locks_manager = std::make_shared<ActionLocksManager>(getGlobalContext());
|
||||
shared->action_locks_manager = std::make_shared<ActionLocksManager>();
|
||||
|
||||
return shared->action_locks_manager;
|
||||
}
|
||||
@ -2063,11 +2063,16 @@ StorageID Context::resolveStorageID(StorageID storage_id, StorageNamespace where
|
||||
if (storage_id.uuid != UUIDHelpers::Nil)
|
||||
return storage_id;
|
||||
|
||||
auto lock = getLock();
|
||||
StorageID resolved = StorageID::createEmpty();
|
||||
std::optional<Exception> exc;
|
||||
auto resolved = resolveStorageIDImpl(std::move(storage_id), where, &exc);
|
||||
{
|
||||
auto lock = getLock();
|
||||
resolved = resolveStorageIDImpl(std::move(storage_id), where, &exc);
|
||||
}
|
||||
if (exc)
|
||||
throw Exception(*exc);
|
||||
if (!resolved.hasUUID() && resolved.database_name != DatabaseCatalog::TEMPORARY_DATABASE)
|
||||
resolved.uuid = DatabaseCatalog::instance().getDatabase(resolved.database_name)->tryGetTableUUID(resolved.table_name);
|
||||
return resolved;
|
||||
}
|
||||
|
||||
@ -2076,8 +2081,18 @@ StorageID Context::tryResolveStorageID(StorageID storage_id, StorageNamespace wh
|
||||
if (storage_id.uuid != UUIDHelpers::Nil)
|
||||
return storage_id;
|
||||
|
||||
auto lock = getLock();
|
||||
return resolveStorageIDImpl(std::move(storage_id), where, nullptr);
|
||||
StorageID resolved = StorageID::createEmpty();
|
||||
{
|
||||
auto lock = getLock();
|
||||
resolved = resolveStorageIDImpl(std::move(storage_id), where, nullptr);
|
||||
}
|
||||
if (resolved && !resolved.hasUUID() && resolved.database_name != DatabaseCatalog::TEMPORARY_DATABASE)
|
||||
{
|
||||
auto db = DatabaseCatalog::instance().tryGetDatabase(resolved.database_name);
|
||||
if (db)
|
||||
resolved.uuid = db->tryGetTableUUID(resolved.table_name);
|
||||
}
|
||||
return resolved;
|
||||
}
|
||||
|
||||
StorageID Context::resolveStorageIDImpl(StorageID storage_id, StorageNamespace where, std::optional<Exception> * exception) const
|
||||
|
@ -638,7 +638,7 @@ void DDLWorker::processTask(DDLTask & task, const ZooKeeperPtr & zookeeper)
|
||||
if (!query_with_table->table.empty())
|
||||
{
|
||||
/// It's not CREATE DATABASE
|
||||
auto table_id = context.resolveStorageID(*query_with_table, Context::ResolveOrdinary);
|
||||
auto table_id = context.tryResolveStorageID(*query_with_table, Context::ResolveOrdinary);
|
||||
storage = DatabaseCatalog::instance().tryGetTable(table_id);
|
||||
}
|
||||
|
||||
|
@ -4,9 +4,14 @@
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
#include <Poco/File.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Storages/StorageMemory.h>
|
||||
#include <Core/BackgroundSchedulePool.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -20,6 +25,7 @@ namespace ErrorCodes
|
||||
extern const int DATABASE_NOT_EMPTY;
|
||||
extern const int DATABASE_ACCESS_DENIED;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NULL_POINTER_DEREFERENCE;
|
||||
}
|
||||
|
||||
TemporaryTableHolder::TemporaryTableHolder(const Context & context_,
|
||||
@ -81,7 +87,7 @@ TemporaryTableHolder & TemporaryTableHolder::operator = (TemporaryTableHolder &&
|
||||
TemporaryTableHolder::~TemporaryTableHolder()
|
||||
{
|
||||
if (id != UUIDHelpers::Nil)
|
||||
temporary_tables->removeTable(*global_context, "_tmp_" + toString(id));
|
||||
temporary_tables->dropTable(*global_context, "_tmp_" + toString(id));
|
||||
}
|
||||
|
||||
StorageID TemporaryTableHolder::getGlobalTableID() const
|
||||
@ -91,7 +97,7 @@ StorageID TemporaryTableHolder::getGlobalTableID() const
|
||||
|
||||
StoragePtr TemporaryTableHolder::getTable() const
|
||||
{
|
||||
auto table = temporary_tables->tryGetTable(*global_context, "_tmp_" + toString(id));
|
||||
auto table = temporary_tables->tryGetTable("_tmp_" + toString(id));
|
||||
if (!table)
|
||||
throw Exception("Temporary table " + getGlobalTableID().getNameForLogs() + " not found", ErrorCodes::LOGICAL_ERROR);
|
||||
return table;
|
||||
@ -100,13 +106,22 @@ StoragePtr TemporaryTableHolder::getTable() const
|
||||
|
||||
void DatabaseCatalog::loadDatabases()
|
||||
{
|
||||
drop_delay_sec = global_context->getConfigRef().getInt("database_atomic_delay_before_drop_table_sec", 60);
|
||||
|
||||
auto db_for_temporary_and_external_tables = std::make_shared<DatabaseMemory>(TEMPORARY_DATABASE);
|
||||
attachDatabase(TEMPORARY_DATABASE, db_for_temporary_and_external_tables);
|
||||
|
||||
loadMarkedAsDroppedTables();
|
||||
auto task_holder = global_context->getSchedulePool().createTask("DatabaseCatalog", [this](){ this->dropTableDataTask(); });
|
||||
drop_task = std::make_unique<BackgroundSchedulePoolTaskHolder>(std::move(task_holder));
|
||||
(*drop_task)->activateAndSchedule();
|
||||
}
|
||||
|
||||
void DatabaseCatalog::shutdown()
|
||||
void DatabaseCatalog::shutdownImpl()
|
||||
{
|
||||
if (drop_task)
|
||||
(*drop_task)->deactivate();
|
||||
|
||||
/** At this point, some tables may have threads that block our mutex.
|
||||
* To shutdown them correctly, we will copy the current list of tables,
|
||||
* and ask them all to finish their work.
|
||||
@ -124,13 +139,8 @@ void DatabaseCatalog::shutdown()
|
||||
for (auto & database : current_databases)
|
||||
database.second->shutdown();
|
||||
|
||||
|
||||
std::lock_guard lock(databases_mutex);
|
||||
for (auto & elem : uuid_map)
|
||||
{
|
||||
std::lock_guard map_lock(elem.mutex);
|
||||
elem.map.clear();
|
||||
}
|
||||
assert(std::find_if_not(uuid_map.begin(), uuid_map.end(), [](const auto & elem) { return elem.map.empty(); }) == uuid_map.end());
|
||||
databases.clear();
|
||||
view_dependencies.clear();
|
||||
}
|
||||
@ -149,7 +159,6 @@ DatabaseAndTable DatabaseCatalog::tryGetByUUID(const UUID & uuid) const
|
||||
|
||||
DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
const StorageID & table_id,
|
||||
const Context & local_context,
|
||||
std::optional<Exception> * exception) const
|
||||
{
|
||||
if (!table_id)
|
||||
@ -197,7 +206,7 @@ DatabaseAndTable DatabaseCatalog::getTableImpl(
|
||||
database = it->second;
|
||||
}
|
||||
|
||||
auto table = database->tryGetTable(local_context, table_id.table_name);
|
||||
auto table = database->tryGetTable(table_id.table_name);
|
||||
if (!table && exception)
|
||||
exception->emplace("Table " + table_id.getNameForLogs() + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
@ -250,8 +259,15 @@ DatabasePtr DatabaseCatalog::detachDatabase(const String & database_name, bool d
|
||||
assertDatabaseExistsUnlocked(database_name);
|
||||
db = databases.find(database_name)->second;
|
||||
|
||||
if (check_empty && !db->empty(*global_context))
|
||||
throw Exception("New table appeared in database being dropped or detached. Try again.", ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
if (check_empty)
|
||||
{
|
||||
if (!db->empty())
|
||||
throw Exception("New table appeared in database being dropped or detached. Try again.",
|
||||
ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
auto * database_atomic = typeid_cast<DatabaseAtomic *>(db.get());
|
||||
if (!drop && database_atomic)
|
||||
database_atomic->assertCanBeDetached(false);
|
||||
}
|
||||
|
||||
databases.erase(database_name);
|
||||
}
|
||||
@ -315,7 +331,7 @@ bool DatabaseCatalog::isTableExist(const DB::StorageID & table_id) const
|
||||
if (iter != databases.end())
|
||||
db = iter->second;
|
||||
}
|
||||
return db && db->isTableExist(*global_context, table_id.table_name);
|
||||
return db && db->isTableExist(table_id.table_name);
|
||||
}
|
||||
|
||||
void DatabaseCatalog::assertTableDoesntExist(const StorageID & table_id) const
|
||||
@ -353,14 +369,25 @@ void DatabaseCatalog::removeUUIDMapping(const UUID & uuid)
|
||||
throw Exception("Mapping for table with UUID=" + toString(uuid) + " doesn't exist", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
DatabaseCatalog::DatabaseCatalog(const Context * global_context_)
|
||||
void DatabaseCatalog::updateUUIDMapping(const UUID & uuid, DatabasePtr database, StoragePtr table)
|
||||
{
|
||||
assert(uuid != UUIDHelpers::Nil && getFirstLevelIdx(uuid) < uuid_map.size());
|
||||
UUIDToStorageMapPart & map_part = uuid_map[getFirstLevelIdx(uuid)];
|
||||
std::lock_guard lock{map_part.mutex};
|
||||
auto it = map_part.map.find(uuid);
|
||||
if (it == map_part.map.end())
|
||||
throw Exception("Mapping for table with UUID=" + toString(uuid) + " doesn't exist", ErrorCodes::LOGICAL_ERROR);
|
||||
it->second = std::make_pair(std::move(database), std::move(table));
|
||||
}
|
||||
|
||||
DatabaseCatalog::DatabaseCatalog(Context * global_context_)
|
||||
: global_context(global_context_), log(&Poco::Logger::get("DatabaseCatalog"))
|
||||
{
|
||||
if (!global_context)
|
||||
throw Exception("DatabaseCatalog is not initialized. It's a bug.", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("DatabaseCatalog is not initialized. It's a bug.", ErrorCodes::NULL_POINTER_DEREFERENCE);
|
||||
}
|
||||
|
||||
DatabaseCatalog & DatabaseCatalog::init(const Context * global_context_)
|
||||
DatabaseCatalog & DatabaseCatalog::init(Context * global_context_)
|
||||
{
|
||||
static DatabaseCatalog database_catalog(global_context_);
|
||||
return database_catalog;
|
||||
@ -371,6 +398,23 @@ DatabaseCatalog & DatabaseCatalog::instance()
|
||||
return init(nullptr);
|
||||
}
|
||||
|
||||
void DatabaseCatalog::shutdown()
|
||||
{
|
||||
try
|
||||
{
|
||||
instance().shutdownImpl();
|
||||
}
|
||||
catch (const Exception & e)
|
||||
{
|
||||
/// If catalog was not initialized yet by init(global_context), instance() throws NULL_POINTER_DEREFERENCE.
|
||||
/// It can happen if some exception was thrown on first steps of startup (e.g. command line arguments parsing).
|
||||
/// Ignore it.
|
||||
if (e.code() == ErrorCodes::NULL_POINTER_DEREFERENCE)
|
||||
return;
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
DatabasePtr DatabaseCatalog::getDatabase(const String & database_name, const Context & local_context) const
|
||||
{
|
||||
String resolved_database = local_context.resolveDatabase(database_name);
|
||||
@ -421,13 +465,13 @@ std::unique_ptr<DDLGuard> DatabaseCatalog::getDDLGuard(const String & database,
|
||||
bool DatabaseCatalog::isDictionaryExist(const StorageID & table_id) const
|
||||
{
|
||||
auto db = tryGetDatabase(table_id.getDatabaseName());
|
||||
return db && db->isDictionaryExist(*global_context, table_id.getTableName());
|
||||
return db && db->isDictionaryExist(table_id.getTableName());
|
||||
}
|
||||
|
||||
StoragePtr DatabaseCatalog::getTable(const StorageID & table_id) const
|
||||
{
|
||||
std::optional<Exception> exc;
|
||||
auto res = getTableImpl(table_id, *global_context, &exc);
|
||||
auto res = getTableImpl(table_id, &exc);
|
||||
if (!res.second)
|
||||
throw Exception(*exc);
|
||||
return res.second;
|
||||
@ -435,12 +479,224 @@ StoragePtr DatabaseCatalog::getTable(const StorageID & table_id) const
|
||||
|
||||
StoragePtr DatabaseCatalog::tryGetTable(const StorageID & table_id) const
|
||||
{
|
||||
return getTableImpl(table_id, *global_context, nullptr).second;
|
||||
return getTableImpl(table_id, nullptr).second;
|
||||
}
|
||||
|
||||
DatabaseAndTable DatabaseCatalog::getDatabaseAndTable(const StorageID & table_id) const
|
||||
{
|
||||
std::optional<Exception> exc;
|
||||
auto res = getTableImpl(table_id, &exc);
|
||||
if (!res.second)
|
||||
throw Exception(*exc);
|
||||
return res;
|
||||
}
|
||||
|
||||
DatabaseAndTable DatabaseCatalog::tryGetDatabaseAndTable(const StorageID & table_id) const
|
||||
{
|
||||
return getTableImpl(table_id, *global_context, nullptr);
|
||||
return getTableImpl(table_id, nullptr);
|
||||
}
|
||||
|
||||
void DatabaseCatalog::loadMarkedAsDroppedTables()
|
||||
{
|
||||
/// /clickhouse_root/metadata_dropped/ contains files with metadata of tables,
|
||||
/// which where marked as dropped by Atomic databases.
|
||||
/// Data directories of such tables still exists in store/
|
||||
/// and metadata still exists in ZooKeeper for ReplicatedMergeTree tables.
|
||||
/// If server restarts before such tables was completely dropped,
|
||||
/// we should load them and enqueue cleanup to remove data from store/ and metadata from ZooKeeper
|
||||
|
||||
std::map<String, StorageID> dropped_metadata;
|
||||
String path = global_context->getPath() + "metadata_dropped/";
|
||||
Poco::DirectoryIterator dir_end;
|
||||
for (Poco::DirectoryIterator it(path); it != dir_end; ++it)
|
||||
{
|
||||
/// File name has the following format:
|
||||
/// database_name.table_name.uuid.sql
|
||||
|
||||
/// Ignore unexpected files
|
||||
if (!it.name().ends_with(".sql"))
|
||||
continue;
|
||||
|
||||
/// Process .sql files with metadata of tables which were marked as dropped
|
||||
StorageID dropped_id = StorageID::createEmpty();
|
||||
size_t dot_pos = it.name().find('.');
|
||||
if (dot_pos == std::string::npos)
|
||||
continue;
|
||||
dropped_id.database_name = unescapeForFileName(it.name().substr(0, dot_pos));
|
||||
|
||||
size_t prev_dot_pos = dot_pos;
|
||||
dot_pos = it.name().find('.', prev_dot_pos + 1);
|
||||
if (dot_pos == std::string::npos)
|
||||
continue;
|
||||
dropped_id.table_name = unescapeForFileName(it.name().substr(prev_dot_pos + 1, dot_pos - prev_dot_pos - 1));
|
||||
|
||||
prev_dot_pos = dot_pos;
|
||||
dot_pos = it.name().find('.', prev_dot_pos + 1);
|
||||
if (dot_pos == std::string::npos)
|
||||
continue;
|
||||
dropped_id.uuid = parse<UUID>(it.name().substr(prev_dot_pos + 1, dot_pos - prev_dot_pos - 1));
|
||||
|
||||
String full_path = path + it.name();
|
||||
dropped_metadata.emplace(std::move(full_path), std::move(dropped_id));
|
||||
}
|
||||
|
||||
ThreadPool pool(SettingMaxThreads().getAutoValue());
|
||||
for (const auto & elem : dropped_metadata)
|
||||
{
|
||||
pool.scheduleOrThrowOnError([&]()
|
||||
{
|
||||
this->enqueueDroppedTableCleanup(elem.second, nullptr, elem.first);
|
||||
});
|
||||
}
|
||||
pool.wait();
|
||||
}
|
||||
|
||||
String DatabaseCatalog::getPathForDroppedMetadata(const StorageID & table_id) const
|
||||
{
|
||||
return global_context->getPath() + "metadata_dropped/" +
|
||||
escapeForFileName(table_id.getDatabaseName()) + "." +
|
||||
escapeForFileName(table_id.getTableName()) + "." +
|
||||
toString(table_id.uuid) + ".sql";
|
||||
}
|
||||
|
||||
void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay)
|
||||
{
|
||||
assert(table_id.hasUUID());
|
||||
assert(!table || table->getStorageID().uuid == table_id.uuid);
|
||||
assert(dropped_metadata_path == getPathForDroppedMetadata(table_id));
|
||||
|
||||
/// Table was removed from database. Enqueue removal of its data from disk.
|
||||
time_t drop_time;
|
||||
if (table)
|
||||
drop_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
|
||||
else
|
||||
{
|
||||
/// Try load table from metadata to drop it correctly (e.g. remove metadata from zk or remove data from all volumes)
|
||||
LOG_INFO(log, "Trying load partially dropped table " << table_id.getNameForLogs() << " from " << dropped_metadata_path);
|
||||
ASTPtr ast = DatabaseOnDisk::parseQueryFromMetadata(log, *global_context, dropped_metadata_path, /*throw_on_error*/ false, /*remove_empty*/false);
|
||||
auto * create = typeid_cast<ASTCreateQuery *>(ast.get());
|
||||
assert(!create || create->uuid == table_id.uuid);
|
||||
|
||||
if (create)
|
||||
{
|
||||
String data_path = "store/" + getPathForUUID(table_id.uuid);
|
||||
create->database = table_id.database_name;
|
||||
create->table = table_id.table_name;
|
||||
try
|
||||
{
|
||||
table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, *global_context, false).second;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Cannot load partially dropped table " + table_id.getNameForLogs() +
|
||||
" from: " + dropped_metadata_path +
|
||||
". Parsed query: " + serializeAST(*create) +
|
||||
". Will remove metadata and " + data_path +
|
||||
". Garbage may be left in ZooKeeper.");
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_WARNING(log, "Cannot parse metadata of partially dropped table "
|
||||
<< table_id.getNameForLogs() << " from " << dropped_metadata_path
|
||||
<< ". Will remove metadata file and data directory. Garbage may be left in /store directory and ZooKeeper.");
|
||||
}
|
||||
|
||||
drop_time = Poco::File(dropped_metadata_path).getLastModified().epochTime();
|
||||
}
|
||||
|
||||
std::lock_guard lock(tables_marked_dropped_mutex);
|
||||
if (ignore_delay)
|
||||
tables_marked_dropped.push_front({table_id, table, dropped_metadata_path, 0});
|
||||
else
|
||||
tables_marked_dropped.push_back({table_id, table, dropped_metadata_path, drop_time});
|
||||
/// If list of dropped tables was empty, start a drop task
|
||||
if (drop_task && tables_marked_dropped.size() == 1)
|
||||
(*drop_task)->schedule();
|
||||
}
|
||||
|
||||
void DatabaseCatalog::dropTableDataTask()
|
||||
{
|
||||
/// Background task that removes data of tables which were marked as dropped by Atomic databases.
|
||||
/// Table can be removed when it's not used by queries and drop_delay_sec elapsed since it was marked as dropped.
|
||||
|
||||
bool need_reschedule = true;
|
||||
TableMarkedAsDropped table;
|
||||
try
|
||||
{
|
||||
std::lock_guard lock(tables_marked_dropped_mutex);
|
||||
time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
|
||||
auto it = std::find_if(tables_marked_dropped.begin(), tables_marked_dropped.end(), [&](const auto & elem)
|
||||
{
|
||||
bool not_in_use = !elem.table || elem.table.unique();
|
||||
bool old_enough = elem.drop_time + drop_delay_sec < current_time;
|
||||
return not_in_use && old_enough;
|
||||
});
|
||||
if (it != tables_marked_dropped.end())
|
||||
{
|
||||
table = std::move(*it);
|
||||
LOG_INFO(log, "Will try drop " + table.table_id.getNameForLogs());
|
||||
tables_marked_dropped.erase(it);
|
||||
}
|
||||
need_reschedule = !tables_marked_dropped.empty();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, __PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
if (table.table_id)
|
||||
{
|
||||
|
||||
try
|
||||
{
|
||||
dropTableFinally(table);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Cannot drop table " + table.table_id.getNameForLogs() +
|
||||
". Will retry later.");
|
||||
{
|
||||
std::lock_guard lock(tables_marked_dropped_mutex);
|
||||
tables_marked_dropped.emplace_back(std::move(table));
|
||||
/// If list of dropped tables was empty, schedule a task to retry deletion.
|
||||
if (tables_marked_dropped.size() == 1)
|
||||
need_reschedule = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Do not schedule a task if there is no tables to drop
|
||||
if (need_reschedule)
|
||||
(*drop_task)->scheduleAfter(reschedule_time_ms);
|
||||
}
|
||||
|
||||
void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) const
|
||||
{
|
||||
if (table.table)
|
||||
{
|
||||
table.table->drop();
|
||||
table.table->is_dropped = true;
|
||||
}
|
||||
|
||||
/// Even if table is not loaded, try remove its data from disk.
|
||||
/// TODO remove data from all volumes
|
||||
String data_path = global_context->getPath() + "store/" + getPathForUUID(table.table_id.uuid);
|
||||
Poco::File table_data_dir{data_path};
|
||||
if (table_data_dir.exists())
|
||||
{
|
||||
LOG_INFO(log, "Removing data directory " << data_path << " of dropped table " << table.table_id.getNameForLogs());
|
||||
table_data_dir.remove(true);
|
||||
}
|
||||
|
||||
LOG_INFO(log, "Removing metadata " << table.metadata_path << " of dropped table " << table.table_id.getNameForLogs());
|
||||
Poco::File(table.metadata_path).remove();
|
||||
}
|
||||
|
||||
String DatabaseCatalog::getPathForUUID(const UUID & uuid)
|
||||
{
|
||||
const size_t uuid_prefix_len = 3;
|
||||
return toString(uuid).substr(0, uuid_prefix_len) + '/' + toString(uuid) + '/';
|
||||
}
|
||||
|
||||
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <unordered_map>
|
||||
#include <mutex>
|
||||
#include <array>
|
||||
#include <list>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -91,6 +92,7 @@ struct TemporaryTableHolder : boost::noncopyable
|
||||
///TODO maybe remove shared_ptr from here?
|
||||
using TemporaryTablesMapping = std::map<String, std::shared_ptr<TemporaryTableHolder>>;
|
||||
|
||||
class BackgroundSchedulePoolTaskHolder;
|
||||
|
||||
class DatabaseCatalog : boost::noncopyable
|
||||
{
|
||||
@ -98,11 +100,11 @@ public:
|
||||
static constexpr const char * TEMPORARY_DATABASE = "_temporary_and_external_tables";
|
||||
static constexpr const char * SYSTEM_DATABASE = "system";
|
||||
|
||||
static DatabaseCatalog & init(const Context * global_context_);
|
||||
static DatabaseCatalog & init(Context * global_context_);
|
||||
static DatabaseCatalog & instance();
|
||||
static void shutdown();
|
||||
|
||||
void loadDatabases();
|
||||
void shutdown();
|
||||
|
||||
/// Get an object that protects the table from concurrently executing multiple DDL operations.
|
||||
std::unique_ptr<DDLGuard> getDDLGuard(const String & database, const String & table);
|
||||
@ -133,9 +135,9 @@ public:
|
||||
|
||||
StoragePtr getTable(const StorageID & table_id) const;
|
||||
StoragePtr tryGetTable(const StorageID & table_id) const;
|
||||
DatabaseAndTable getDatabaseAndTable(const StorageID & table_id) const { return getTableImpl(table_id, *global_context); }
|
||||
DatabaseAndTable getDatabaseAndTable(const StorageID & table_id) const;
|
||||
DatabaseAndTable tryGetDatabaseAndTable(const StorageID & table_id) const;
|
||||
DatabaseAndTable getTableImpl(const StorageID & table_id, const Context & local_context, std::optional<Exception> * exception = nullptr) const;
|
||||
DatabaseAndTable getTableImpl(const StorageID & table_id, std::optional<Exception> * exception = nullptr) const;
|
||||
|
||||
void addDependency(const StorageID & from, const StorageID & where);
|
||||
void removeDependency(const StorageID & from, const StorageID & where);
|
||||
@ -149,13 +151,24 @@ public:
|
||||
/// Such tables can be accessed by persistent UUID instead of database and table name.
|
||||
void addUUIDMapping(const UUID & uuid, DatabasePtr database, StoragePtr table);
|
||||
void removeUUIDMapping(const UUID & uuid);
|
||||
/// For moving table between databases
|
||||
void updateUUIDMapping(const UUID & uuid, DatabasePtr database, StoragePtr table);
|
||||
|
||||
static String getPathForUUID(const UUID & uuid);
|
||||
|
||||
DatabaseAndTable tryGetByUUID(const UUID & uuid) const;
|
||||
|
||||
String getPathForDroppedMetadata(const StorageID & table_id) const;
|
||||
void enqueueDroppedTableCleanup(StorageID table_id, StoragePtr table, String dropped_metadata_path, bool ignore_delay = false);
|
||||
|
||||
private:
|
||||
DatabaseCatalog(const Context * global_context_);
|
||||
DatabaseCatalog(Context * global_context_);
|
||||
void assertDatabaseExistsUnlocked(const String & database_name) const;
|
||||
void assertDatabaseDoesntExistUnlocked(const String & database_name) const;
|
||||
|
||||
void shutdownImpl();
|
||||
|
||||
|
||||
struct UUIDToStorageMapPart
|
||||
{
|
||||
std::unordered_map<UUID, DatabaseAndTable> map;
|
||||
@ -170,9 +183,24 @@ private:
|
||||
return uuid.toUnderType().low >> (64 - bits_for_first_level);
|
||||
}
|
||||
|
||||
struct TableMarkedAsDropped
|
||||
{
|
||||
StorageID table_id = StorageID::createEmpty();
|
||||
StoragePtr table;
|
||||
String metadata_path;
|
||||
time_t drop_time;
|
||||
};
|
||||
using TablesMarkedAsDropped = std::list<TableMarkedAsDropped>;
|
||||
|
||||
void loadMarkedAsDroppedTables();
|
||||
void dropTableDataTask();
|
||||
void dropTableFinally(const TableMarkedAsDropped & table) const;
|
||||
|
||||
static constexpr size_t reschedule_time_ms = 100;
|
||||
|
||||
private:
|
||||
/// For some reason Context is required to get Storage from Database object
|
||||
const Context * global_context;
|
||||
Context * global_context;
|
||||
mutable std::mutex databases_mutex;
|
||||
|
||||
ViewDependencies view_dependencies;
|
||||
@ -191,6 +219,12 @@ private:
|
||||
DDLGuards ddl_guards;
|
||||
/// If you capture mutex and ddl_guards_mutex, then you need to grab them strictly in this order.
|
||||
mutable std::mutex ddl_guards_mutex;
|
||||
|
||||
TablesMarkedAsDropped tables_marked_dropped;
|
||||
mutable std::mutex tables_marked_dropped_mutex;
|
||||
|
||||
std::unique_ptr<BackgroundSchedulePoolTaskHolder> drop_task;
|
||||
time_t drop_delay_sec = 60;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ LoadablesConfigurationPtr ExternalLoaderDatabaseConfigRepository::load(const std
|
||||
|
||||
bool ExternalLoaderDatabaseConfigRepository::exists(const std::string & loadable_definition_name)
|
||||
{
|
||||
return database.isDictionaryExist(context, trimDatabaseName(loadable_definition_name, database));
|
||||
return database.isDictionaryExist(trimDatabaseName(loadable_definition_name, database));
|
||||
}
|
||||
|
||||
Poco::Timestamp ExternalLoaderDatabaseConfigRepository::getUpdateTime(const std::string & loadable_definition_name)
|
||||
@ -51,7 +51,7 @@ std::set<std::string> ExternalLoaderDatabaseConfigRepository::getAllLoadablesDef
|
||||
{
|
||||
std::set<std::string> result;
|
||||
const auto & dbname = database.getDatabaseName();
|
||||
auto itr = database.getDictionariesIterator(context);
|
||||
auto itr = database.getDictionariesIterator();
|
||||
while (itr && itr->isValid())
|
||||
{
|
||||
result.insert(dbname + "." + itr->name());
|
||||
|
@ -10,6 +10,7 @@
|
||||
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
@ -95,9 +96,14 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
|
||||
if (!create.storage)
|
||||
{
|
||||
/// For new-style databases engine is explicitly specified in .sql
|
||||
/// When attaching old-style database during server startup, we must always use Ordinary engine
|
||||
if (create.attach)
|
||||
throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||
bool old_style_database = context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary;
|
||||
auto engine = std::make_shared<ASTFunction>();
|
||||
auto storage = std::make_shared<ASTStorage>();
|
||||
engine->name = "Ordinary";
|
||||
engine->name = old_style_database ? "Ordinary" : "Atomic";
|
||||
storage->set(storage->engine, engine);
|
||||
create.set(create.storage, storage);
|
||||
}
|
||||
@ -109,9 +115,11 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
||||
}
|
||||
|
||||
String database_name_escaped = escapeForFileName(database_name);
|
||||
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);
|
||||
|
||||
/// Create directories for tables metadata.
|
||||
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);
|
||||
@ -146,7 +154,7 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
bool renamed = false;
|
||||
try
|
||||
{
|
||||
//FIXME is it possible to attach db only after it was loaded? (no, loadStoredObjects adds view dependencies)
|
||||
/// TODO Attach db only after it was loaded. Now it's not possible because of view dependencies
|
||||
DatabaseCatalog::instance().attachDatabase(database_name, database);
|
||||
added = true;
|
||||
|
||||
@ -436,7 +444,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(AS
|
||||
create.columns_list->setOrReplace(create.columns_list->constraints, new_constraints);
|
||||
|
||||
validateTableStructure(create, properties);
|
||||
|
||||
/// Set the table engine if it was not specified explicitly.
|
||||
setEngine(create);
|
||||
return properties;
|
||||
@ -497,7 +504,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
|
||||
String as_database_name = context.resolveDatabase(create.as_database);
|
||||
String as_table_name = create.as_table;
|
||||
|
||||
ASTPtr as_create_ptr = DatabaseCatalog::instance().getDatabase(as_database_name)->getCreateTableQuery(context, as_table_name);
|
||||
ASTPtr as_create_ptr = DatabaseCatalog::instance().getDatabase(as_database_name)->getCreateTableQuery(as_table_name);
|
||||
const auto & as_create = as_create_ptr->as<ASTCreateQuery &>();
|
||||
|
||||
const String qualified_name = backQuoteIfNeed(as_database_name) + "." + backQuoteIfNeed(as_table_name);
|
||||
@ -534,19 +541,22 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
// If this is a stub ATTACH query, read the query definition from the database
|
||||
if (create.attach && !create.storage && !create.columns_list)
|
||||
{
|
||||
bool if_not_exists = create.if_not_exists;
|
||||
// Table SQL definition is available even if the table is detached
|
||||
auto database_name = create.database.empty() ? current_database : create.database;
|
||||
auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateTableQuery(context, create.table);
|
||||
auto database = DatabaseCatalog::instance().getDatabase(database_name);
|
||||
bool if_not_exists = create.if_not_exists;
|
||||
|
||||
// Table SQL definition is available even if the table is detached
|
||||
auto query = database->getCreateTableQuery(create.table);
|
||||
create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE
|
||||
create.attach = true;
|
||||
create.attach_short_syntax = true;
|
||||
create.if_not_exists = if_not_exists;
|
||||
}
|
||||
|
||||
if (!create.temporary && create.database.empty())
|
||||
create.database = current_database;
|
||||
if (!create.to_table.empty() && create.to_database.empty())
|
||||
create.to_database = current_database;
|
||||
if (create.to_table_id && create.to_table_id.database_name.empty())
|
||||
create.to_table_id.database_name = current_database;
|
||||
|
||||
if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view))
|
||||
{
|
||||
@ -559,25 +569,38 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
|
||||
/// Actually creates table
|
||||
bool created = doCreateTable(create, properties);
|
||||
|
||||
if (!created) /// Table already exists
|
||||
return {};
|
||||
|
||||
return fillTableIfNeeded(create);
|
||||
}
|
||||
|
||||
bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
|
||||
bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
||||
const InterpreterCreateQuery::TableProperties & properties)
|
||||
{
|
||||
std::unique_ptr<DDLGuard> guard;
|
||||
|
||||
String data_path;
|
||||
DatabasePtr database;
|
||||
|
||||
String table_name = create.table;
|
||||
const String table_name = create.table;
|
||||
bool need_add_to_database = !create.temporary;
|
||||
if (need_add_to_database)
|
||||
{
|
||||
database = DatabaseCatalog::instance().getDatabase(create.database);
|
||||
if (database->getEngineName() == "Atomic")
|
||||
{
|
||||
/// TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/
|
||||
if (create.attach && create.uuid == UUIDHelpers::Nil)
|
||||
throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY);
|
||||
if (!create.attach && create.uuid == UUIDHelpers::Nil)
|
||||
create.uuid = UUIDHelpers::generateV4();
|
||||
}
|
||||
else
|
||||
{
|
||||
if (create.uuid != UUIDHelpers::Nil)
|
||||
throw Exception("Table UUID specified, but engine of database " + create.database + " is not Atomic", ErrorCodes::INCORRECT_QUERY);
|
||||
}
|
||||
|
||||
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
|
||||
* If table doesnt exist, one thread is creating table, while others wait in DDLGuard.
|
||||
@ -585,7 +608,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
|
||||
guard = DatabaseCatalog::instance().getDDLGuard(create.database, table_name);
|
||||
|
||||
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
|
||||
if (database->isTableExist(context, table_name))
|
||||
if (database->isTableExist(table_name))
|
||||
{
|
||||
/// TODO Check structure of table
|
||||
if (create.if_not_exists)
|
||||
@ -604,10 +627,14 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
|
||||
else
|
||||
throw Exception("Table " + create.database + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
data_path = database->getTableDataPath(create);
|
||||
if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists())
|
||||
throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (context.tryResolveStorageID({"", table_name}, Context::ResolveExternal) && create.if_not_exists)
|
||||
if (create.if_not_exists && context.tryResolveStorageID({"", table_name}, Context::ResolveExternal))
|
||||
return false;
|
||||
|
||||
auto temporary_table = TemporaryTableHolder(context, properties.columns, query_ptr);
|
||||
@ -621,8 +648,8 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
|
||||
{
|
||||
const auto & table_function = create.as_table_function->as<ASTFunction &>();
|
||||
const auto & factory = TableFunctionFactory::instance();
|
||||
//FIXME storage will have wrong database name
|
||||
res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table);
|
||||
res->renameInMemory({create.database, create.table, create.uuid});
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -681,7 +708,7 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
|
||||
auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, dictionary_name);
|
||||
DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name);
|
||||
|
||||
if (database->isDictionaryExist(context, dictionary_name))
|
||||
if (database->isDictionaryExist(dictionary_name))
|
||||
{
|
||||
/// TODO Check structure of dictionary
|
||||
if (create.if_not_exists)
|
||||
@ -693,13 +720,9 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
|
||||
|
||||
if (create.attach)
|
||||
{
|
||||
auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateDictionaryQuery(context, dictionary_name);
|
||||
auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateDictionaryQuery(dictionary_name);
|
||||
create = query->as<ASTCreateQuery &>();
|
||||
create.attach = true;
|
||||
}
|
||||
|
||||
if (create.attach)
|
||||
{
|
||||
auto config = getDictionaryConfigurationFromAST(create);
|
||||
auto modification_time = database->getObjectMetadataModificationTime(dictionary_name);
|
||||
database->attachDictionary(dictionary_name, DictionaryAttachInfo{query_ptr, config, modification_time});
|
||||
@ -714,7 +737,11 @@ BlockIO InterpreterCreateQuery::execute()
|
||||
{
|
||||
auto & create = query_ptr->as<ASTCreateQuery &>();
|
||||
if (!create.cluster.empty())
|
||||
{
|
||||
/// NOTE: if it's CREATE query and create.database is DatabaseAtomic, different UUIDs will be generated on all servers.
|
||||
/// However, it allows to use UUID as replica name.
|
||||
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess());
|
||||
}
|
||||
|
||||
context.checkAccess(getRequiredAccess());
|
||||
|
||||
@ -767,8 +794,8 @@ AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const
|
||||
required_access.emplace_back(AccessType::CREATE_TABLE, create.database, create.table);
|
||||
}
|
||||
|
||||
if (!create.to_table.empty())
|
||||
required_access.emplace_back(AccessType::SELECT | AccessType::INSERT, create.to_database, create.to_table);
|
||||
if (create.to_table_id)
|
||||
required_access.emplace_back(AccessType::SELECT | AccessType::INSERT, create.to_table_id.database_name, create.to_table_id.table_name);
|
||||
|
||||
if (create.storage && create.storage->engine)
|
||||
{
|
||||
|
@ -68,7 +68,7 @@ private:
|
||||
AccessRightsElements getRequiredAccess() const;
|
||||
|
||||
/// Create IStorage and add it to database. If table already exists and IF NOT EXISTS specified, do nothing and return false.
|
||||
bool doCreateTable(const ASTCreateQuery & create, const TableProperties & properties);
|
||||
bool doCreateTable(ASTCreateQuery & create, const TableProperties & properties);
|
||||
/// Inserts data in created table if it's CREATE ... SELECT
|
||||
BlockIO fillTableIfNeeded(const ASTCreateQuery & create);
|
||||
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Databases/DatabaseAtomic.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -43,7 +44,7 @@ BlockIO InterpreterDropQuery::execute()
|
||||
if (!drop.table.empty())
|
||||
{
|
||||
if (!drop.is_dictionary)
|
||||
return executeToTable(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
|
||||
return executeToTable({drop.database, drop.table, drop.uuid}, drop);
|
||||
else
|
||||
return executeToDictionary(drop.database, drop.table, drop.kind, drop.if_exists, drop.temporary, drop.no_ddl_lock);
|
||||
}
|
||||
@ -55,101 +56,70 @@ BlockIO InterpreterDropQuery::execute()
|
||||
|
||||
|
||||
BlockIO InterpreterDropQuery::executeToTable(
|
||||
const String & database_name_,
|
||||
const String & table_name,
|
||||
ASTDropQuery::Kind kind,
|
||||
bool if_exists,
|
||||
bool is_temporary,
|
||||
bool no_ddl_lock)
|
||||
const StorageID & table_id_,
|
||||
const ASTDropQuery & query)
|
||||
{
|
||||
if (is_temporary || database_name_.empty())
|
||||
if (query.temporary || table_id_.database_name.empty())
|
||||
{
|
||||
if (context.tryResolveStorageID({"", table_name}, Context::ResolveExternal))
|
||||
return executeToTemporaryTable(table_name, kind);
|
||||
if (context.tryResolveStorageID(table_id_, Context::ResolveExternal))
|
||||
return executeToTemporaryTable(table_id_.getTableName(), query.kind);
|
||||
}
|
||||
|
||||
if (is_temporary)
|
||||
if (query.temporary)
|
||||
{
|
||||
if (if_exists)
|
||||
if (query.if_exists)
|
||||
return {};
|
||||
throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist",
|
||||
throw Exception("Temporary table " + backQuoteIfNeed(table_id_.table_name) + " doesn't exist",
|
||||
ErrorCodes::UNKNOWN_TABLE);
|
||||
}
|
||||
|
||||
String database_name = context.resolveDatabase(database_name_);
|
||||
auto table_id = query.if_exists ? context.tryResolveStorageID(table_id_, Context::ResolveOrdinary)
|
||||
: context.resolveStorageID(table_id_, Context::ResolveOrdinary);
|
||||
if (!table_id)
|
||||
return {};
|
||||
|
||||
auto ddl_guard = (!no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(database_name, table_name) : nullptr);
|
||||
auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr);
|
||||
|
||||
auto [database, table] = tryGetDatabaseAndTable(database_name, table_name, if_exists);
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id)
|
||||
: DatabaseCatalog::instance().getDatabaseAndTable(table_id);
|
||||
|
||||
if (database && table)
|
||||
{
|
||||
if (query_ptr->as<ASTDropQuery &>().is_view && !table->isView())
|
||||
throw Exception("Table " + backQuoteIfNeed(table_name) + " is not a View", ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception("Table " + table_id.getNameForLogs() + " is not a View", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
auto table_id = table->getStorageID();
|
||||
if (kind == ASTDropQuery::Kind::Detach)
|
||||
if (query.kind == ASTDropQuery::Kind::Detach)
|
||||
{
|
||||
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
|
||||
table->shutdown();
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
TableStructureWriteLockHolder table_lock;
|
||||
if (database->getEngineName() != "Atomic")
|
||||
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
/// Drop table from memory, don't touch data and metadata
|
||||
database->detachTable(table_name);
|
||||
database->detachTable(table_id.table_name);
|
||||
}
|
||||
else if (kind == ASTDropQuery::Kind::Truncate)
|
||||
else if (query.kind == ASTDropQuery::Kind::Truncate)
|
||||
{
|
||||
context.checkAccess(AccessType::TRUNCATE, table_id);
|
||||
table->checkTableCanBeDropped();
|
||||
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
/// Drop table data, don't touch metadata
|
||||
table->truncate(query_ptr, context, table_lock);
|
||||
}
|
||||
else if (kind == ASTDropQuery::Kind::Drop)
|
||||
else if (query.kind == ASTDropQuery::Kind::Drop)
|
||||
{
|
||||
context.checkAccess(table->isView() ? AccessType::DROP_VIEW : AccessType::DROP_TABLE, table_id);
|
||||
table->checkTableCanBeDropped();
|
||||
|
||||
table->shutdown();
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
|
||||
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
TableStructureWriteLockHolder table_lock;
|
||||
if (database->getEngineName() != "Atomic")
|
||||
table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
|
||||
const std::string metadata_file_without_extension = database->getMetadataPath() + escapeForFileName(table_id.table_name);
|
||||
const auto prev_metadata_name = metadata_file_without_extension + ".sql";
|
||||
const auto drop_metadata_name = metadata_file_without_extension + ".sql.tmp_drop";
|
||||
|
||||
/// Try to rename metadata file and delete the data
|
||||
try
|
||||
{
|
||||
/// There some kind of tables that have no metadata - ignore renaming
|
||||
if (Poco::File(prev_metadata_name).exists())
|
||||
Poco::File(prev_metadata_name).renameTo(drop_metadata_name);
|
||||
/// Delete table data
|
||||
table->drop(table_lock);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (Poco::File(drop_metadata_name).exists())
|
||||
Poco::File(drop_metadata_name).renameTo(prev_metadata_name);
|
||||
throw;
|
||||
}
|
||||
|
||||
String table_data_path_relative = database->getTableDataPath(table_name);
|
||||
|
||||
/// Delete table metadata and table itself from memory
|
||||
database->removeTable(context, table_name);
|
||||
table->is_dropped = true;
|
||||
|
||||
/// If it is not virtual database like Dictionary then drop remaining data dir
|
||||
if (!table_data_path_relative.empty())
|
||||
{
|
||||
String table_data_path = context.getPath() + table_data_path_relative;
|
||||
if (Poco::File(table_data_path).exists())
|
||||
Poco::File(table_data_path).remove(true);
|
||||
}
|
||||
database->dropTable(context, table_id.table_name, query.no_delay);
|
||||
}
|
||||
}
|
||||
|
||||
@ -174,7 +144,7 @@ BlockIO InterpreterDropQuery::executeToDictionary(
|
||||
|
||||
DatabasePtr database = tryGetDatabase(database_name, if_exists);
|
||||
|
||||
if (!database || !database->isDictionaryExist(context, dictionary_name))
|
||||
if (!database || !database->isDictionaryExist(dictionary_name))
|
||||
{
|
||||
if (!if_exists)
|
||||
throw Exception(
|
||||
@ -215,9 +185,7 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name,
|
||||
StoragePtr table = DatabaseCatalog::instance().getTable(resolved_id);
|
||||
if (kind == ASTDropQuery::Kind::Truncate)
|
||||
{
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
auto table_lock =
|
||||
table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
/// Drop table data, don't touch metadata
|
||||
table->truncate(query_ptr, context, table_lock);
|
||||
}
|
||||
@ -225,11 +193,9 @@ BlockIO InterpreterDropQuery::executeToTemporaryTable(const String & table_name,
|
||||
{
|
||||
context_handle.removeExternalTable(table_name);
|
||||
table->shutdown();
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
auto table_lock =
|
||||
table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
/// Delete table data
|
||||
table->drop(table_lock);
|
||||
table->drop();
|
||||
table->is_dropped = true;
|
||||
}
|
||||
}
|
||||
@ -259,19 +225,26 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS
|
||||
/// DETACH or DROP all tables and dictionaries inside database.
|
||||
/// First we should DETACH or DROP dictionaries because StorageDictionary
|
||||
/// must be detached only by detaching corresponding dictionary.
|
||||
for (auto iterator = database->getDictionariesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = database->getDictionariesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
String current_dictionary = iterator->name();
|
||||
executeToDictionary(database_name, current_dictionary, kind, false, false, false);
|
||||
}
|
||||
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
ASTDropQuery query;
|
||||
query.kind = kind;
|
||||
query.database = database_name;
|
||||
for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
String current_table_name = iterator->name();
|
||||
executeToTable(database_name, current_table_name, kind, false, false, false);
|
||||
query.table = iterator->name();
|
||||
executeToTable({query.database, query.table}, query);
|
||||
}
|
||||
}
|
||||
|
||||
auto * database_atomic = typeid_cast<DatabaseAtomic *>(database.get());
|
||||
if (!drop && database_atomic)
|
||||
database_atomic->assertCanBeDetached(true);
|
||||
|
||||
/// DETACH or DROP database itself
|
||||
DatabaseCatalog::instance().detachDatabase(database_name, drop, database->shouldBeEmptyOnDetach());
|
||||
}
|
||||
@ -281,23 +254,6 @@ BlockIO InterpreterDropQuery::executeToDatabase(const String & database_name, AS
|
||||
}
|
||||
|
||||
|
||||
DatabaseAndTable InterpreterDropQuery::tryGetDatabaseAndTable(const String & database_name, const String & table_name, bool if_exists)
|
||||
{
|
||||
DatabasePtr database = tryGetDatabase(database_name, if_exists);
|
||||
|
||||
if (database)
|
||||
{
|
||||
StoragePtr table = database->tryGetTable(context, table_name);
|
||||
if (!table && !if_exists)
|
||||
throw Exception("Table " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(table_name) + " doesn't exist",
|
||||
ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
return {std::move(database), std::move(table)};
|
||||
}
|
||||
return {};
|
||||
}
|
||||
|
||||
|
||||
AccessRightsElements InterpreterDropQuery::getRequiredAccessForDDLOnCluster() const
|
||||
{
|
||||
AccessRightsElements required_access;
|
||||
|
@ -30,12 +30,10 @@ private:
|
||||
|
||||
BlockIO executeToDatabase(const String & database_name, ASTDropQuery::Kind kind, bool if_exists);
|
||||
|
||||
BlockIO executeToTable(const String & database_name, const String & table_name, ASTDropQuery::Kind kind, bool if_exists, bool is_temporary, bool no_ddl_lock);
|
||||
BlockIO executeToTable(const StorageID & table_id, const ASTDropQuery & query);
|
||||
|
||||
BlockIO executeToDictionary(const String & database_name, const String & dictionary_name, ASTDropQuery::Kind kind, bool if_exists, bool is_temporary, bool no_ddl_lock);
|
||||
|
||||
DatabaseAndTable tryGetDatabaseAndTable(const String & database_name, const String & table_name, bool if_exists);
|
||||
|
||||
BlockIO executeToTemporaryTable(const String & table_name, ASTDropQuery::Kind kind);
|
||||
};
|
||||
}
|
||||
|
@ -77,17 +77,15 @@ BlockIO InterpreterRenameQuery::execute()
|
||||
|
||||
for (auto & elem : descriptions)
|
||||
{
|
||||
database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name));
|
||||
auto from_table = database_catalog.getTable({elem.from_database_name, elem.from_table_name});
|
||||
auto from_table_lock =
|
||||
from_table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
if (!rename.exchange)
|
||||
database_catalog.assertTableDoesntExist(StorageID(elem.to_database_name, elem.to_table_name));
|
||||
|
||||
database_catalog.getDatabase(elem.from_database_name)->renameTable(
|
||||
context,
|
||||
elem.from_table_name,
|
||||
*database_catalog.getDatabase(elem.to_database_name),
|
||||
elem.to_table_name,
|
||||
from_table_lock);
|
||||
rename.exchange);
|
||||
}
|
||||
|
||||
return {};
|
||||
|
@ -14,7 +14,6 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/InterpreterShowCreateQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -50,7 +49,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||
auto resolve_table_type = show_query->temporary ? Context::ResolveExternal : Context::ResolveOrdinary;
|
||||
auto table_id = context.resolveStorageID(*show_query, resolve_table_type);
|
||||
context.checkAccess(AccessType::SHOW_COLUMNS, table_id);
|
||||
create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(context, table_id.table_name);
|
||||
create_query = DatabaseCatalog::instance().getDatabase(table_id.database_name)->getCreateTableQuery(table_id.table_name);
|
||||
}
|
||||
else if ((show_query = query_ptr->as<ASTShowCreateDatabaseQuery>()))
|
||||
{
|
||||
@ -58,7 +57,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||
throw Exception("Temporary databases are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
show_query->database = context.resolveDatabase(show_query->database);
|
||||
context.checkAccess(AccessType::SHOW_DATABASES, show_query->database);
|
||||
create_query = DatabaseCatalog::instance().getDatabase(show_query->database)->getCreateDatabaseQuery(context);
|
||||
create_query = DatabaseCatalog::instance().getDatabase(show_query->database)->getCreateDatabaseQuery();
|
||||
}
|
||||
else if ((show_query = query_ptr->as<ASTShowCreateDictionaryQuery>()))
|
||||
{
|
||||
@ -66,7 +65,7 @@ BlockInputStreamPtr InterpreterShowCreateQuery::executeImpl()
|
||||
throw Exception("Temporary dictionaries are not possible.", ErrorCodes::SYNTAX_ERROR);
|
||||
show_query->database = context.resolveDatabase(show_query->database);
|
||||
context.checkAccess(AccessType::SHOW_DICTIONARIES, show_query->database, show_query->table);
|
||||
create_query = DatabaseCatalog::instance().getDatabase(show_query->database)->getCreateDictionaryQuery(context, show_query->table);
|
||||
create_query = DatabaseCatalog::instance().getDatabase(show_query->database)->getCreateDictionaryQuery(show_query->table);
|
||||
}
|
||||
|
||||
if (!create_query && show_query && show_query->temporary)
|
||||
|
@ -143,7 +143,7 @@ void InterpreterSystemQuery::startStopAction(StorageActionBlockType action_type,
|
||||
auto access = context.getAccess();
|
||||
for (auto & elem : DatabaseCatalog::instance().getDatabases())
|
||||
{
|
||||
for (auto iterator = elem.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = elem.second->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (!access->isGranted(log, getRequiredAccessType(action_type), elem.first, iterator->name()))
|
||||
continue;
|
||||
@ -330,7 +330,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica,
|
||||
{
|
||||
/// If table was already dropped by anyone, an exception will be thrown
|
||||
auto table_lock = table->lockExclusively(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
||||
create_ast = database->getCreateTableQuery(system_context, replica.table_name);
|
||||
create_ast = database->getCreateTableQuery(replica.table_name);
|
||||
|
||||
database->detachTable(replica.table_name);
|
||||
}
|
||||
@ -343,16 +343,17 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica,
|
||||
|
||||
auto columns = InterpreterCreateQuery::getColumnsDescription(*create.columns_list->columns, system_context);
|
||||
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints);
|
||||
auto data_path = database->getTableDataPath(create);
|
||||
|
||||
table = StorageFactory::instance().get(create,
|
||||
database->getTableDataPath(create),
|
||||
data_path,
|
||||
system_context,
|
||||
system_context.getGlobalContext(),
|
||||
columns,
|
||||
constraints,
|
||||
false);
|
||||
|
||||
database->createTable(system_context, replica.table_name, table, create_ast);
|
||||
database->attachTable(replica.table_name, table, data_path);
|
||||
|
||||
table->startup();
|
||||
return table;
|
||||
@ -366,7 +367,7 @@ void InterpreterSystemQuery::restartReplicas(Context & system_context)
|
||||
for (auto & elem : catalog.getDatabases())
|
||||
{
|
||||
DatabasePtr & database = elem.second;
|
||||
for (auto iterator = database->getTablesIterator(system_context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
replica_names.emplace_back(StorageID{database->getDatabaseName(), iterator->name()});
|
||||
|
@ -1,11 +1,6 @@
|
||||
#include <iomanip>
|
||||
#include <thread>
|
||||
#include <future>
|
||||
|
||||
#include <Common/ThreadPool.h>
|
||||
|
||||
#include <Poco/DirectoryIterator.h>
|
||||
#include <Poco/FileStream.h>
|
||||
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
@ -21,7 +16,6 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
|
||||
#include <Common/Stopwatch.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
@ -39,7 +33,6 @@ static void executeCreateQuery(
|
||||
ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + file_name, 0, context.getSettingsRef().max_parser_depth);
|
||||
|
||||
auto & ast_create_query = ast->as<ASTCreateQuery &>();
|
||||
ast_create_query.attach = true;
|
||||
ast_create_query.database = database;
|
||||
|
||||
InterpreterCreateQuery interpreter(ast, context);
|
||||
@ -55,19 +48,27 @@ static void loadDatabase(
|
||||
const String & database_path,
|
||||
bool force_restore_data)
|
||||
{
|
||||
/// There may exist .sql file with database creation statement.
|
||||
/// Or, if it is absent, then database with default engine is created.
|
||||
|
||||
String database_attach_query;
|
||||
String database_metadata_file = database_path + ".sql";
|
||||
|
||||
if (Poco::File(database_metadata_file).exists())
|
||||
{
|
||||
/// There is .sql file with database creation statement.
|
||||
ReadBufferFromFile in(database_metadata_file, 1024);
|
||||
readStringUntilEOF(database_attach_query, in);
|
||||
}
|
||||
else if (Poco::File(database_path).exists())
|
||||
{
|
||||
/// Database exists, but .sql file is absent. It's old-style Ordinary database (e.g. system or default)
|
||||
database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database) + " ENGINE = Ordinary";
|
||||
}
|
||||
else
|
||||
database_attach_query = "ATTACH DATABASE " + backQuoteIfNeed(database);
|
||||
{
|
||||
/// It's first server run and we need create default and system databases.
|
||||
/// .sql file with database engine will be written for CREATE query.
|
||||
database_attach_query = "CREATE DATABASE " + backQuoteIfNeed(database);
|
||||
}
|
||||
|
||||
executeCreateQuery(database_attach_query, context, database,
|
||||
database_metadata_file, force_restore_data);
|
||||
@ -77,7 +78,7 @@ static void loadDatabase(
|
||||
#define SYSTEM_DATABASE "system"
|
||||
|
||||
|
||||
void loadMetadata(Context & context)
|
||||
void loadMetadata(Context & context, const String & default_database_name)
|
||||
{
|
||||
String path = context.getPath() + "metadata";
|
||||
|
||||
@ -107,6 +108,13 @@ void loadMetadata(Context & context)
|
||||
databases.emplace(unescapeForFileName(it.name()), it.path().toString());
|
||||
}
|
||||
|
||||
/// clickhouse-local creates DatabaseMemory as default database by itself
|
||||
/// For clickhouse-server we need create default database
|
||||
bool create_default_db_if_not_exists = !default_database_name.empty();
|
||||
bool metadata_dir_for_default_db_already_exists = databases.count(default_database_name);
|
||||
if (create_default_db_if_not_exists && !metadata_dir_for_default_db_already_exists)
|
||||
databases.emplace(default_database_name, path + "/" + escapeForFileName(default_database_name));
|
||||
|
||||
for (const auto & [name, db_path] : databases)
|
||||
loadDatabase(context, name, db_path, has_force_restore_data_flag);
|
||||
|
||||
|
@ -11,6 +11,6 @@ class Context;
|
||||
void loadMetadataSystem(Context & context);
|
||||
|
||||
/// Load tables from databases and add them to context. Database 'system' is ignored. Use separate function to load system tables.
|
||||
void loadMetadata(Context & context);
|
||||
void loadMetadata(Context & context, const String & default_database_name = {});
|
||||
|
||||
}
|
||||
|
@ -102,8 +102,8 @@ int main()
|
||||
auto system_database = std::make_shared<DatabaseMemory>("system");
|
||||
DatabaseCatalog::instance().attachDatabase("system", system_database);
|
||||
//context.setCurrentDatabase("system");
|
||||
system_database->attachTable("one", StorageSystemOne::create("one"));
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false));
|
||||
system_database->attachTable("one", StorageSystemOne::create("one"), {});
|
||||
system_database->attachTable("numbers", StorageSystemNumbers::create(StorageID("system", "numbers"), false), {});
|
||||
|
||||
size_t success = 0;
|
||||
for (auto & entry : queries)
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -234,6 +235,10 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
<< (if_not_exists ? "IF NOT EXISTS " : "")
|
||||
<< (settings.hilite ? hilite_none : "")
|
||||
<< (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
|
||||
|
||||
if (uuid != UUIDHelpers::Nil)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " UUID " << (settings.hilite ? hilite_none : "")
|
||||
<< quoteString(toString(uuid));
|
||||
if (live_view_timeout)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH TIMEOUT " << (settings.hilite ? hilite_none : "")
|
||||
<< *live_view_timeout;
|
||||
@ -253,11 +258,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS " << (settings.hilite ? hilite_none : "");
|
||||
as_table_function->formatImpl(settings, state, frame);
|
||||
}
|
||||
if (!to_table.empty())
|
||||
if (to_table_id)
|
||||
{
|
||||
settings.ostr
|
||||
<< (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "")
|
||||
<< (!to_database.empty() ? backQuoteIfNeed(to_database) + "." : "") << backQuoteIfNeed(to_table);
|
||||
<< (!to_table_id.database_name.empty() ? backQuoteIfNeed(to_table_id.database_name) + "." : "")
|
||||
<< backQuoteIfNeed(to_table_id.table_name);
|
||||
}
|
||||
|
||||
if (!as_table.empty())
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Parsers/ASTDictionary.h>
|
||||
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
|
||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -64,8 +65,8 @@ public:
|
||||
ASTColumns * columns_list = nullptr;
|
||||
ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of dictionary
|
||||
ASTExpressionList * tables = nullptr;
|
||||
String to_database; /// For CREATE MATERIALIZED VIEW mv TO table.
|
||||
String to_table;
|
||||
//FIXME
|
||||
StorageID to_table_id = StorageID::createEmpty(); /// For CREATE MATERIALIZED VIEW mv TO table.
|
||||
ASTStorage * storage = nullptr;
|
||||
String as_database;
|
||||
String as_table;
|
||||
@ -73,6 +74,7 @@ public:
|
||||
ASTSelectWithUnionQuery * select = nullptr;
|
||||
ASTDictionary * dictionary = nullptr; /// dictionary definition (layout, primary key, etc.)
|
||||
std::optional<UInt64> live_view_timeout; /// For CREATE LIVE VIEW ... WITH TIMEOUT ...
|
||||
bool attach_short_syntax{false};
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char delim) const override { return (attach ? "AttachQuery" : "CreateQuery") + (delim + database) + delim + table; }
|
||||
|
@ -65,6 +65,9 @@ void ASTDropQuery::formatQueryImpl(const FormatSettings & settings, FormatState
|
||||
settings.ostr << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table);
|
||||
|
||||
formatOnCluster(settings);
|
||||
|
||||
if (no_delay)
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << " NO DELAY" << (settings.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -31,6 +31,8 @@ public:
|
||||
/// Same as above
|
||||
bool is_view{false};
|
||||
|
||||
bool no_delay{false};
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char) const override;
|
||||
ASTPtr clone() const override;
|
||||
|
@ -13,6 +13,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||
extern const int SYNTAX_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -163,4 +164,17 @@ void setIdentifierSpecial(ASTPtr & ast)
|
||||
id->semantic->special = true;
|
||||
}
|
||||
|
||||
StorageID getTableIdentifier(const ASTPtr & ast)
|
||||
{
|
||||
if (!ast)
|
||||
throw Exception("AST node is nullptr", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
const auto & identifier = dynamic_cast<const ASTIdentifier &>(*ast);
|
||||
if (identifier.name_parts.size() > 2)
|
||||
throw Exception("Logical error: more than two components in table expression", ErrorCodes::SYNTAX_ERROR);
|
||||
|
||||
if (identifier.name_parts.size() == 2)
|
||||
return { identifier.name_parts[0], identifier.name_parts[1], identifier.uuid };
|
||||
return { "", identifier.name, identifier.uuid };
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -68,6 +68,7 @@ private:
|
||||
friend struct IdentifierSemantic;
|
||||
friend ASTPtr createTableIdentifier(const StorageID & table_id);
|
||||
friend void setIdentifierSpecial(ASTPtr & ast);
|
||||
friend StorageID getTableIdentifier(const ASTPtr & ast);
|
||||
};
|
||||
|
||||
|
||||
@ -80,6 +81,7 @@ void setIdentifierSpecial(ASTPtr & ast);
|
||||
String getIdentifierName(const IAST * ast);
|
||||
std::optional<String> tryGetIdentifierName(const IAST * ast);
|
||||
bool tryGetIdentifierNameInto(const IAST * ast, String & name);
|
||||
StorageID getTableIdentifier(const ASTPtr & ast);
|
||||
|
||||
inline String getIdentifierName(const ASTPtr & ast) { return getIdentifierName(ast.get()); }
|
||||
inline std::optional<String> tryGetIdentifierName(const ASTPtr & ast) { return tryGetIdentifierName(ast.get()); }
|
||||
|
@ -26,8 +26,7 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s
|
||||
}
|
||||
else
|
||||
settings.ostr << (settings.hilite ? hilite_none : "")
|
||||
<< (!table_id.database_name.empty() ? backQuoteIfNeed(table_id.database_name) + "." : "") << backQuoteIfNeed(table_id.table_name)
|
||||
<< (table_id.hasUUID() ? " UUID " : "") << (table_id.hasUUID() ? quoteString(toString(table_id.uuid)) : "");
|
||||
<< (!table_id.database_name.empty() ? backQuoteIfNeed(table_id.database_name) + "." : "") << backQuoteIfNeed(table_id.table_name);
|
||||
|
||||
if (columns)
|
||||
{
|
||||
|
@ -29,6 +29,8 @@ public:
|
||||
using Elements = std::vector<Element>;
|
||||
Elements elements;
|
||||
|
||||
bool exchange{false}; /// For EXCHANGE TABLES
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char) const override { return "Rename"; }
|
||||
|
||||
@ -59,7 +61,8 @@ public:
|
||||
protected:
|
||||
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "RENAME TABLE " << (settings.hilite ? hilite_none : "");
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "")
|
||||
<< (exchange ? "EXCHANGE TABLES " : "RENAME TABLE ") << (settings.hilite ? hilite_none : "");
|
||||
|
||||
for (auto it = elements.cbegin(); it != elements.cend(); ++it)
|
||||
{
|
||||
@ -67,7 +70,7 @@ protected:
|
||||
settings.ostr << ", ";
|
||||
|
||||
settings.ostr << (!it->from.database.empty() ? backQuoteIfNeed(it->from.database) + "." : "") << backQuoteIfNeed(it->from.table)
|
||||
<< (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "")
|
||||
<< (settings.hilite ? hilite_keyword : "") << (exchange ? " AND " : " TO ") << (settings.hilite ? hilite_none : "")
|
||||
<< (!it->to.database.empty() ? backQuoteIfNeed(it->to.database) + "." : "") << backQuoteIfNeed(it->to.table);
|
||||
}
|
||||
|
||||
|
@ -33,6 +33,7 @@
|
||||
#include <boost/algorithm/string.hpp>
|
||||
#include "ASTColumnsMatcher.h"
|
||||
|
||||
#include <Interpreters/StorageID.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -190,9 +191,22 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex
|
||||
name += parts.back();
|
||||
}
|
||||
|
||||
ParserKeyword s_uuid("UUID");
|
||||
UUID uuid = UUIDHelpers::Nil;
|
||||
|
||||
if (table_name_with_optional_uuid && parts.size() <= 2 && s_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 (parts.size() == 1)
|
||||
parts.clear();
|
||||
node = std::make_shared<ASTIdentifier>(name, std::move(parts));
|
||||
node->as<ASTIdentifier>()->uuid = uuid;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
@ -55,13 +55,18 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/** An identifier, possibly containing a dot, for example, x_yz123 or `something special` or Hits.EventTime
|
||||
/** An identifier, possibly containing a dot, for example, x_yz123 or `something special` or Hits.EventTime,
|
||||
* possibly with UUID clause like `db name`.`table name` UUID 'xxxxxxxx-xxxx-xxxx-xxxx-xxxxxxxxxxxx'
|
||||
*/
|
||||
class ParserCompoundIdentifier : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false)
|
||||
: table_name_with_optional_uuid(table_name_with_optional_uuid_) {}
|
||||
protected:
|
||||
const char * getName() const override { return "compound identifier"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
bool table_name_with_optional_uuid;
|
||||
};
|
||||
|
||||
/// Just *
|
||||
|
@ -334,10 +334,12 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_temporary("TEMPORARY");
|
||||
ParserKeyword s_attach("ATTACH");
|
||||
ParserKeyword s_temporary("TEMPORARY");
|
||||
ParserKeyword s_table("TABLE");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserCompoundIdentifier table_name_p(true);
|
||||
ParserKeyword s_on("ON");
|
||||
ParserKeyword s_as("AS");
|
||||
ParserToken s_dot(TokenType::Dot);
|
||||
ParserToken s_lparen(TokenType::OpeningRoundBracket);
|
||||
@ -349,11 +351,8 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
ParserFunction table_function_p;
|
||||
ParserNameList names_p;
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
ASTPtr columns_list;
|
||||
ASTPtr to_database;
|
||||
ASTPtr to_table;
|
||||
ASTPtr storage;
|
||||
ASTPtr as_database;
|
||||
ASTPtr as_table;
|
||||
@ -383,22 +382,17 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
if_not_exists = true;
|
||||
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
if (!table_name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
database = table;
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserKeyword{"ON"}.ignore(pos, expected))
|
||||
if (s_on.ignore(pos, expected))
|
||||
{
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
StorageID table_id = getTableIdentifier(table);
|
||||
|
||||
// Shortcut for ATTACH a previously detached table
|
||||
if (attach && (!pos.isValid() || pos.get().type == TokenType::Semicolon))
|
||||
{
|
||||
@ -409,8 +403,9 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -467,13 +462,11 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->temporary = is_temporary;
|
||||
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
tryGetIdentifierNameInto(to_database, query->to_database);
|
||||
tryGetIdentifierNameInto(to_table, query->to_table);
|
||||
|
||||
query->set(query->columns_list, columns_list);
|
||||
query->set(query->storage, storage);
|
||||
|
||||
@ -489,6 +482,7 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_attach("ATTACH");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserCompoundIdentifier table_name_p(true);
|
||||
ParserKeyword s_as("AS");
|
||||
ParserKeyword s_view("VIEW");
|
||||
ParserKeyword s_live("LIVE");
|
||||
@ -500,11 +494,9 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
ParserTablePropertiesDeclarationList table_properties_p;
|
||||
ParserSelectWithUnionQuery select_p;
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
ASTPtr columns_list;
|
||||
ASTPtr to_database;
|
||||
ASTPtr to_table;
|
||||
ASTPtr columns_list;
|
||||
ASTPtr storage;
|
||||
ASTPtr as_database;
|
||||
ASTPtr as_table;
|
||||
@ -532,16 +524,9 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
if_not_exists = true;
|
||||
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
if (!table_name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
database = table;
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserKeyword{"WITH TIMEOUT"}.ignore(pos, expected))
|
||||
{
|
||||
if (!ParserNumber{}.parse(pos, live_view_timeout, expected))
|
||||
@ -557,15 +542,8 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
// TO [db.]table
|
||||
if (ParserKeyword{"TO"}.ignore(pos, expected))
|
||||
{
|
||||
if (!name_p.parse(pos, to_table, expected))
|
||||
if (!table_name_p.parse(pos, to_table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
to_database = to_table;
|
||||
if (!name_p.parse(pos, to_table, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/// Optional - a list of columns can be specified. It must fully comply with SELECT.
|
||||
@ -593,12 +571,14 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->is_live_view = true;
|
||||
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
StorageID table_id = getTableIdentifier(table);
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
tryGetIdentifierNameInto(to_database, query->to_database);
|
||||
tryGetIdentifierNameInto(to_table, query->to_table);
|
||||
if (to_table)
|
||||
query->to_table_id = getTableIdentifier(to_table);
|
||||
|
||||
query->set(query->columns_list, columns_list);
|
||||
|
||||
@ -673,6 +653,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
ParserKeyword s_create("CREATE");
|
||||
ParserKeyword s_attach("ATTACH");
|
||||
ParserKeyword s_if_not_exists("IF NOT EXISTS");
|
||||
ParserCompoundIdentifier table_name_p(true);
|
||||
ParserKeyword s_as("AS");
|
||||
ParserKeyword s_view("VIEW");
|
||||
ParserKeyword s_materialized("MATERIALIZED");
|
||||
@ -687,11 +668,9 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
ParserSelectWithUnionQuery select_p;
|
||||
ParserNameList names_p;
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
ASTPtr columns_list;
|
||||
ASTPtr to_database;
|
||||
ASTPtr to_table;
|
||||
ASTPtr columns_list;
|
||||
ASTPtr storage;
|
||||
ASTPtr as_database;
|
||||
ASTPtr as_table;
|
||||
@ -732,16 +711,9 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
if (!replace_view && s_if_not_exists.ignore(pos, expected))
|
||||
if_not_exists = true;
|
||||
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
if (!table_name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
database = table;
|
||||
if (!name_p.parse(pos, table, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (ParserKeyword{"ON"}.ignore(pos, expected))
|
||||
{
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
@ -751,15 +723,8 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
// TO [db.]table
|
||||
if (ParserKeyword{"TO"}.ignore(pos, expected))
|
||||
{
|
||||
if (!name_p.parse(pos, to_table, expected))
|
||||
if (!table_name_p.parse(pos, to_table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
to_database = to_table;
|
||||
if (!name_p.parse(pos, to_table, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/// Optional - a list of columns can be specified. It must fully comply with SELECT.
|
||||
@ -800,12 +765,14 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
||||
query->is_populate = is_populate;
|
||||
query->replace_view = replace_view;
|
||||
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
StorageID table_id = getTableIdentifier(table);
|
||||
query->database = table_id.database_name;
|
||||
query->table = table_id.table_name;
|
||||
query->uuid = table_id.uuid;
|
||||
query->cluster = cluster_str;
|
||||
|
||||
tryGetIdentifierNameInto(to_database, query->to_database);
|
||||
tryGetIdentifierNameInto(to_table, query->to_table);
|
||||
if (to_table)
|
||||
query->to_table_id = getTableIdentifier(to_table);
|
||||
|
||||
query->set(query->columns_list, columns_list);
|
||||
query->set(query->storage, storage);
|
||||
@ -892,10 +859,8 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E
|
||||
query->is_dictionary = true;
|
||||
query->attach = attach;
|
||||
|
||||
if (database)
|
||||
query->database = typeid_cast<const ASTIdentifier &>(*database).name;
|
||||
|
||||
query->table = typeid_cast<const ASTIdentifier &>(*name).name;
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(name, query->table);
|
||||
|
||||
query->if_not_exists = if_not_exists;
|
||||
query->set(query->dictionary_attributes_list, attributes);
|
||||
|
@ -289,7 +289,7 @@ protected:
|
||||
};
|
||||
|
||||
/** Query like this:
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster]
|
||||
* (
|
||||
* name1 type1,
|
||||
* name2 type2,
|
||||
@ -299,10 +299,10 @@ protected:
|
||||
* ) ENGINE = engine
|
||||
*
|
||||
* Or:
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2 [ENGINE = engine]
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] AS [db2.]name2 [ENGINE = engine]
|
||||
*
|
||||
* Or:
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name AS ENGINE = engine SELECT ...
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] AS ENGINE = engine SELECT ...
|
||||
*
|
||||
*/
|
||||
class ParserCreateTableQuery : public IParserBase
|
||||
@ -312,7 +312,7 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/// CREATE|ATTACH LIVE VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] AS SELECT ...
|
||||
/// CREATE|ATTACH LIVE VIEW [IF NOT EXISTS] [db.]name [UUID 'uuid'] [TO [db.]name] AS SELECT ...
|
||||
class ParserCreateLiveViewQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
@ -328,7 +328,7 @@ protected:
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
||||
/// CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
|
||||
/// CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [VIEW]] [IF NOT EXISTS] [db.]name [UUID 'uuid'] [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ...
|
||||
class ParserCreateViewQuery : public IParserBase
|
||||
{
|
||||
protected:
|
||||
|
@ -21,6 +21,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserToken s_dot(TokenType::Dot);
|
||||
ParserKeyword s_if_exists("IF EXISTS");
|
||||
ParserIdentifier name_p;
|
||||
ParserKeyword s_no_delay("NO DELAY");
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
@ -29,6 +30,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
|
||||
bool temporary = false;
|
||||
bool is_dictionary = false;
|
||||
bool is_view = false;
|
||||
bool no_delay = false;
|
||||
|
||||
if (s_database.ignore(pos, expected))
|
||||
{
|
||||
@ -76,6 +78,9 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
if (s_no_delay.ignore(pos, expected))
|
||||
no_delay = true;
|
||||
}
|
||||
|
||||
auto query = std::make_shared<ASTDropQuery>();
|
||||
@ -86,6 +91,7 @@ bool parseDropQuery(IParser::Pos & pos, ASTPtr & node, Expected & expected)
|
||||
query->temporary = temporary;
|
||||
query->is_dictionary = is_dictionary;
|
||||
query->is_view = is_view;
|
||||
query->no_delay = no_delay;
|
||||
|
||||
tryGetIdentifierNameInto(database, query->database);
|
||||
tryGetIdentifierNameInto(table, query->table);
|
||||
|
@ -41,13 +41,27 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
{
|
||||
ParserKeyword s_rename_table("RENAME TABLE");
|
||||
ParserKeyword s_to("TO");
|
||||
ParserKeyword s_exchange_tables("EXCHANGE TABLES");
|
||||
ParserKeyword s_and("AND");
|
||||
ParserToken s_comma(TokenType::Comma);
|
||||
|
||||
bool exchange = false;
|
||||
|
||||
if (!s_rename_table.ignore(pos, expected))
|
||||
return false;
|
||||
{
|
||||
if (s_exchange_tables.ignore(pos, expected))
|
||||
exchange = true;
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
ASTRenameQuery::Elements elements;
|
||||
|
||||
auto ignore_delim = [&]()
|
||||
{
|
||||
return exchange ? s_and.ignore(pos) : s_to.ignore(pos);
|
||||
};
|
||||
|
||||
while (true)
|
||||
{
|
||||
if (!elements.empty() && !s_comma.ignore(pos))
|
||||
@ -56,7 +70,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
elements.push_back(ASTRenameQuery::Element());
|
||||
|
||||
if (!parseDatabaseAndTable(elements.back().from, pos, expected)
|
||||
|| !s_to.ignore(pos)
|
||||
|| !ignore_delim()
|
||||
|| !parseDatabaseAndTable(elements.back().to, pos, expected))
|
||||
return false;
|
||||
}
|
||||
@ -73,6 +87,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
node = query;
|
||||
|
||||
query->elements = elements;
|
||||
query->exchange = exchange;
|
||||
return true;
|
||||
}
|
||||
|
||||
|
@ -393,7 +393,7 @@ void IStorage::alter(
|
||||
auto table_id = getStorageID();
|
||||
StorageInMemoryMetadata metadata = getInMemoryMetadata();
|
||||
params.apply(metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
|
||||
setColumns(std::move(metadata.columns));
|
||||
}
|
||||
|
||||
@ -431,15 +431,14 @@ BlockInputStreams IStorage::readStreams(
|
||||
|
||||
StorageID IStorage::getStorageID() const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(id_mutex);
|
||||
std::lock_guard lock(id_mutex);
|
||||
return storage_id;
|
||||
}
|
||||
|
||||
void IStorage::renameInMemory(const String & new_database_name, const String & new_table_name)
|
||||
void IStorage::renameInMemory(const StorageID & new_table_id)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(id_mutex);
|
||||
storage_id.database_name = new_database_name;
|
||||
storage_id.table_name = new_table_name;
|
||||
std::lock_guard lock(id_mutex);
|
||||
storage_id = new_table_id;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -317,9 +317,10 @@ public:
|
||||
/** Delete the table data. Called before deleting the directory with the data.
|
||||
* The method can be called only after detaching table from Context (when no queries are performed with table).
|
||||
* The table is not usable during and after call to this method.
|
||||
* If some queries may still use the table, then it must be called under exclusive lock.
|
||||
* If you do not need any action other than deleting the directory with data, you can leave this method blank.
|
||||
*/
|
||||
virtual void drop(TableStructureWriteLockHolder &) {}
|
||||
virtual void drop() {}
|
||||
|
||||
/** Clear the table data and leave it empty.
|
||||
* Must be called under lockForAlter.
|
||||
@ -333,18 +334,18 @@ public:
|
||||
* Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately.
|
||||
* In this function, you need to rename the directory with the data, if any.
|
||||
* Called when the table structure is locked for write.
|
||||
* Table UUID must remain unchanged, unless table moved between Ordinary and Atomic databases.
|
||||
*/
|
||||
virtual void rename(const String & /*new_path_to_table_data*/, const String & new_database_name, const String & new_table_name,
|
||||
TableStructureWriteLockHolder &)
|
||||
virtual void rename(const String & /*new_path_to_table_data*/, const StorageID & new_table_id)
|
||||
{
|
||||
renameInMemory(new_database_name, new_table_name);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
/**
|
||||
* Just updates names of database and table without moving any data on disk
|
||||
* Can be called directly only from DatabaseAtomic.
|
||||
*/
|
||||
virtual void renameInMemory(const String & new_database_name, const String & new_table_name);
|
||||
virtual void renameInMemory(const StorageID & new_table_id);
|
||||
|
||||
/** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters.
|
||||
* This method must fully execute the ALTER query, taking care of the locks itself.
|
||||
|
@ -507,7 +507,7 @@ StorageLiveView::~StorageLiveView()
|
||||
}
|
||||
}
|
||||
|
||||
void StorageLiveView::drop(TableStructureWriteLockHolder &)
|
||||
void StorageLiveView::drop()
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
DatabaseCatalog::instance().removeDependency(select_table_id, table_id);
|
||||
|
@ -119,7 +119,7 @@ public:
|
||||
}
|
||||
|
||||
void checkTableCanBeDropped() const override;
|
||||
void drop(TableStructureWriteLockHolder &) override;
|
||||
void drop() override;
|
||||
void startup() override;
|
||||
void shutdown() override;
|
||||
|
||||
|
@ -1294,9 +1294,7 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeData::rename(
|
||||
const String & new_table_path, const String & new_database_name,
|
||||
const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void MergeTreeData::rename(const String & new_table_path, const StorageID & new_table_id)
|
||||
{
|
||||
auto disks = getStoragePolicy()->getDisks();
|
||||
|
||||
@ -1309,14 +1307,14 @@ void MergeTreeData::rename(
|
||||
for (const auto & disk : disks)
|
||||
{
|
||||
auto new_table_path_parent = parentPath(new_table_path);
|
||||
disk->createDirectory(new_table_path_parent);
|
||||
disk->createDirectories(new_table_path_parent);
|
||||
disk->moveDirectory(relative_data_path, new_table_path);
|
||||
}
|
||||
|
||||
global_context.dropCaches();
|
||||
|
||||
relative_data_path = new_table_path;
|
||||
renameInMemory(new_database_name, new_table_name);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
void MergeTreeData::dropAllData()
|
||||
|
@ -521,8 +521,7 @@ public:
|
||||
/// Moves the entire data directory.
|
||||
/// Flushes the uncompressed blocks cache and the marks cache.
|
||||
/// Must be called with locked lockStructureForAlter().
|
||||
void rename(const String & new_table_path, const String & new_database_name,
|
||||
const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_table_path, const StorageID & new_table_id) override;
|
||||
|
||||
/// Check if the ALTER can be performed:
|
||||
/// - all needed columns are present.
|
||||
|
@ -786,7 +786,7 @@ void StorageBuffer::alter(const AlterCommands & params, const Context & context,
|
||||
|
||||
StorageInMemoryMetadata metadata = getInMemoryMetadata();
|
||||
params.apply(metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
|
||||
setColumns(std::move(metadata.columns));
|
||||
}
|
||||
|
||||
|
@ -542,7 +542,7 @@ void StorageDistributed::alter(const AlterCommands & params, const Context & con
|
||||
checkAlterIsPossible(params, context.getSettingsRef());
|
||||
StorageInMemoryMetadata metadata = getInMemoryMetadata();
|
||||
params.apply(metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
|
||||
setColumns(std::move(metadata.columns));
|
||||
}
|
||||
|
||||
@ -798,12 +798,11 @@ void StorageDistributed::flushClusterNodesAllData()
|
||||
node.second.flushAllData();
|
||||
}
|
||||
|
||||
void StorageDistributed::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name,
|
||||
TableStructureWriteLockHolder &)
|
||||
void StorageDistributed::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||
{
|
||||
if (!relative_data_path.empty())
|
||||
renameOnDisk(new_path_to_table_data);
|
||||
renameInMemory(new_database_name, new_table_name);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
void StorageDistributed::renameOnDisk(const String & new_path_to_table_data)
|
||||
{
|
||||
|
@ -81,12 +81,10 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
void drop(TableStructureWriteLockHolder &) override {}
|
||||
|
||||
/// Removes temporary data in local filesystem.
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
void renameOnDisk(const String & new_path_to_table_data);
|
||||
|
||||
void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) override;
|
||||
|
@ -126,7 +126,6 @@ void checkCreationIsAllowed(const Context & context_global, const std::string &
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
StorageFile::StorageFile(int table_fd_, CommonArguments args)
|
||||
: StorageFile(args)
|
||||
{
|
||||
@ -498,7 +497,7 @@ Strings StorageFile::getDataPaths() const
|
||||
return paths;
|
||||
}
|
||||
|
||||
void StorageFile::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageFile::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||
{
|
||||
if (!is_db_table)
|
||||
throw Exception("Can't rename table " + getStorageID().getNameForLogs() + " binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||
@ -513,7 +512,7 @@ void StorageFile::rename(const String & new_path_to_table_data, const String & n
|
||||
Poco::File(paths[0]).renameTo(path_new);
|
||||
|
||||
paths[0] = std::move(path_new);
|
||||
renameInMemory(new_database_name, new_table_name);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
void StorageFile::truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &)
|
||||
|
@ -38,7 +38,7 @@ public:
|
||||
|
||||
void truncate(const ASTPtr & /*query*/, const Context & /* context */, TableStructureWriteLockHolder &) override;
|
||||
|
||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
Strings getDataPaths() const override;
|
||||
|
||||
|
@ -514,7 +514,7 @@ void StorageLog::loadMarks()
|
||||
}
|
||||
|
||||
|
||||
void StorageLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
@ -527,7 +527,7 @@ void StorageLog::rename(const String & new_path_to_table_data, const String & ne
|
||||
file.second.data_file_path = table_path + fileName(file.second.data_file_path);
|
||||
|
||||
marks_file_path = table_path + DBMS_STORAGE_LOG_MARKS_FILE_NAME;
|
||||
renameInMemory(new_database_name, new_table_name);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
|
||||
|
@ -34,11 +34,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
void rename(
|
||||
const String & new_path_to_table_data,
|
||||
const String & new_database_name,
|
||||
const String & new_table_name,
|
||||
TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||
|
||||
|
@ -34,9 +34,11 @@ namespace ErrorCodes
|
||||
extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW;
|
||||
}
|
||||
|
||||
static inline String generateInnerTableName(const String & table_name)
|
||||
static inline String generateInnerTableName(const StorageID & view_id)
|
||||
{
|
||||
return ".inner." + table_name;
|
||||
if (view_id.hasUUID())
|
||||
return ".inner_id." + toString(view_id.uuid);
|
||||
return ".inner." + view_id.getTableName();
|
||||
}
|
||||
|
||||
static StorageID extractDependentTableFromSelectQuery(ASTSelectQuery & query, const Context & context, bool add_default_db = true)
|
||||
@ -106,7 +108,7 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
/// If the destination table is not set, use inner table
|
||||
has_inner_table = query.to_table.empty();
|
||||
has_inner_table = query.to_table_id.empty();
|
||||
if (has_inner_table && !query.storage)
|
||||
throw Exception(
|
||||
"You must specify where to save results of a MaterializedView query: either ENGINE or an existing table in a TO clause",
|
||||
@ -124,18 +126,18 @@ StorageMaterializedView::StorageMaterializedView(
|
||||
select_table_id = extractDependentTableFromSelectQuery(select_query, local_context);
|
||||
|
||||
if (!has_inner_table)
|
||||
target_table_id = StorageID(query.to_database, query.to_table);
|
||||
target_table_id = query.to_table_id;
|
||||
else if (attach_)
|
||||
{
|
||||
/// If there is an ATTACH request, then the internal table must already be created.
|
||||
target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID().table_name));
|
||||
target_table_id = StorageID(getStorageID().database_name, generateInnerTableName(getStorageID()));
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We will create a query to create an internal table.
|
||||
auto manual_create_query = std::make_shared<ASTCreateQuery>();
|
||||
manual_create_query->database = getStorageID().database_name;
|
||||
manual_create_query->table = generateInnerTableName(getStorageID().table_name);
|
||||
manual_create_query->table = generateInnerTableName(getStorageID());
|
||||
|
||||
auto new_columns_list = std::make_shared<ASTColumns>();
|
||||
new_columns_list->set(new_columns_list->columns, query.columns_list->columns->ptr());
|
||||
@ -219,6 +221,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context,
|
||||
drop_query->database = target_table_id.database_name;
|
||||
drop_query->table = target_table_id.table_name;
|
||||
drop_query->kind = kind;
|
||||
drop_query->no_delay = true;
|
||||
ASTPtr ast_drop_query = drop_query;
|
||||
InterpreterDropQuery drop_interpreter(ast_drop_query, global_context);
|
||||
drop_interpreter.execute();
|
||||
@ -226,7 +229,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context,
|
||||
}
|
||||
|
||||
|
||||
void StorageMaterializedView::drop(TableStructureWriteLockHolder &)
|
||||
void StorageMaterializedView::drop()
|
||||
{
|
||||
auto table_id = getStorageID();
|
||||
if (!select_table_id.empty())
|
||||
@ -287,7 +290,7 @@ void StorageMaterializedView::alter(
|
||||
}
|
||||
/// end modify query
|
||||
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
|
||||
setColumns(std::move(metadata.columns));
|
||||
}
|
||||
|
||||
@ -328,12 +331,14 @@ void StorageMaterializedView::mutate(const MutationCommands & commands, const Co
|
||||
getTargetTable()->mutate(commands, context);
|
||||
}
|
||||
|
||||
void StorageMaterializedView::rename(
|
||||
const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageMaterializedView::renameInMemory(const StorageID & new_table_id)
|
||||
{
|
||||
if (has_inner_table && tryGetTargetTable())
|
||||
auto old_table_id = getStorageID();
|
||||
bool from_atomic_to_atomic_database = old_table_id.hasUUID() && new_table_id.hasUUID();
|
||||
|
||||
if (has_inner_table && tryGetTargetTable() && !from_atomic_to_atomic_database)
|
||||
{
|
||||
auto new_target_table_name = generateInnerTableName(new_table_name);
|
||||
auto new_target_table_name = generateInnerTableName(new_table_id);
|
||||
auto rename = std::make_shared<ASTRenameQuery>();
|
||||
|
||||
ASTRenameQuery::Table from;
|
||||
@ -353,8 +358,8 @@ void StorageMaterializedView::rename(
|
||||
target_table_id.table_name = new_target_table_name;
|
||||
}
|
||||
|
||||
auto old_table_id = getStorageID();
|
||||
IStorage::renameInMemory(new_database_name, new_table_name);
|
||||
IStorage::renameInMemory(new_table_id);
|
||||
// TODO Actually we don't need to update dependency if MV has UUID, but then db and table name will be outdated
|
||||
DatabaseCatalog::instance().updateDependency(select_table_id, old_table_id, select_table_id, getStorageID());
|
||||
}
|
||||
|
||||
|
@ -21,6 +21,7 @@ public:
|
||||
|
||||
ASTPtr getSelectQuery() const { return select->clone(); }
|
||||
ASTPtr getInnerQuery() const { return inner_query->clone(); }
|
||||
bool hasInnerTable() const { return has_inner_table; }
|
||||
|
||||
NameAndTypePair getColumn(const String & column_name) const override;
|
||||
bool hasColumn(const String & column_name) const override;
|
||||
@ -39,7 +40,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
void drop(TableStructureWriteLockHolder &) override;
|
||||
void drop() override;
|
||||
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
@ -53,7 +54,7 @@ public:
|
||||
|
||||
void mutate(const MutationCommands & commands, const Context & context) override;
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void renameInMemory(const StorageID & new_table_id) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
|
@ -124,7 +124,7 @@ BlockOutputStreamPtr StorageMemory::write(
|
||||
}
|
||||
|
||||
|
||||
void StorageMemory::drop(TableStructureWriteLockHolder &)
|
||||
void StorageMemory::drop()
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
data.clear();
|
||||
|
@ -38,7 +38,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
void drop(TableStructureWriteLockHolder &) override;
|
||||
void drop() override;
|
||||
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
|
@ -419,7 +419,7 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator() const
|
||||
checkStackSize();
|
||||
auto database = DatabaseCatalog::instance().getDatabase(source_database);
|
||||
auto table_name_match = [this](const String & table_name_) { return table_name_regexp.match(table_name_); };
|
||||
return database->getTablesIterator(global_context, table_name_match);
|
||||
return database->getTablesIterator(table_name_match);
|
||||
}
|
||||
|
||||
|
||||
@ -443,7 +443,7 @@ void StorageMerge::alter(
|
||||
|
||||
StorageInMemoryMetadata storage_metadata = getInMemoryMetadata();
|
||||
params.apply(storage_metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, storage_metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, storage_metadata);
|
||||
setColumns(storage_metadata.columns);
|
||||
}
|
||||
|
||||
|
@ -183,7 +183,7 @@ void StorageMergeTree::checkPartitionCanBeDropped(const ASTPtr & partition)
|
||||
global_context.checkPartitionCanBeDropped(table_id.database_name, table_id.table_name, partition_size);
|
||||
}
|
||||
|
||||
void StorageMergeTree::drop(TableStructureWriteLockHolder &)
|
||||
void StorageMergeTree::drop()
|
||||
{
|
||||
shutdown();
|
||||
dropAllData();
|
||||
@ -227,7 +227,7 @@ void StorageMergeTree::alter(
|
||||
|
||||
changeSettings(metadata.settings_ast, table_lock_holder);
|
||||
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -239,7 +239,7 @@ void StorageMergeTree::alter(
|
||||
|
||||
setTTLExpressions(metadata.columns, metadata.ttl_for_table_ast);
|
||||
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
|
||||
|
||||
String mutation_file_name;
|
||||
Int64 mutation_version = -1;
|
||||
|
@ -63,7 +63,7 @@ public:
|
||||
|
||||
CancellationCode killMutation(const String & mutation_id) override;
|
||||
|
||||
void drop(TableStructureWriteLockHolder &) override;
|
||||
void drop() override;
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
void alter(const AlterCommands & commands, const Context & context, TableStructureWriteLockHolder & table_lock_holder) override;
|
||||
|
@ -53,7 +53,7 @@ void StorageNull::alter(
|
||||
|
||||
StorageInMemoryMetadata metadata = getInMemoryMetadata();
|
||||
params.apply(metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, metadata);
|
||||
setColumns(std::move(metadata.columns));
|
||||
}
|
||||
|
||||
|
@ -513,7 +513,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
}
|
||||
|
||||
auto table_id = getStorageID();
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(global_context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(global_context, table_id, metadata);
|
||||
|
||||
/// Even if the primary/sorting keys didn't change we must reinitialize it
|
||||
/// because primary key column types might have changed.
|
||||
@ -2953,11 +2953,15 @@ void StorageReplicatedMergeTree::startup()
|
||||
move_parts_task_handle = pool.createTask([this] { return movePartsTask(); });
|
||||
pool.startTask(move_parts_task_handle);
|
||||
}
|
||||
need_shutdown.store(true);
|
||||
}
|
||||
|
||||
|
||||
void StorageReplicatedMergeTree::shutdown()
|
||||
{
|
||||
if (!need_shutdown.load())
|
||||
return;
|
||||
|
||||
clearOldPartsFromFilesystem(true);
|
||||
/// Cancel fetches, merges and mutations to force the queue_task to finish ASAP.
|
||||
fetcher.blocker.cancelForever();
|
||||
@ -2991,6 +2995,7 @@ void StorageReplicatedMergeTree::shutdown()
|
||||
std::unique_lock lock(data_parts_exchange_endpoint->rwlock);
|
||||
}
|
||||
data_parts_exchange_endpoint.reset();
|
||||
need_shutdown.store(false);
|
||||
}
|
||||
|
||||
|
||||
@ -3290,7 +3295,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
|
||||
changeSettings(metadata.settings_ast, table_lock_holder);
|
||||
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, metadata);
|
||||
return;
|
||||
}
|
||||
|
||||
@ -3352,7 +3357,7 @@ void StorageReplicatedMergeTree::alter(
|
||||
/// Just change settings
|
||||
current_metadata.settings_ast = future_metadata.settings_ast;
|
||||
changeSettings(current_metadata.settings_ast, table_lock_holder);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id.table_name, current_metadata);
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, current_metadata);
|
||||
}
|
||||
|
||||
/// We can be sure, that in case of successfull commit in zookeeper our
|
||||
@ -3687,7 +3692,7 @@ void StorageReplicatedMergeTree::checkPartitionCanBeDropped(const ASTPtr & parti
|
||||
}
|
||||
|
||||
|
||||
void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &)
|
||||
void StorageReplicatedMergeTree::drop()
|
||||
{
|
||||
{
|
||||
auto zookeeper = tryGetZooKeeper();
|
||||
@ -3724,11 +3729,9 @@ void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &)
|
||||
}
|
||||
|
||||
|
||||
void StorageReplicatedMergeTree::rename(
|
||||
const String & new_path_to_table_data, const String & new_database_name,
|
||||
const String & new_table_name, TableStructureWriteLockHolder & lock)
|
||||
void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||
{
|
||||
MergeTreeData::rename(new_path_to_table_data, new_database_name, new_table_name, lock);
|
||||
MergeTreeData::rename(new_path_to_table_data, new_table_id);
|
||||
|
||||
/// Update table name in zookeeper
|
||||
auto zookeeper = getZooKeeper();
|
||||
|
@ -113,11 +113,11 @@ public:
|
||||
|
||||
/** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper.
|
||||
*/
|
||||
void drop(TableStructureWriteLockHolder &) override;
|
||||
void drop() override;
|
||||
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
bool supportsIndexForIn() const override { return true; }
|
||||
|
||||
@ -288,6 +288,8 @@ private:
|
||||
/// True if replica was created for existing table with fixed granularity
|
||||
bool other_replicas_fixed_granularity = false;
|
||||
|
||||
std::atomic_bool need_shutdown{false};
|
||||
|
||||
template <class Func>
|
||||
void foreachCommittedParts(const Func & func) const;
|
||||
|
||||
|
@ -197,15 +197,14 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path)
|
||||
}
|
||||
|
||||
|
||||
void StorageSetOrJoinBase::rename(
|
||||
const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageSetOrJoinBase::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||
{
|
||||
/// Rename directory with data.
|
||||
String new_path = base_path + new_path_to_table_data;
|
||||
Poco::File(path).renameTo(new_path);
|
||||
|
||||
path = new_path;
|
||||
renameInMemory(new_database_name, new_table_name);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
|
||||
|
@ -19,7 +19,7 @@ class StorageSetOrJoinBase : public IStorage
|
||||
friend class SetOrJoinBlockOutputStream;
|
||||
|
||||
public:
|
||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
|
@ -237,7 +237,7 @@ StorageStripeLog::StorageStripeLog(
|
||||
}
|
||||
|
||||
|
||||
void StorageStripeLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageStripeLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
@ -245,7 +245,7 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Strin
|
||||
|
||||
table_path = new_path_to_table_data;
|
||||
file_checker.setPath(table_path + "sizes.json");
|
||||
renameInMemory(new_database_name, new_table_name);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
|
||||
|
@ -35,11 +35,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
void rename(
|
||||
const String & new_path_to_table_data,
|
||||
const String & new_database_name,
|
||||
const String & new_table_name,
|
||||
TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||
|
||||
|
@ -375,7 +375,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
|
||||
}
|
||||
|
||||
|
||||
void StorageTinyLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageTinyLog::rename(const String & new_path_to_table_data, const StorageID & new_table_id)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
@ -386,7 +386,7 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const String
|
||||
|
||||
for (auto & file : files)
|
||||
file.second.data_file_path = table_path + fileName(file.second.data_file_path);
|
||||
renameInMemory(new_database_name, new_table_name);
|
||||
renameInMemory(new_table_id);
|
||||
}
|
||||
|
||||
|
||||
@ -437,13 +437,15 @@ void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWri
|
||||
addFiles(column.name, *column.type);
|
||||
}
|
||||
|
||||
void StorageTinyLog::drop(TableStructureWriteLockHolder &)
|
||||
void StorageTinyLog::drop()
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
disk->removeRecursive(table_path);
|
||||
if (disk->exists(table_path))
|
||||
disk->removeRecursive(table_path);
|
||||
files.clear();
|
||||
}
|
||||
|
||||
|
||||
void registerStorageTinyLog(StorageFactory & factory)
|
||||
{
|
||||
StorageFactory::StorageFeatures features{
|
||||
|
@ -34,11 +34,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
void rename(
|
||||
const String & new_path_to_table_data,
|
||||
const String & new_database_name,
|
||||
const String & new_table_name,
|
||||
TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||
|
||||
@ -46,7 +42,7 @@ public:
|
||||
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
void drop(TableStructureWriteLockHolder &) override;
|
||||
void drop() override;
|
||||
|
||||
protected:
|
||||
StorageTinyLog(
|
||||
|
@ -303,7 +303,7 @@ Pipes StorageSystemColumns::read(
|
||||
const DatabasePtr database = databases.at(database_name);
|
||||
offsets[i] = i ? offsets[i - 1] : 0;
|
||||
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
const String & table_name = iterator->name();
|
||||
storages.emplace(std::piecewise_construct,
|
||||
|
@ -25,7 +25,7 @@ NamesAndTypesList StorageSystemGraphite::getNamesAndTypes()
|
||||
/*
|
||||
* Looking for (Replicated)*GraphiteMergeTree and get all configuration parameters for them
|
||||
*/
|
||||
static StorageSystemGraphite::Configs getConfigs(const Context & context)
|
||||
static StorageSystemGraphite::Configs getConfigs()
|
||||
{
|
||||
const Databases databases = DatabaseCatalog::instance().getDatabases();
|
||||
StorageSystemGraphite::Configs graphite_configs;
|
||||
@ -36,7 +36,7 @@ static StorageSystemGraphite::Configs getConfigs(const Context & context)
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
const auto & table = iterator->table();
|
||||
|
||||
@ -71,9 +71,9 @@ static StorageSystemGraphite::Configs getConfigs(const Context & context)
|
||||
return graphite_configs;
|
||||
}
|
||||
|
||||
void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const
|
||||
void StorageSystemGraphite::fillData(MutableColumns & res_columns, const Context &, const SelectQueryInfo &) const
|
||||
{
|
||||
Configs graphite_configs = getConfigs(context);
|
||||
Configs graphite_configs = getConfigs();
|
||||
|
||||
for (const auto & config : graphite_configs)
|
||||
{
|
||||
|
@ -50,7 +50,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex
|
||||
|
||||
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
|
||||
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (!dynamic_cast<const MergeTreeData *>(iterator->table().get()))
|
||||
continue;
|
||||
|
@ -110,7 +110,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const
|
||||
const DatabasePtr database = databases.at(database_name);
|
||||
|
||||
offsets[i] = i ? offsets[i - 1] : 0;
|
||||
for (auto iterator = database->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = database->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
String table_name = iterator->name();
|
||||
StoragePtr storage = iterator->table();
|
||||
|
@ -76,7 +76,7 @@ Pipes StorageSystemReplicas::read(
|
||||
if (db.second->getEngineName() == "Lazy")
|
||||
continue;
|
||||
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (!dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
continue;
|
||||
|
@ -60,7 +60,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const
|
||||
|
||||
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, db.first);
|
||||
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(); iterator->isValid(); iterator->next())
|
||||
{
|
||||
if (!dynamic_cast<const StorageReplicatedMergeTree *>(iterator->table().get()))
|
||||
continue;
|
||||
|
@ -18,6 +18,7 @@
|
||||
#include <Disks/DiskSpaceMonitor.h>
|
||||
#include <Processors/Sources/SourceWithProgress.h>
|
||||
#include <Processors/Pipe.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -36,6 +37,7 @@ StorageSystemTables::StorageSystemTables(const std::string & name_)
|
||||
{
|
||||
{"database", std::make_shared<DataTypeString>()},
|
||||
{"name", std::make_shared<DataTypeString>()},
|
||||
{"uuid", std::make_shared<DataTypeUUID>()},
|
||||
{"engine", std::make_shared<DataTypeString>()},
|
||||
{"is_temporary", std::make_shared<DataTypeUInt8>()},
|
||||
{"data_paths", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
|
||||
@ -74,7 +76,7 @@ static bool needLockStructure(const DatabasePtr & database, const Block & header
|
||||
if (database->getEngineName() != "Lazy")
|
||||
return true;
|
||||
|
||||
static const std::set<std::string> columns_without_lock = { "database", "name", "metadata_modification_time" };
|
||||
static const std::set<std::string> columns_without_lock = { "database", "name", "uuid", "metadata_modification_time" };
|
||||
for (const auto & column : header.getColumnsWithTypeAndName())
|
||||
{
|
||||
if (columns_without_lock.find(column.name) == columns_without_lock.end())
|
||||
@ -152,6 +154,10 @@ protected:
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(table.first);
|
||||
|
||||
// uuid
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(table.second->getStorageID().uuid);
|
||||
|
||||
// engine
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(table.second->getName());
|
||||
@ -226,7 +232,7 @@ protected:
|
||||
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
|
||||
|
||||
if (!tables_it || !tables_it->isValid())
|
||||
tables_it = database->getTablesIterator(context);
|
||||
tables_it = database->getTablesIterator();
|
||||
|
||||
const bool need_lock_structure = needLockStructure(database, getPort().getHeader());
|
||||
|
||||
@ -271,6 +277,9 @@ protected:
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(table_name);
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(tables_it->uuid());
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
{
|
||||
assert(table != nullptr);
|
||||
@ -322,7 +331,7 @@ protected:
|
||||
|
||||
if (columns_mask[src_index] || columns_mask[src_index + 1])
|
||||
{
|
||||
ASTPtr ast = database->tryGetCreateTableQuery(context, table_name);
|
||||
ASTPtr ast = database->tryGetCreateTableQuery(table_name);
|
||||
|
||||
if (columns_mask[src_index++])
|
||||
res_columns[res_index++]->insert(ast ? queryToString(ast) : "");
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user