mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
use relative paths in IStorage::rename(...)
This commit is contained in:
parent
ac3f53f300
commit
514868cebc
@ -156,7 +156,7 @@ void DatabaseOnDisk::renameTable(
|
||||
/// Notify the table that it is renamed. If the table does not support renaming, exception is thrown.
|
||||
try
|
||||
{
|
||||
table->rename(context.getPath() + "/data/" + escapeForFileName(to_database_concrete->getDatabaseName()) + "/",
|
||||
table->rename("/data/" + escapeForFileName(to_database_concrete->getDatabaseName()) + "/" + escapeForFileName(to_table_name) + '/',
|
||||
to_database_concrete->getDatabaseName(),
|
||||
to_table_name, lock);
|
||||
}
|
||||
|
@ -304,23 +304,12 @@ public:
|
||||
* In this function, you need to rename the directory with the data, if any.
|
||||
* Called when the table structure is locked for write.
|
||||
*/
|
||||
virtual void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/,
|
||||
virtual void rename(const String & /*new_path_to_table_data*/, const String & /*new_database_name*/, const String & /*new_table_name*/,
|
||||
TableStructureWriteLockHolder &)
|
||||
{
|
||||
throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
// TODO refactor rename() and renameAtomic()
|
||||
|
||||
/**
|
||||
* Just updates names of database and table without moving any data on disk
|
||||
* Can be called only from DatabaseAtomic.
|
||||
*/
|
||||
virtual void renameAtomic(const String & /*new_database_name*/, const String & /*new_table_name*/)
|
||||
{
|
||||
throw Exception("Method renameAtomic is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/** 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.
|
||||
* To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata.
|
||||
|
@ -1215,13 +1215,9 @@ void MergeTreeData::clearPartsFromFilesystem(const DataPartsVector & parts_to_re
|
||||
}
|
||||
|
||||
void MergeTreeData::rename(
|
||||
const String & /*new_path_to_db*/, const String & new_database_name,
|
||||
const String & new_table_path, const String & new_database_name,
|
||||
const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
{
|
||||
auto old_table_path = relative_data_path;
|
||||
auto new_db_path = "data/" + escapeForFileName(new_database_name) + '/';
|
||||
auto new_table_path = new_db_path + escapeForFileName(new_table_name) + '/';
|
||||
|
||||
auto disks = storage_policy->getDisks();
|
||||
|
||||
for (const auto & disk : disks)
|
||||
@ -1232,15 +1228,16 @@ void MergeTreeData::rename(
|
||||
|
||||
for (const auto & disk : disks)
|
||||
{
|
||||
disk->createDirectory(new_db_path);
|
||||
disk->moveFile(old_table_path, new_table_path);
|
||||
auto new_table_path_parent = Poco::Path(new_table_path).makeParent().toString();
|
||||
disk->createDirectory(new_table_path_parent);
|
||||
disk->moveFile(relative_data_path, new_table_path);
|
||||
}
|
||||
|
||||
global_context.dropCaches();
|
||||
|
||||
database_name = new_database_name;
|
||||
table_name = new_table_name;
|
||||
relative_data_path = new_table_path;
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
}
|
||||
|
||||
void MergeTreeData::dropAllData()
|
||||
@ -3532,9 +3529,7 @@ void MergeTreeData::freezePartitionsByMatcher(MatcherFn matcher, const String &
|
||||
|
||||
String part_absolute_path = Poco::Path(part->getFullPath()).absolute().toString();
|
||||
String backup_part_absolute_path = backup_path
|
||||
+ "data/"
|
||||
+ escapeForFileName(getDatabaseName()) + "/"
|
||||
+ escapeForFileName(getTableName()) + "/"
|
||||
+ relative_data_path
|
||||
+ part->relative_path;
|
||||
localBackup(part_absolute_path, backup_part_absolute_path);
|
||||
part->is_frozen.store(true, std::memory_order_relaxed);
|
||||
|
@ -539,7 +539,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_path_to_db, const String & new_database_name,
|
||||
void rename(const String & new_path_to_table_data, const String & new_database_name,
|
||||
const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
|
||||
/// Check if the ALTER can be performed:
|
||||
|
@ -472,7 +472,7 @@ void StorageDistributed::createDirectoryMonitors()
|
||||
if (path.empty())
|
||||
return;
|
||||
|
||||
Poco::File{path}.createDirectory();
|
||||
Poco::File{path}.createDirectories();
|
||||
|
||||
std::filesystem::directory_iterator begin(path);
|
||||
std::filesystem::directory_iterator end;
|
||||
|
@ -82,7 +82,7 @@ public:
|
||||
/// Removes temporary data in local filesystem.
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
void rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override
|
||||
void rename(const String & /*new_path_to_table_data*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override
|
||||
{
|
||||
table_name = new_table_name;
|
||||
database_name = new_database_name;
|
||||
|
@ -159,13 +159,14 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu
|
||||
if (relative_table_dir_path.empty())
|
||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
String table_dir_path = args.context.getPath() + relative_table_dir_path + "/";
|
||||
String table_dir_path = base_path + relative_table_dir_path + "/";
|
||||
Poco::File(table_dir_path).createDirectories();
|
||||
paths = {getTablePath(table_dir_path, format_name)};
|
||||
}
|
||||
|
||||
StorageFile::StorageFile(CommonArguments args)
|
||||
: table_name(args.table_name), database_name(args.database_name), format_name(args.format_name), compression_method(args.compression_method)
|
||||
: table_name(args.table_name), database_name(args.database_name), format_name(args.format_name)
|
||||
, compression_method(args.compression_method), base_path(args.context.getPath())
|
||||
{
|
||||
setColumns(args.columns);
|
||||
setConstraints(args.constraints);
|
||||
@ -342,7 +343,7 @@ Strings StorageFile::getDataPaths() const
|
||||
return paths;
|
||||
}
|
||||
|
||||
void StorageFile::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageFile::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
{
|
||||
if (!is_db_table)
|
||||
throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED);
|
||||
@ -352,7 +353,7 @@ void StorageFile::rename(const String & new_path_to_db, const String & new_datab
|
||||
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
std::string path_new = getTablePath(new_path_to_db + escapeForFileName(new_table_name), format_name);
|
||||
std::string path_new = getTablePath(base_path + new_path_to_table_data, format_name);
|
||||
Poco::File(Poco::Path(path_new).parent()).createDirectories();
|
||||
Poco::File(paths[0]).renameTo(path_new);
|
||||
|
||||
|
@ -38,7 +38,7 @@ public:
|
||||
const ASTPtr & query,
|
||||
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 rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
|
||||
Strings getDataPaths() const override;
|
||||
|
||||
@ -76,6 +76,7 @@ private:
|
||||
int table_fd = -1;
|
||||
String compression_method;
|
||||
|
||||
std::string base_path;
|
||||
std::vector<std::string> paths;
|
||||
|
||||
bool is_db_table = true; /// Table is stored in real database, not user's file
|
||||
|
@ -425,7 +425,7 @@ StorageLog::StorageLog(
|
||||
const ConstraintsDescription & constraints_,
|
||||
size_t max_compress_block_size_,
|
||||
const Context & context_)
|
||||
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||
: base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||
max_compress_block_size(max_compress_block_size_),
|
||||
file_checker(path + "sizes.json")
|
||||
{
|
||||
@ -513,12 +513,12 @@ void StorageLog::loadMarks()
|
||||
}
|
||||
|
||||
|
||||
void StorageLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
/// Rename directory with data.
|
||||
String new_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
|
||||
String new_path = base_path + new_path_to_table_data;
|
||||
Poco::File(path).renameTo(new_path);
|
||||
|
||||
path = new_path;
|
||||
|
@ -38,7 +38,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, 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 rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||
|
||||
@ -63,6 +63,7 @@ protected:
|
||||
const Context & context_);
|
||||
|
||||
private:
|
||||
String base_path;
|
||||
String path;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
@ -3755,10 +3755,10 @@ void StorageReplicatedMergeTree::drop(TableStructureWriteLockHolder &)
|
||||
|
||||
|
||||
void StorageReplicatedMergeTree::rename(
|
||||
const String & new_path_to_db, const String & new_database_name,
|
||||
const String & new_path_to_table_data, const String & new_database_name,
|
||||
const String & new_table_name, TableStructureWriteLockHolder & lock)
|
||||
{
|
||||
MergeTreeData::rename(new_path_to_db, new_database_name, new_table_name, lock);
|
||||
MergeTreeData::rename(new_path_to_table_data, new_database_name, new_table_name, lock);
|
||||
|
||||
/// Update table name in zookeeper
|
||||
auto zookeeper = getZooKeeper();
|
||||
|
@ -119,7 +119,7 @@ public:
|
||||
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
|
||||
bool supportsIndexForIn() const override { return true; }
|
||||
|
||||
|
@ -102,7 +102,8 @@ StorageSetOrJoinBase::StorageSetOrJoinBase(
|
||||
if (relative_path_.empty())
|
||||
throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||
|
||||
path = context_.getPath() + relative_path_;
|
||||
base_path = context_.getPath();
|
||||
path = base_path + relative_path_;
|
||||
}
|
||||
|
||||
|
||||
@ -201,10 +202,10 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path)
|
||||
|
||||
|
||||
void StorageSetOrJoinBase::rename(
|
||||
const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
{
|
||||
/// Rename directory with data.
|
||||
String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/";
|
||||
String new_path = base_path + new_path_to_table_data;
|
||||
Poco::File(path).renameTo(new_path);
|
||||
|
||||
path = new_path;
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
String getTableName() const override { return table_name; }
|
||||
String getDatabaseName() const override { return database_name; }
|
||||
|
||||
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
void rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override;
|
||||
|
||||
@ -37,6 +37,7 @@ protected:
|
||||
const ConstraintsDescription & constraints_,
|
||||
const Context & context_);
|
||||
|
||||
String base_path;
|
||||
String path;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
@ -202,7 +202,7 @@ StorageStripeLog::StorageStripeLog(
|
||||
bool attach,
|
||||
size_t max_compress_block_size_,
|
||||
const Context & context_)
|
||||
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||
: base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||
max_compress_block_size(max_compress_block_size_),
|
||||
file_checker(path + "sizes.json"),
|
||||
log(&Logger::get("StorageStripeLog"))
|
||||
@ -223,13 +223,13 @@ StorageStripeLog::StorageStripeLog(
|
||||
}
|
||||
|
||||
|
||||
void StorageStripeLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageStripeLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
/// Rename directory with data.
|
||||
String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/";
|
||||
Poco::File(path).renameTo(new_path_to_db + escapeForFileName(new_table_name));
|
||||
String new_path = base_path + new_path_to_table_data;
|
||||
Poco::File(path).renameTo(new_path);
|
||||
|
||||
path = new_path;
|
||||
table_name = new_table_name;
|
||||
|
@ -40,7 +40,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, 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 rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||
|
||||
@ -58,6 +58,7 @@ public:
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
private:
|
||||
String base_path;
|
||||
String path;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
@ -331,7 +331,7 @@ StorageTinyLog::StorageTinyLog(
|
||||
bool attach,
|
||||
size_t max_compress_block_size_,
|
||||
const Context & context_)
|
||||
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||
: base_path(context_.getPath()), path(base_path + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||
max_compress_block_size(max_compress_block_size_),
|
||||
file_checker(path + "sizes.json"),
|
||||
log(&Logger::get("StorageTinyLog"))
|
||||
@ -378,12 +378,12 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
|
||||
}
|
||||
|
||||
|
||||
void StorageTinyLog::rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
void StorageTinyLog::rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||
{
|
||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||
|
||||
/// Rename directory with data.
|
||||
String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/";
|
||||
String new_path = base_path + new_path_to_table_data;
|
||||
Poco::File(path).renameTo(new_path);
|
||||
|
||||
path = new_path;
|
||||
|
@ -39,7 +39,7 @@ public:
|
||||
|
||||
BlockOutputStreamPtr write(const ASTPtr & query, 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 rename(const String & new_path_to_table_data, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override;
|
||||
|
||||
CheckResults checkData(const ASTPtr & /* query */, const Context & /* context */) override;
|
||||
|
||||
@ -57,6 +57,7 @@ public:
|
||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||
|
||||
private:
|
||||
String base_path;
|
||||
String path;
|
||||
String table_name;
|
||||
String database_name;
|
||||
|
Loading…
Reference in New Issue
Block a user