2019-10-23 13:46:38 +00:00
|
|
|
#include <Databases/DatabaseAtomic.h>
|
2019-10-30 12:17:52 +00:00
|
|
|
#include <Databases/DatabaseOnDisk.h>
|
2019-11-11 11:34:03 +00:00
|
|
|
#include <Poco/File.h>
|
|
|
|
#include <IO/ReadHelpers.h>
|
2020-01-15 16:17:04 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2020-01-23 19:10:09 +00:00
|
|
|
#include <Common/Stopwatch.h>
|
2020-01-27 20:31:39 +00:00
|
|
|
#include <Parsers/formatAST.h>
|
2020-03-23 00:12:13 +00:00
|
|
|
#include <Common/rename.h>
|
2020-04-10 01:35:37 +00:00
|
|
|
#include <Storages/StorageMaterializedView.h>
|
2019-10-23 13:46:38 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
2019-11-11 11:34:03 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int UNKNOWN_TABLE;
|
|
|
|
extern const int TABLE_ALREADY_EXISTS;
|
2020-03-23 00:12:13 +00:00
|
|
|
extern const int CANNOT_ASSIGN_ALTER;
|
2020-04-10 23:02:15 +00:00
|
|
|
extern const int DATABASE_NOT_EMPTY;
|
2020-04-11 15:38:41 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2019-11-11 11:34:03 +00:00
|
|
|
}
|
2019-10-23 13:46:38 +00:00
|
|
|
|
2020-04-01 22:41:29 +00:00
|
|
|
class AtomicDatabaseTablesSnapshotIterator final : public DatabaseTablesSnapshotIterator
|
|
|
|
{
|
|
|
|
public:
|
2020-04-08 01:02:00 +00:00
|
|
|
explicit AtomicDatabaseTablesSnapshotIterator(DatabaseTablesSnapshotIterator && base)
|
2020-04-01 22:41:29 +00:00
|
|
|
: DatabaseTablesSnapshotIterator(std::move(base)) {}
|
|
|
|
UUID uuid() const override { return table()->getStorageID().uuid; }
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2020-01-22 11:30:11 +00:00
|
|
|
DatabaseAtomic::DatabaseAtomic(String name_, String metadata_path_, Context & context_)
|
2020-04-06 23:22:44 +00:00
|
|
|
: DatabaseOrdinary(name_, metadata_path_, "store/", "DatabaseAtomic (" + name_ + ")", context_)
|
2020-04-10 00:08:43 +00:00
|
|
|
, path_to_table_symlinks(context_.getPath() + "data/" + escapeForFileName(name_) + "/")
|
2019-10-23 13:46:38 +00:00
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
/// Symlinks in data/db_name/ directory are not used by ClickHouse,
|
|
|
|
/// it's needed only for convenient introspection.
|
2020-04-10 00:08:43 +00:00
|
|
|
Poco::File(path_to_table_symlinks).createDirectories();
|
2019-11-11 11:34:03 +00:00
|
|
|
}
|
|
|
|
|
2019-12-25 21:17:49 +00:00
|
|
|
String DatabaseAtomic::getTableDataPath(const String & table_name) const
|
2019-11-11 11:34:03 +00:00
|
|
|
{
|
2019-12-02 19:11:18 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2019-11-11 11:34:03 +00:00
|
|
|
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);
|
2020-01-16 18:13:18 +00:00
|
|
|
assert(it->second != data_path && !it->second.empty());
|
|
|
|
return it->second;
|
2019-11-11 11:34:03 +00:00
|
|
|
}
|
|
|
|
|
2019-12-25 21:17:49 +00:00
|
|
|
String DatabaseAtomic::getTableDataPath(const ASTCreateQuery & query) const
|
2019-11-11 11:34:03 +00:00
|
|
|
{
|
2020-04-06 23:22:44 +00:00
|
|
|
auto tmp = data_path + DatabaseCatalog::getPathForUUID(query.uuid);
|
2020-01-16 18:13:18 +00:00
|
|
|
assert(tmp != data_path && !tmp.empty());
|
|
|
|
return tmp;
|
2019-11-11 11:34:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void DatabaseAtomic::drop(const Context &)
|
|
|
|
{
|
2020-04-10 00:08:43 +00:00
|
|
|
Poco::File(path_to_table_symlinks).remove(true);
|
2020-01-27 20:31:39 +00:00
|
|
|
Poco::File(getMetadataPath()).remove(true);
|
2019-11-11 11:34:03 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path)
|
|
|
|
{
|
2020-01-16 18:13:18 +00:00
|
|
|
assert(relative_table_path != data_path && !relative_table_path.empty());
|
2020-04-06 23:22:44 +00:00
|
|
|
DetachedTables not_in_use;
|
2020-04-22 20:43:10 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2020-04-06 23:22:44 +00:00
|
|
|
not_in_use = cleenupDetachedTables();
|
2020-03-30 19:15:18 +00:00
|
|
|
assertDetachedTableNotInUse(table->getStorageID().uuid);
|
2020-04-22 20:43:10 +00:00
|
|
|
DatabaseWithDictionaries::attachTableUnlocked(name, table, lock);
|
2019-12-02 19:11:18 +00:00
|
|
|
table_name_to_path.emplace(std::make_pair(name, relative_table_path));
|
2020-04-10 00:08:43 +00:00
|
|
|
tryCreateSymlink(name, relative_table_path);
|
2019-10-23 13:46:38 +00:00
|
|
|
}
|
|
|
|
|
2019-11-11 11:34:03 +00:00
|
|
|
StoragePtr DatabaseAtomic::detachTable(const String & name)
|
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
DetachedTables not_in_use;
|
|
|
|
std::unique_lock lock(mutex);
|
|
|
|
auto table = DatabaseWithDictionaries::detachTableUnlocked(name, lock);
|
2020-03-23 00:12:13 +00:00
|
|
|
table_name_to_path.erase(name);
|
2020-03-30 19:15:18 +00:00
|
|
|
detached_tables.emplace(table->getStorageID().uuid, table);
|
2020-04-06 23:22:44 +00:00
|
|
|
not_in_use = cleenupDetachedTables();
|
2020-04-10 00:08:43 +00:00
|
|
|
tryRemoveSymlink(name);
|
2020-03-23 00:12:13 +00:00
|
|
|
return table;
|
2019-11-11 11:34:03 +00:00
|
|
|
}
|
2019-10-23 13:46:38 +00:00
|
|
|
|
2020-03-20 00:07:52 +00:00
|
|
|
void DatabaseAtomic::dropTable(const Context &, const String & table_name, bool no_delay)
|
2020-01-22 11:30:11 +00:00
|
|
|
{
|
|
|
|
String table_metadata_path = getObjectMetadataPath(table_name);
|
2020-03-23 00:12:13 +00:00
|
|
|
String table_metadata_path_drop;
|
|
|
|
StoragePtr table;
|
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
std::unique_lock lock(mutex);
|
|
|
|
table = getTableUnlocked(table_name, lock);
|
2020-03-23 00:12:13 +00:00
|
|
|
table_metadata_path_drop = DatabaseCatalog::instance().getPathForDroppedMetadata(table->getStorageID());
|
2020-04-22 20:43:10 +00:00
|
|
|
Poco::File(table_metadata_path).renameTo(table_metadata_path_drop); /// Mark table as dropped
|
|
|
|
DatabaseWithDictionaries::detachTableUnlocked(table_name, lock); /// Should never throw
|
2020-03-23 00:12:13 +00:00
|
|
|
table_name_to_path.erase(table_name);
|
|
|
|
}
|
2020-04-10 00:08:43 +00:00
|
|
|
tryRemoveSymlink(table_name);
|
2020-04-22 20:43:10 +00:00
|
|
|
/// 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.
|
2020-03-20 00:07:52 +00:00
|
|
|
DatabaseCatalog::instance().enqueueDroppedTableCleanup(table->getStorageID(), table, table_metadata_path_drop, no_delay);
|
2020-01-22 11:30:11 +00:00
|
|
|
}
|
|
|
|
|
2019-11-11 14:28:28 +00:00
|
|
|
void DatabaseAtomic::renameTable(const Context & context, const String & table_name, IDatabase & to_database,
|
2020-03-31 20:38:05 +00:00
|
|
|
const String & to_table_name, bool exchange)
|
2019-11-11 14:28:28 +00:00
|
|
|
{
|
|
|
|
if (typeid(*this) != typeid(to_database))
|
2019-12-02 19:11:18 +00:00
|
|
|
{
|
|
|
|
if (!typeid_cast<DatabaseOrdinary *>(&to_database))
|
|
|
|
throw Exception("Moving tables between databases of different engines is not supported", ErrorCodes::NOT_IMPLEMENTED);
|
2020-03-31 20:38:05 +00:00
|
|
|
/// Allow moving tables between Atomic and Ordinary (with table lock)
|
|
|
|
DatabaseOnDisk::renameTable(context, table_name, to_database, to_table_name, exchange);
|
2019-12-02 19:11:18 +00:00
|
|
|
return;
|
|
|
|
}
|
2020-03-31 20:38:05 +00:00
|
|
|
auto & other_db = dynamic_cast<DatabaseAtomic &>(to_database);
|
2020-04-10 01:35:37 +00:00
|
|
|
bool inside_database = this == &other_db;
|
2019-11-11 14:28:28 +00:00
|
|
|
|
2020-03-23 00:12:13 +00:00
|
|
|
String old_metadata_path = getObjectMetadataPath(table_name);
|
2020-03-23 22:40:40 +00:00
|
|
|
String new_metadata_path = to_database.getObjectMetadataPath(to_table_name);
|
2019-11-11 14:28:28 +00:00
|
|
|
|
2020-04-10 23:02:15 +00:00
|
|
|
auto detach = [](DatabaseAtomic & db, const String & table_name_)
|
2020-03-23 00:12:13 +00:00
|
|
|
{
|
2020-04-07 14:05:51 +00:00
|
|
|
auto table_data_path_saved = db.table_name_to_path.find(table_name_)->second;
|
2020-03-31 20:38:05 +00:00
|
|
|
db.tables.erase(table_name_);
|
|
|
|
db.table_name_to_path.erase(table_name_);
|
2020-04-10 23:02:15 +00:00
|
|
|
db.tryRemoveSymlink(table_name_);
|
2020-04-07 14:05:51 +00:00
|
|
|
return table_data_path_saved;
|
2020-03-31 20:38:05 +00:00
|
|
|
};
|
|
|
|
|
2020-04-10 23:02:15 +00:00
|
|
|
auto attach = [](DatabaseAtomic & db, const String & table_name_, const String & table_data_path_, const StoragePtr & table_)
|
2020-03-31 20:38:05 +00:00
|
|
|
{
|
|
|
|
db.tables.emplace(table_name_, table_);
|
|
|
|
db.table_name_to_path.emplace(table_name_, table_data_path_);
|
2020-04-10 23:02:15 +00:00
|
|
|
db.tryCreateSymlink(table_name_, table_data_path_);
|
2020-03-31 20:38:05 +00:00
|
|
|
};
|
|
|
|
|
2020-04-11 15:38:41 +00:00
|
|
|
auto assert_can_move_mat_view = [inside_database](const StoragePtr & table_)
|
2020-04-10 01:35:37 +00:00
|
|
|
{
|
|
|
|
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);
|
|
|
|
};
|
|
|
|
|
2020-03-31 20:38:05 +00:00
|
|
|
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};
|
2020-03-23 00:12:13 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2020-03-31 20:38:05 +00:00
|
|
|
other_db_lock = std::unique_lock{other_db.mutex};
|
|
|
|
db_lock = std::unique_lock{mutex};
|
2020-03-23 00:12:13 +00:00
|
|
|
}
|
2020-03-31 20:38:05 +00:00
|
|
|
|
2020-04-22 20:43:10 +00:00
|
|
|
StoragePtr table = getTableUnlocked(table_name, db_lock);
|
2020-04-11 15:38:41 +00:00
|
|
|
assert_can_move_mat_view(table);
|
2020-04-10 00:08:43 +00:00
|
|
|
StoragePtr other_table;
|
|
|
|
if (exchange)
|
2020-04-10 01:35:37 +00:00
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
other_table = other_db.getTableUnlocked(to_table_name, other_db_lock);
|
2020-04-11 15:38:41 +00:00
|
|
|
assert_can_move_mat_view(other_table);
|
2020-04-10 01:35:37 +00:00
|
|
|
}
|
2020-04-10 00:08:43 +00:00
|
|
|
|
2020-04-22 20:43:10 +00:00
|
|
|
/// Table renaming actually begins here
|
2020-03-31 20:38:05 +00:00
|
|
|
if (exchange)
|
|
|
|
renameExchange(old_metadata_path, new_metadata_path);
|
|
|
|
else
|
|
|
|
renameNoReplace(old_metadata_path, new_metadata_path);
|
|
|
|
|
2020-04-22 20:43:10 +00:00
|
|
|
/// After metadata was successfully moved, the following methods should not throw (if them do, it's a logical error)
|
2020-03-31 20:38:05 +00:00
|
|
|
table_data_path = detach(*this, table_name);
|
|
|
|
if (exchange)
|
|
|
|
other_table_data_path = detach(other_db, to_table_name);
|
|
|
|
|
2020-04-07 14:05:51 +00:00
|
|
|
table->renameInMemory({other_db.getDatabaseName(), to_table_name, table->getStorageID().uuid});
|
2020-03-31 20:38:05 +00:00
|
|
|
if (exchange)
|
2020-04-07 14:05:51 +00:00
|
|
|
other_table->renameInMemory({getDatabaseName(), table_name, other_table->getStorageID().uuid});
|
2020-03-31 20:38:05 +00:00
|
|
|
|
|
|
|
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);
|
2019-11-11 14:28:28 +00:00
|
|
|
}
|
|
|
|
|
2020-03-23 00:12:13 +00:00
|
|
|
void DatabaseAtomic::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
|
|
|
|
const String & table_metadata_tmp_path, const String & table_metadata_path)
|
|
|
|
{
|
2020-04-06 23:22:44 +00:00
|
|
|
DetachedTables not_in_use;
|
2020-03-23 00:12:13 +00:00
|
|
|
auto table_data_path = getTableDataPath(query);
|
|
|
|
try
|
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
std::unique_lock lock{mutex};
|
2020-04-06 23:22:44 +00:00
|
|
|
not_in_use = cleenupDetachedTables();
|
2020-03-30 19:15:18 +00:00
|
|
|
assertDetachedTableNotInUse(query.uuid);
|
2020-03-23 00:12:13 +00:00
|
|
|
renameNoReplace(table_metadata_tmp_path, table_metadata_path);
|
2020-04-22 20:43:10 +00:00
|
|
|
attachTableUnlocked(query.table, table, lock); /// Should never throw
|
2020-03-23 00:12:13 +00:00
|
|
|
table_name_to_path.emplace(query.table, table_data_path);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
Poco::File(table_metadata_tmp_path).remove();
|
|
|
|
throw;
|
|
|
|
}
|
2020-04-10 00:08:43 +00:00
|
|
|
tryCreateSymlink(query.table, table_data_path);
|
2020-03-23 00:12:13 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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(); });
|
|
|
|
|
2020-04-22 20:43:10 +00:00
|
|
|
std::unique_lock lock{mutex};
|
|
|
|
auto actual_table_id = getTableUnlocked(table_id.table_name, lock)->getStorageID();
|
2020-03-23 00:12:13 +00:00
|
|
|
|
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2020-03-30 19:15:18 +00:00
|
|
|
void DatabaseAtomic::assertDetachedTableNotInUse(const UUID & uuid)
|
|
|
|
{
|
2020-04-22 20:43:10 +00:00
|
|
|
/// 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.
|
2020-03-30 19:15:18 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2020-04-06 23:22:44 +00:00
|
|
|
DatabaseAtomic::DetachedTables DatabaseAtomic::cleenupDetachedTables()
|
2020-03-30 19:15:18 +00:00
|
|
|
{
|
2020-04-06 23:22:44 +00:00
|
|
|
DetachedTables not_in_use;
|
2020-03-30 19:15:18 +00:00
|
|
|
auto it = detached_tables.begin();
|
|
|
|
while (it != detached_tables.end())
|
|
|
|
{
|
|
|
|
if (it->second.unique())
|
2020-04-06 23:22:44 +00:00
|
|
|
{
|
|
|
|
not_in_use.emplace(it->first, it->second);
|
2020-03-30 19:15:18 +00:00
|
|
|
it = detached_tables.erase(it);
|
2020-04-06 23:22:44 +00:00
|
|
|
}
|
2020-03-30 19:15:18 +00:00
|
|
|
else
|
|
|
|
++it;
|
|
|
|
}
|
2020-04-22 20:43:10 +00:00
|
|
|
/// It should be destroyed in caller with released database mutex
|
2020-04-06 23:22:44 +00:00
|
|
|
return not_in_use;
|
2020-03-30 19:15:18 +00:00
|
|
|
}
|
|
|
|
|
2020-04-10 23:02:15 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2020-04-01 22:41:29 +00:00
|
|
|
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)));
|
|
|
|
}
|
|
|
|
|
2020-04-13 14:09:56 +00:00
|
|
|
UUID DatabaseAtomic::tryGetTableUUID(const String & table_name) const
|
|
|
|
{
|
2020-04-23 16:51:48 +00:00
|
|
|
if (auto table = tryGetTable(table_name))
|
2020-04-13 14:09:56 +00:00
|
|
|
return table->getStorageID().uuid;
|
|
|
|
return UUIDHelpers::Nil;
|
|
|
|
}
|
|
|
|
|
2020-04-10 00:08:43 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-10-23 13:46:38 +00:00
|
|
|
}
|
|
|
|
|