make data path relative

This commit is contained in:
Alexander Tokmakov 2019-10-25 22:07:47 +03:00
parent 9c461ff143
commit 80c9b2ec1c
30 changed files with 176 additions and 144 deletions

View File

@ -32,22 +32,22 @@ namespace ErrorCodes
static constexpr size_t METADATA_FILE_BUFFER_SIZE = 32768;
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context)
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_)
: name(name_)
, metadata_path(metadata_path_)
, data_path(context.getPath() + "data/" + escapeForFileName(name) + "/")
, data_path("data/" + escapeForFileName(name) + "/")
, expiration_time(expiration_time_)
, log(&Logger::get("DatabaseLazy (" + name + ")"))
{
Poco::File(getDataPath()).createDirectories();
Poco::File(context_.getPath() + getDataPath()).createDirectories();
}
void DatabaseLazy::loadTables(
Context & /* context */,
Context & context,
bool /* has_force_restore_data_flag */)
{
DatabaseOnDisk::iterateTableFiles(*this, log, [this](const String & file_name)
DatabaseOnDisk::iterateTableFiles(*this, log, context, [this](const String & file_name)
{
const std::string table_name = file_name.substr(0, file_name.size() - 4);
attachTable(table_name, nullptr);
@ -134,9 +134,9 @@ void DatabaseLazy::alterTable(
}
void DatabaseLazy::drop()
void DatabaseLazy::drop(const Context & context)
{
DatabaseOnDisk::drop(*this);
DatabaseOnDisk::drop(*this, context);
}
bool DatabaseLazy::isTableExist(

View File

@ -16,7 +16,7 @@ class DatabaseLazyIterator;
class DatabaseLazy : public IDatabase
{
public:
DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context);
DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, const Context & context_);
String getEngineName() const override { return "Lazy"; }
@ -68,7 +68,7 @@ public:
String getMetadataPath() const override;
String getTableMetadataPath(const String & table_name) const override;
void drop() override;
void drop(const Context & context) override;
bool isTableExist(
const Context & context,

View File

@ -220,9 +220,9 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const IDatabase & database, const
return ast;
}
void DatabaseOnDisk::drop(const IDatabase & database)
void DatabaseOnDisk::drop(const IDatabase & database, const Context & context)
{
Poco::File(database.getDataPath()).remove(false);
Poco::File(context.getPath() + database.getDataPath()).remove(false);
Poco::File(database.getMetadataPath()).remove(false);
}
@ -248,7 +248,7 @@ time_t DatabaseOnDisk::getTableMetadataModificationTime(
}
}
void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function)
void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function)
{
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it(database.getMetadataPath()); dir_it != dir_end; ++dir_it)
@ -266,7 +266,7 @@ void DatabaseOnDisk::iterateTableFiles(const IDatabase & database, Poco::Logger
if (endsWith(dir_it.name(), tmp_drop_ext))
{
const std::string table_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
if (Poco::File(database.getDataPath() + '/' + table_name).exists())
if (Poco::File(context.getPath() + database.getDataPath() + '/' + table_name).exists())
{
Poco::File(dir_it->path()).renameTo(table_name + ".sql");
LOG_WARNING(log, "Table " << backQuote(table_name) << " was not dropped previously");

View File

@ -60,7 +60,7 @@ public:
const IDatabase & database,
const Context & context);
static void drop(const IDatabase & database);
static void drop(const IDatabase & database, const Context & context);
static String getTableMetadataPath(
const IDatabase & database,
@ -72,7 +72,7 @@ public:
using IteratingFunction = std::function<void(const String &)>;
static void iterateTableFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function);
static void iterateTableFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function);
private:
static ASTPtr getCreateTableQueryImpl(

View File

@ -87,13 +87,13 @@ static void loadTable(
}
DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context)
DatabaseOrdinary::DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context_)
: DatabaseWithOwnTablesBase(std::move(name_))
, metadata_path(metadata_path_)
, data_path(context.getPath() + "data/" + escapeForFileName(name) + "/")
, data_path("data/" + escapeForFileName(name) + "/")
, log(&Logger::get("DatabaseOrdinary (" + name + ")"))
{
Poco::File(getDataPath()).createDirectories();
Poco::File(context_.getPath() + getDataPath()).createDirectories();
}
@ -104,7 +104,7 @@ void DatabaseOrdinary::loadTables(
using FileNames = std::vector<std::string>;
FileNames file_names;
DatabaseOnDisk::iterateTableFiles(*this, log, [&file_names](const String & file_name)
DatabaseOnDisk::iterateTableFiles(*this, log, context, [&file_names](const String & file_name)
{
file_names.push_back(file_name);
});
@ -299,9 +299,9 @@ void DatabaseOrdinary::alterTable(
}
void DatabaseOrdinary::drop()
void DatabaseOrdinary::drop(const Context & context)
{
DatabaseOnDisk::drop(*this);
DatabaseOnDisk::drop(*this, context);
}

View File

@ -66,7 +66,7 @@ public:
String getMetadataPath() const override;
String getTableMetadataPath(const String & table_name) const override;
void drop() override;
void drop(const Context & context) override;
private:
const String metadata_path;

View File

@ -9,6 +9,7 @@
#include <Storages/IStorage.h>
#include <Storages/StorageFactory.h>
#include <Common/typeid_cast.h>
#include <Common/escapeForFileName.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <sstream>
@ -57,7 +58,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query)
std::pair<String, StoragePtr> createTableFromDefinition(
const String & definition,
const String & database_name,
const String & database_data_path,
const String & relative_db_path,
Context & context,
bool has_force_restore_data_flag,
const String & description_for_error_message)
@ -90,7 +91,8 @@ std::pair<String, StoragePtr> createTableFromDefinition(
ast_create_query.table,
StorageFactory::instance().get(
ast_create_query,
database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(),
relative_db_path + escapeForFileName(ast_create_query.table) + '/',
ast_create_query.table, database_name, context, context.getGlobalContext(),
columns, constraints,
true, has_force_restore_data_flag)
};

View File

@ -31,7 +31,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query);
std::pair<String, StoragePtr> createTableFromDefinition(
const String & definition,
const String & database_name,
const String & database_data_path,
const String & relative_db_path,
Context & context,
bool has_force_restore_data_flag,
const String & description_for_error_message);

View File

@ -156,7 +156,7 @@ public:
virtual void shutdown() = 0;
/// Delete data and metadata stored inside the database, if exists.
virtual void drop() {}
virtual void drop(const Context & /*context*/) {}
virtual ~IDatabase() {}
};

View File

@ -646,7 +646,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
else
{
res = StorageFactory::instance().get(create,
data_path,
data_path + escapeForFileName(table_name) + "/",
table_name,
database_name,
context,

View File

@ -119,7 +119,7 @@ BlockIO InterpreterDropQuery::executeToTable(String & database_name_, String & t
database_and_table.first->removeTable(context, database_and_table.second->getTableName());
database_and_table.second->is_dropped = true;
String database_data_path = database_and_table.first->getDataPath();
String database_data_path = context.getPath() + database_and_table.first->getDataPath();
/// If it is not virtual database like Dictionary then drop remaining data dir
if (!database_data_path.empty())
@ -206,7 +206,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQ
database->shutdown();
/// Delete the database.
database->drop();
database->drop(context);
/// Old ClickHouse versions did not store database.sql files
Poco::File database_metadata_file(context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql");

View File

@ -5,6 +5,7 @@
#include <Common/typeid_cast.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <Common/ThreadPool.h>
#include <Common/escapeForFileName.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExternalDictionariesLoader.h>
#include <Interpreters/EmbeddedDictionaries.h>
@ -289,7 +290,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints);
StoragePtr table = StorageFactory::instance().get(create,
data_path,
data_path + escapeForFileName(table_name) + "/",
table_name,
database_name,
system_context,

View File

@ -285,6 +285,17 @@ public:
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

@ -215,12 +215,12 @@ StorageDistributed::StorageDistributed(
const String & cluster_name_,
const Context & context_,
const ASTPtr & sharding_key_,
const String & data_path_,
const String & relative_data_path_,
bool attach_)
: table_name(table_name_), database_name(database_name_),
remote_database(remote_database_), remote_table(remote_table_),
global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_),
path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(table_name) + '/'))
path(relative_data_path_.empty() ? "" : (context_.getPath() + relative_data_path_))
{
setColumns(columns_);
setConstraints(constraints_);
@ -250,9 +250,9 @@ StorageDistributed::StorageDistributed(
const String & cluster_name_,
const Context & context_,
const ASTPtr & sharding_key_,
const String & data_path_,
const String & relative_data_path_,
bool attach)
: StorageDistributed(database_name_, table_name_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, data_path_, attach)
: StorageDistributed(database_name_, table_name_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, relative_data_path_, attach)
{
remote_table_function_ptr = remote_table_function_ptr_;
}
@ -641,7 +641,7 @@ void registerStorageDistributed(StorageFactory & factory)
return StorageDistributed::create(
args.database_name, args.table_name, args.columns, args.constraints,
remote_database, remote_table, cluster_name,
args.context, sharding_key, args.data_path,
args.context, sharding_key, args.relative_data_path,
args.attach);
});
}

View File

@ -172,7 +172,7 @@ protected:
const String & cluster_name_,
const Context & context_,
const ASTPtr & sharding_key_,
const String & data_path_,
const String & relative_data_path_,
bool attach_);
StorageDistributed(
@ -184,7 +184,7 @@ protected:
const String & cluster_name_,
const Context & context_,
const ASTPtr & sharding_key_,
const String & data_path_,
const String & relative_data_path_,
bool attach);
ClusterPtr skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info);

View File

@ -40,9 +40,9 @@ void StorageFactory::registerStorage(const std::string & name, Creator creator)
StoragePtr StorageFactory::get(
const ASTCreateQuery & query,
const String & data_path,
const String & relative_data_path,
const String & table_name,
const String & database_name,
const String & database_name, // TODO remove
Context & local_context,
Context & context,
const ColumnsDescription & columns,
@ -150,9 +150,9 @@ StoragePtr StorageFactory::get(
.engine_args = args,
.storage_def = storage_def,
.query = query,
.data_path = data_path,
.relative_data_path = relative_data_path,
.table_name = table_name,
.database_name = database_name,
.database_name = database_name, //TODO remove
.local_context = local_context,
.context = context,
.columns = columns,

View File

@ -32,9 +32,11 @@ public:
ASTs & engine_args;
ASTStorage * storage_def;
const ASTCreateQuery & query;
const String & data_path;
/// Path to table data.
/// Relative to <path> from server config (possibly <path> of some <disk> of some <volume> for *MergeTree)
const String & relative_data_path;
const String & table_name;
const String & database_name;
const String & database_name; //TODO remove
Context & local_context;
Context & context;
const ColumnsDescription & columns;
@ -47,7 +49,7 @@ public:
StoragePtr get(
const ASTCreateQuery & query,
const String & data_path,
const String & relative_data_path,
const String & table_name,
const String & database_name,
Context & local_context,

View File

@ -52,7 +52,7 @@ namespace
/* Recursive directory listing with matched paths as a result.
* Have the same method in StorageHDFS.
*/
std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match)
static std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match)
{
const size_t first_glob = for_match.find_first_of("*?{");
@ -96,13 +96,13 @@ std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_fo
return result;
}
std::string getTablePath(const std::string & db_dir_path, const std::string & table_name, const std::string & format_name)
static std::string getTablePath(const std::string & table_dir_path, const std::string & format_name)
{
return db_dir_path + escapeForFileName(table_name) + "/data." + escapeForFileName(format_name);
return table_dir_path + "/data." + escapeForFileName(format_name);
}
/// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..').
void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path)
static void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path)
{
if (context_global.getApplicationType() != Context::ApplicationType::SERVER)
return;
@ -121,7 +121,7 @@ void checkCreationIsAllowed(Context & context_global, const std::string & db_dir
StorageFile::StorageFile(
const std::string & table_path_,
int table_fd_,
const std::string & db_dir_path,
const std::string & relative_table_dir_path,
const std::string & database_name_,
const std::string & table_name_,
const std::string & format_name_,
@ -136,26 +136,27 @@ StorageFile::StorageFile(
if (table_fd < 0) /// Will use file
{
String table_dir_path = context_global.getPath() + relative_table_dir_path + "/";
use_table_fd = false;
if (!table_path_.empty()) /// Is user's file
{
Poco::Path poco_path = Poco::Path(table_path_);
if (poco_path.isRelative())
poco_path = Poco::Path(db_dir_path, poco_path);
poco_path = Poco::Path(table_dir_path, poco_path);
const std::string path = poco_path.absolute().toString();
paths = listFilesWithRegexpMatching("/", path);
for (const auto & cur_path : paths)
checkCreationIsAllowed(context_global, db_dir_path, cur_path);
checkCreationIsAllowed(context_global, table_dir_path, cur_path);
is_db_table = false;
}
else /// Is DB's file
{
if (db_dir_path.empty())
if (relative_table_dir_path.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
paths = {getTablePath(db_dir_path, table_name, format_name)};
paths = {getTablePath(table_dir_path, format_name)};
is_db_table = true;
Poco::File(Poco::Path(paths.back()).parent()).createDirectories();
}
@ -344,7 +345,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, new_table_name, format_name);
std::string path_new = getTablePath(new_path_to_db + escapeForFileName(new_table_name), format_name);
Poco::File(Poco::Path(path_new).parent()).createDirectories();
Poco::File(paths[0]).renameTo(path_new);
@ -400,7 +401,7 @@ void registerStorageFile(StorageFactory & factory)
return StorageFile::create(
source_path, source_fd,
args.data_path,
args.relative_data_path,
args.database_name, args.table_name, format_name, args.columns, args.constraints,
args.context);
});

View File

@ -54,7 +54,7 @@ protected:
StorageFile(
const std::string & table_path_,
int table_fd_,
const std::string & db_dir_path,
const std::string & relative_table_dir_path,
const std::string & database_name_,
const std::string & table_name_,
const std::string & format_name_,

View File

@ -28,7 +28,7 @@ namespace ErrorCodes
}
StorageJoin::StorageJoin(
const String & path_,
const String & relative_path_,
const String & database_name_,
const String & table_name_,
const Names & key_names_,
@ -38,8 +38,9 @@ StorageJoin::StorageJoin(
ASTTableJoin::Strictness strictness_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
bool overwrite)
: StorageSetOrJoinBase{path_, database_name_, table_name_, columns_, constraints_}
bool overwrite,
const Context & context_)
: StorageSetOrJoinBase{relative_path_, database_name_, table_name_, columns_, constraints_, context_}
, key_names(key_names_)
, use_nulls(use_nulls_)
, limits(limits_)
@ -162,7 +163,7 @@ void registerStorageJoin(StorageFactory & factory)
}
return StorageJoin::create(
args.data_path,
args.relative_data_path,
args.database_name,
args.table_name,
key_names,
@ -172,7 +173,8 @@ void registerStorageJoin(StorageFactory & factory)
strictness,
args.columns,
args.constraints,
join_any_take_last_row);
join_any_take_last_row,
args.context);
});
}

View File

@ -59,7 +59,7 @@ private:
protected:
StorageJoin(
const String & path_,
const String & relative_path_,
const String & database_name_,
const String & table_name_,
const Names & key_names_,
@ -68,7 +68,8 @@ protected:
ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
bool overwrite);
bool overwrite,
const Context & context_);
};
}

View File

@ -211,7 +211,7 @@ Block LogBlockInputStream::readImpl()
}
catch (Exception & e)
{
e.addMessage("while reading column " + name_type.name + " at " + storage.path + escapeForFileName(storage.table_name));
e.addMessage("while reading column " + name_type.name + " at " + storage.path);
throw;
}
@ -418,29 +418,30 @@ void LogBlockOutputStream::writeMarks(MarksForColumns && marks)
}
StorageLog::StorageLog(
const std::string & path_,
const std::string & relative_path_,
const std::string & database_name_,
const std::string & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
size_t max_compress_block_size_)
: path(path_), table_name(table_name_), database_name(database_name_),
size_t max_compress_block_size_,
const Context & context_)
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
max_compress_block_size(max_compress_block_size_),
file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json")
file_checker(path + "sizes.json")
{
setColumns(columns_);
setConstraints(constraints_);
if (path.empty())
if (relative_path_.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
/// create files if they do not exist
Poco::File(path + escapeForFileName(table_name) + '/').createDirectories();
Poco::File(path).createDirectories();
for (const auto & column : getColumns().getAllPhysical())
addFiles(column.name, *column.type);
marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
marks_file = Poco::File(path + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
}
@ -459,7 +460,7 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type)
ColumnData & column_data = files[stream_name];
column_data.column_index = file_count;
column_data.data_file = Poco::File{
path + escapeForFileName(table_name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION};
path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION};
column_names_by_idx.push_back(stream_name);
++file_count;
@ -517,24 +518,25 @@ void StorageLog::rename(const String & new_path_to_db, const String & new_databa
std::unique_lock<std::shared_mutex> lock(rwlock);
/// Rename directory with data.
Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
String new_path = new_path_to_db + escapeForFileName(new_table_name) + '/';
Poco::File(path).renameTo(new_path);
path = new_path_to_db;
path = new_path;
table_name = new_table_name;
database_name = new_database_name;
file_checker.setPath(path + escapeForFileName(table_name) + '/' + "sizes.json");
file_checker.setPath(path + "sizes.json");
for (auto & file : files)
file.second.data_file = Poco::File(path + escapeForFileName(table_name) + '/' + Poco::Path(file.second.data_file.path()).getFileName());
file.second.data_file = Poco::File(path + Poco::Path(file.second.data_file.path()).getFileName());
marks_file = Poco::File(path + escapeForFileName(table_name) + '/' + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
marks_file = Poco::File(path + DBMS_STORAGE_LOG_MARKS_FILE_NAME);
}
void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
{
std::shared_lock<std::shared_mutex> lock(rwlock);
String table_dir = path + escapeForFileName(table_name);
String table_dir = path;
files.clear();
file_count = 0;
@ -647,8 +649,8 @@ void registerStorageLog(StorageFactory & factory)
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return StorageLog::create(
args.data_path, args.database_name, args.table_name, args.columns, args.constraints,
args.context.getSettings().max_compress_block_size);
args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints,
args.context.getSettings().max_compress_block_size, args.context);
});
}

View File

@ -44,7 +44,7 @@ public:
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
std::string fullPath() const { return path + escapeForFileName(table_name) + '/';}
std::string fullPath() const { return path; }
Strings getDataPaths() const override { return {fullPath()}; }
@ -54,12 +54,13 @@ protected:
* consisting of the specified columns; Create files if they do not exist.
*/
StorageLog(
const std::string & path_,
const std::string & relative_path_,
const std::string & database_name_,
const std::string & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
size_t max_compress_block_size_);
size_t max_compress_block_size_,
const Context & context_);
private:
String path;
@ -124,7 +125,7 @@ private:
*/
const Marks & getMarksWithRealRowCount() const;
std::string getFullPath() const { return path + escapeForFileName(table_name) + '/'; }
std::string getFullPath() const { return path; }
};
}

View File

@ -87,31 +87,33 @@ BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const
StorageSetOrJoinBase::StorageSetOrJoinBase(
const String & path_,
const String & relative_path_,
const String & database_name_,
const String & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_)
const ConstraintsDescription & constraints_,
const Context & context_)
: table_name(table_name_), database_name(database_name_)
{
setColumns(columns_);
setConstraints(constraints_);
if (path_.empty())
if (relative_path_.empty())
throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
path = path_ + escapeForFileName(table_name_) + '/';
path = context_.getPath() + relative_path_;
}
StorageSet::StorageSet(
const String & path_,
const String & relative_path_,
const String & database_name_,
const String & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_)
: StorageSetOrJoinBase{path_, database_name_, table_name_, columns_, constraints_},
const ConstraintsDescription & constraints_,
const Context & context_)
: StorageSetOrJoinBase{relative_path_, database_name_, table_name_, columns_, constraints_, context_},
set(std::make_shared<Set>(SizeLimits(), false))
{
Block header = getSampleBlock();
@ -197,10 +199,10 @@ void StorageSetOrJoinBase::rename(
const String & new_path_to_db, 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 = new_path_to_db + escapeForFileName(new_table_name) + "/";
Poco::File(path).renameTo(new_path);
path = new_path + "/";
path = new_path;
table_name = new_table_name;
database_name = new_database_name;
}
@ -215,7 +217,7 @@ void registerStorageSet(StorageFactory & factory)
"Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return StorageSet::create(args.data_path, args.database_name, args.table_name, args.columns, args.constraints);
return StorageSet::create(args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints, args.context);
});
}

View File

@ -30,11 +30,12 @@ public:
protected:
StorageSetOrJoinBase(
const String & path_,
const String & relative_path_,
const String & database_name_,
const String & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_);
const ConstraintsDescription & constraints_,
const Context & context_);
String path;
String table_name;
@ -79,11 +80,12 @@ private:
protected:
StorageSet(
const String & path_,
const String & relative_path_,
const String & database_name_,
const String & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_);
const ConstraintsDescription & constraints_,
const Context & context_);
};
}

View File

@ -121,8 +121,8 @@ private:
started = true;
data_in.emplace(
storage.full_path() + "data.bin", 0, 0,
std::min(static_cast<Poco::File::FileSize>(max_read_buffer_size), Poco::File(storage.full_path() + "data.bin").getSize()));
storage.fullPath() + "data.bin", 0, 0,
std::min(static_cast<Poco::File::FileSize>(max_read_buffer_size), Poco::File(storage.fullPath() + "data.bin").getSize()));
block_in.emplace(*data_in, 0, index_begin, index_end);
}
@ -135,11 +135,11 @@ class StripeLogBlockOutputStream final : public IBlockOutputStream
public:
explicit StripeLogBlockOutputStream(StorageStripeLog & storage_)
: storage(storage_), lock(storage.rwlock),
data_out_compressed(storage.full_path() + "data.bin", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT),
data_out_compressed(storage.fullPath() + "data.bin", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT),
data_out(data_out_compressed, CompressionCodecFactory::instance().getDefaultCodec(), storage.max_compress_block_size),
index_out_compressed(storage.full_path() + "index.mrk", INDEX_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT),
index_out_compressed(storage.fullPath() + "index.mrk", INDEX_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT),
index_out(index_out_compressed),
block_out(data_out, 0, storage.getSampleBlock(), false, &index_out, Poco::File(storage.full_path() + "data.bin").getSize())
block_out(data_out, 0, storage.getSampleBlock(), false, &index_out, Poco::File(storage.fullPath() + "data.bin").getSize())
{
}
@ -194,30 +194,30 @@ private:
StorageStripeLog::StorageStripeLog(
const std::string & path_,
const std::string & relative_path_,
const std::string & database_name_,
const std::string & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
bool attach,
size_t max_compress_block_size_)
: path(path_), table_name(table_name_), database_name(database_name_),
size_t max_compress_block_size_,
const Context & context_)
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
max_compress_block_size(max_compress_block_size_),
file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json"),
file_checker(path + "sizes.json"),
log(&Logger::get("StorageStripeLog"))
{
setColumns(columns_);
setConstraints(constraints_);
if (path.empty())
if (relative_path_.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
String full_path = path + escapeForFileName(table_name) + '/';
if (!attach)
{
/// create files if they do not exist
if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
throwFromErrnoWithPath("Cannot create directory " + full_path, full_path,
if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
throwFromErrnoWithPath("Cannot create directory " + path, path,
ErrorCodes::CANNOT_CREATE_DIRECTORY);
}
}
@ -228,12 +228,13 @@ void StorageStripeLog::rename(const String & new_path_to_db, const String & new_
std::unique_lock<std::shared_mutex> lock(rwlock);
/// Rename directory with data.
Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/";
Poco::File(path).renameTo(new_path_to_db + escapeForFileName(new_table_name));
path = new_path_to_db;
path = new_path;
table_name = new_table_name;
database_name = new_database_name;
file_checker.setPath(path + escapeForFileName(table_name) + "/" + "sizes.json");
file_checker.setPath(path + "sizes.json");
}
@ -251,10 +252,10 @@ BlockInputStreams StorageStripeLog::read(
NameSet column_names_set(column_names.begin(), column_names.end());
if (!Poco::File(full_path() + "index.mrk").exists())
if (!Poco::File(fullPath() + "index.mrk").exists())
return { std::make_shared<NullBlockInputStream>(getSampleBlockForColumns(column_names)) };
CompressedReadBufferFromFile index_in(full_path() + "index.mrk", 0, 0, INDEX_BUFFER_SIZE);
CompressedReadBufferFromFile index_in(fullPath() + "index.mrk", 0, 0, INDEX_BUFFER_SIZE);
std::shared_ptr<const IndexForNativeFormat> index{std::make_shared<IndexForNativeFormat>(index_in, column_names_set)};
BlockInputStreams res;
@ -301,11 +302,11 @@ void StorageStripeLog::truncate(const ASTPtr &, const Context &, TableStructureW
std::shared_lock<std::shared_mutex> lock(rwlock);
auto file = Poco::File(path + escapeForFileName(table_name));
auto file = Poco::File(path);
file.remove(true);
file.createDirectories();
file_checker = FileChecker{path + escapeForFileName(table_name) + '/' + "sizes.json"};
file_checker = FileChecker{path + "sizes.json"};
}
@ -319,8 +320,8 @@ void registerStorageStripeLog(StorageFactory & factory)
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return StorageStripeLog::create(
args.data_path, args.database_name, args.table_name, args.columns, args.constraints,
args.attach, args.context.getSettings().max_compress_block_size);
args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints,
args.attach, args.context.getSettings().max_compress_block_size, args.context);
});
}

View File

@ -51,9 +51,9 @@ public:
};
using Files_t = std::map<String, ColumnData>;
std::string full_path() const { return path + escapeForFileName(table_name) + '/';}
std::string fullPath() const { return path; }
Strings getDataPaths() const override { return {full_path()}; }
Strings getDataPaths() const override { return {fullPath()}; }
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
@ -71,13 +71,14 @@ private:
protected:
StorageStripeLog(
const std::string & path_,
const std::string & relative_path_,
const std::string & database_name_,
const std::string & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
bool attach,
size_t max_compress_block_size_);
size_t max_compress_block_size_,
const Context & context_);
};
}

View File

@ -323,30 +323,30 @@ void TinyLogBlockOutputStream::write(const Block & block)
StorageTinyLog::StorageTinyLog(
const std::string & path_,
const std::string & relative_path_,
const std::string & database_name_,
const std::string & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
bool attach,
size_t max_compress_block_size_)
: path(path_), table_name(table_name_), database_name(database_name_),
size_t max_compress_block_size_,
const Context & context_)
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
max_compress_block_size(max_compress_block_size_),
file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json"),
file_checker(path + "sizes.json"),
log(&Logger::get("StorageTinyLog"))
{
setColumns(columns_);
setConstraints(constraints_);
if (path.empty())
if (relative_path_.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
String full_path = path + escapeForFileName(table_name) + '/';
if (!attach)
{
/// create files if they do not exist
if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
throwFromErrnoWithPath("Cannot create directory " + full_path, full_path,
if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
throwFromErrnoWithPath("Cannot create directory " + path, path,
ErrorCodes::CANNOT_CREATE_DIRECTORY);
}
@ -369,7 +369,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
ColumnData column_data;
files.insert(std::make_pair(stream_name, column_data));
files[stream_name].data_file = Poco::File(
path + escapeForFileName(table_name) + '/' + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
path + stream_name + DBMS_STORAGE_LOG_DATA_FILE_EXTENSION);
}
};
@ -383,15 +383,16 @@ void StorageTinyLog::rename(const String & new_path_to_db, const String & new_da
std::unique_lock<std::shared_mutex> lock(rwlock);
/// Rename directory with data.
Poco::File(path + escapeForFileName(table_name)).renameTo(new_path_to_db + escapeForFileName(new_table_name));
String new_path = new_path_to_db + escapeForFileName(new_table_name) + "/";
Poco::File(path).renameTo(new_path);
path = new_path_to_db;
path = new_path;
table_name = new_table_name;
database_name = new_database_name;
file_checker.setPath(path + escapeForFileName(table_name) + "/" + "sizes.json");
file_checker.setPath(path + "sizes.json");
for (Files_t::iterator it = files.begin(); it != files.end(); ++it)
it->second.data_file = Poco::File(path + escapeForFileName(table_name) + '/' + Poco::Path(it->second.data_file.path()).getFileName());
it->second.data_file = Poco::File(path + Poco::Path(it->second.data_file.path()).getFileName());
}
@ -431,12 +432,12 @@ void StorageTinyLog::truncate(const ASTPtr &, const Context &, TableStructureWri
std::unique_lock<std::shared_mutex> lock(rwlock);
auto file = Poco::File(path + escapeForFileName(table_name));
auto file = Poco::File(path);
file.remove(true);
file.createDirectories();
files.clear();
file_checker = FileChecker{path + escapeForFileName(table_name) + '/' + "sizes.json"};
file_checker = FileChecker{path + "sizes.json"};
for (const auto &column : getColumns().getAllPhysical())
addFiles(column.name, *column.type);
@ -453,8 +454,8 @@ void registerStorageTinyLog(StorageFactory & factory)
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return StorageTinyLog::create(
args.data_path, args.database_name, args.table_name, args.columns, args.constraints,
args.attach, args.context.getSettings().max_compress_block_size);
args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints,
args.attach, args.context.getSettings().max_compress_block_size, args.context);
});
}

View File

@ -50,7 +50,7 @@ public:
};
using Files_t = std::map<String, ColumnData>;
std::string fullPath() const { return path + escapeForFileName(table_name) + '/';}
std::string fullPath() const { return path; }
Strings getDataPaths() const override { return {fullPath()}; }
@ -75,13 +75,14 @@ private:
protected:
StorageTinyLog(
const std::string & path_,
const std::string & relative_path_,
const std::string & database_name_,
const std::string & table_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
bool attach,
size_t max_compress_block_size_);
size_t max_compress_block_size_,
const Context & context_);
};
}

View File

@ -9,6 +9,7 @@ namespace DB
NamesAndTypesList StorageSystemDatabases::getNamesAndTypes()
{
//FIXME data_path
return {
{"name", std::make_shared<DataTypeString>()},
{"engine", std::make_shared<DataTypeString>()},