use relative paths in IStorage::rename(...)

This commit is contained in:
Alexander Tokmakov 2019-12-24 16:29:53 +03:00
parent ac3f53f300
commit 514868cebc
18 changed files with 44 additions and 53 deletions

View File

@ -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);
}

View File

@ -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.

View File

@ -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);

View File

@ -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:

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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

View 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;

View File

@ -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;

View File

@ -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();

View File

@ -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; }

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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;