mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
make data path relative
This commit is contained in:
parent
9041977bb6
commit
4f58d662f7
@ -30,22 +30,22 @@ namespace ErrorCodes
|
|||||||
|
|
||||||
|
|
||||||
|
|
||||||
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_)
|
: name(name_)
|
||||||
, metadata_path(metadata_path_)
|
, metadata_path(metadata_path_)
|
||||||
, data_path(context.getPath() + "data/" + escapeForFileName(name) + "/")
|
, data_path("data/" + escapeForFileName(name) + "/")
|
||||||
, expiration_time(expiration_time_)
|
, expiration_time(expiration_time_)
|
||||||
, log(&Logger::get("DatabaseLazy (" + name + ")"))
|
, log(&Logger::get("DatabaseLazy (" + name + ")"))
|
||||||
{
|
{
|
||||||
Poco::File(getDataPath()).createDirectories();
|
Poco::File(context_.getPath() + getDataPath()).createDirectories();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DatabaseLazy::loadStoredObjects(
|
void DatabaseLazy::loadStoredObjects(
|
||||||
Context & /* context */,
|
Context & context,
|
||||||
bool /* has_force_restore_data_flag */)
|
bool /* has_force_restore_data_flag */)
|
||||||
{
|
{
|
||||||
DatabaseOnDisk::iterateMetadataFiles(*this, log, [this](const String & file_name)
|
DatabaseOnDisk::iterateMetadataFiles(*this, log, context, [this](const String & file_name)
|
||||||
{
|
{
|
||||||
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
const std::string table_name = file_name.substr(0, file_name.size() - 4);
|
||||||
attachTable(table_name, nullptr);
|
attachTable(table_name, nullptr);
|
||||||
@ -185,9 +185,9 @@ void DatabaseLazy::alterTable(
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DatabaseLazy::drop()
|
void DatabaseLazy::drop(const Context & context)
|
||||||
{
|
{
|
||||||
DatabaseOnDisk::drop(*this);
|
DatabaseOnDisk::drop(*this, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool DatabaseLazy::isTableExist(
|
bool DatabaseLazy::isTableExist(
|
||||||
|
@ -18,7 +18,7 @@ class DatabaseLazyIterator;
|
|||||||
class DatabaseLazy : public IDatabase
|
class DatabaseLazy : public IDatabase
|
||||||
{
|
{
|
||||||
public:
|
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"; }
|
String getEngineName() const override { return "Lazy"; }
|
||||||
|
|
||||||
@ -87,7 +87,7 @@ public:
|
|||||||
String getMetadataPath() const override;
|
String getMetadataPath() const override;
|
||||||
String getObjectMetadataPath(const String & table_name) const override;
|
String getObjectMetadataPath(const String & table_name) const override;
|
||||||
|
|
||||||
void drop() override;
|
void drop(const Context & context) override;
|
||||||
|
|
||||||
bool isTableExist(
|
bool isTableExist(
|
||||||
const Context & context,
|
const Context & context,
|
||||||
|
@ -332,7 +332,7 @@ void DatabaseMySQL::shutdown()
|
|||||||
local_tables_cache.clear();
|
local_tables_cache.clear();
|
||||||
}
|
}
|
||||||
|
|
||||||
void DatabaseMySQL::drop()
|
void DatabaseMySQL::drop(const Context & /*context*/)
|
||||||
{
|
{
|
||||||
Poco::File(getMetadataPath()).remove(true);
|
Poco::File(getMetadataPath()).remove(true);
|
||||||
}
|
}
|
||||||
|
@ -60,7 +60,7 @@ public:
|
|||||||
|
|
||||||
void shutdown() override;
|
void shutdown() override;
|
||||||
|
|
||||||
void drop() override;
|
void drop(const Context & /*context*/) override;
|
||||||
|
|
||||||
String getMetadataPath() const override;
|
String getMetadataPath() const override;
|
||||||
|
|
||||||
|
@ -493,9 +493,9 @@ ASTPtr DatabaseOnDisk::getCreateDatabaseQuery(const IDatabase & database, const
|
|||||||
return ast;
|
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);
|
Poco::File(database.getMetadataPath()).remove(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -517,7 +517,7 @@ time_t DatabaseOnDisk::getObjectMetadataModificationTime(
|
|||||||
return static_cast<time_t>(0);
|
return static_cast<time_t>(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function)
|
void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function)
|
||||||
{
|
{
|
||||||
Poco::DirectoryIterator dir_end;
|
Poco::DirectoryIterator dir_end;
|
||||||
for (Poco::DirectoryIterator dir_it(database.getMetadataPath()); dir_it != dir_end; ++dir_it)
|
for (Poco::DirectoryIterator dir_it(database.getMetadataPath()); dir_it != dir_end; ++dir_it)
|
||||||
@ -535,7 +535,7 @@ void DatabaseOnDisk::iterateMetadataFiles(const IDatabase & database, Poco::Logg
|
|||||||
if (endsWith(dir_it.name(), tmp_drop_ext))
|
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));
|
const std::string object_name = dir_it.name().substr(0, dir_it.name().size() - strlen(tmp_drop_ext));
|
||||||
if (Poco::File(database.getDataPath() + '/' + object_name).exists())
|
if (Poco::File(context.getPath() + database.getDataPath() + '/' + object_name).exists())
|
||||||
{
|
{
|
||||||
/// TODO maybe complete table drop and remove all table data (including data on other volumes and metadata in ZK)
|
/// 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(database.getMetadataPath() + object_name + ".sql");
|
Poco::File(dir_it->path()).renameTo(database.getMetadataPath() + object_name + ".sql");
|
||||||
|
@ -98,7 +98,7 @@ public:
|
|||||||
const IDatabase & database,
|
const IDatabase & database,
|
||||||
const Context & context);
|
const Context & context);
|
||||||
|
|
||||||
static void drop(const IDatabase & database);
|
static void drop(const IDatabase & database, const Context & context);
|
||||||
|
|
||||||
static String getObjectMetadataPath(
|
static String getObjectMetadataPath(
|
||||||
const IDatabase & database,
|
const IDatabase & database,
|
||||||
@ -110,7 +110,7 @@ public:
|
|||||||
|
|
||||||
|
|
||||||
using IteratingFunction = std::function<void(const String &)>;
|
using IteratingFunction = std::function<void(const String &)>;
|
||||||
static void iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const IteratingFunction & iterating_function);
|
static void iterateMetadataFiles(const IDatabase & database, Poco::Logger * log, const Context & context, const IteratingFunction & iterating_function);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
static ASTPtr getCreateTableQueryImpl(
|
static ASTPtr getCreateTableQueryImpl(
|
||||||
|
@ -115,13 +115,13 @@ namespace
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
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_))
|
: DatabaseWithOwnTablesBase(std::move(name_))
|
||||||
, metadata_path(metadata_path_)
|
, metadata_path(metadata_path_)
|
||||||
, data_path(context.getPath() + "data/" + escapeForFileName(name) + "/")
|
, data_path("data/" + escapeForFileName(name) + "/")
|
||||||
, log(&Logger::get("DatabaseOrdinary (" + name + ")"))
|
, log(&Logger::get("DatabaseOrdinary (" + name + ")"))
|
||||||
{
|
{
|
||||||
Poco::File(getDataPath()).createDirectories();
|
Poco::File(context_.getPath() + getDataPath()).createDirectories();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -138,7 +138,7 @@ void DatabaseOrdinary::loadStoredObjects(
|
|||||||
FileNames file_names;
|
FileNames file_names;
|
||||||
|
|
||||||
size_t total_dictionaries = 0;
|
size_t total_dictionaries = 0;
|
||||||
DatabaseOnDisk::iterateMetadataFiles(*this, log, [&file_names, &total_dictionaries, this](const String & file_name)
|
DatabaseOnDisk::iterateMetadataFiles(*this, log, context, [&file_names, &total_dictionaries, this](const String & file_name)
|
||||||
{
|
{
|
||||||
String full_path = metadata_path + "/" + file_name;
|
String full_path = metadata_path + "/" + file_name;
|
||||||
try
|
try
|
||||||
@ -374,9 +374,9 @@ void DatabaseOrdinary::alterTable(
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void DatabaseOrdinary::drop()
|
void DatabaseOrdinary::drop(const Context & context)
|
||||||
{
|
{
|
||||||
DatabaseOnDisk::drop(*this);
|
DatabaseOnDisk::drop(*this, context);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -83,7 +83,7 @@ public:
|
|||||||
String getMetadataPath() const override;
|
String getMetadataPath() const override;
|
||||||
String getObjectMetadataPath(const String & table_name) const override;
|
String getObjectMetadataPath(const String & table_name) const override;
|
||||||
|
|
||||||
void drop() override;
|
void drop(const Context & context) override;
|
||||||
|
|
||||||
private:
|
private:
|
||||||
const String metadata_path;
|
const String metadata_path;
|
||||||
|
@ -13,6 +13,7 @@
|
|||||||
#include <Storages/StorageDictionary.h>
|
#include <Storages/StorageDictionary.h>
|
||||||
#include <Storages/StorageFactory.h>
|
#include <Storages/StorageFactory.h>
|
||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
|
#include <Common/escapeForFileName.h>
|
||||||
#include <TableFunctions/TableFunctionFactory.h>
|
#include <TableFunctions/TableFunctionFactory.h>
|
||||||
#include <Dictionaries/DictionaryFactory.h>
|
#include <Dictionaries/DictionaryFactory.h>
|
||||||
|
|
||||||
@ -34,6 +35,7 @@ namespace ErrorCodes
|
|||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
|
|
||||||
|
|
||||||
StoragePtr getDictionaryStorage(const Context & context, const String & table_name, const String & db_name)
|
StoragePtr getDictionaryStorage(const Context & context, const String & table_name, const String & db_name)
|
||||||
{
|
{
|
||||||
auto dict_name = db_name + "." + table_name;
|
auto dict_name = db_name + "." + table_name;
|
||||||
|
@ -237,7 +237,7 @@ public:
|
|||||||
virtual void shutdown() = 0;
|
virtual void shutdown() = 0;
|
||||||
|
|
||||||
/// Delete data and metadata stored inside the database, if exists.
|
/// Delete data and metadata stored inside the database, if exists.
|
||||||
virtual void drop() {}
|
virtual void drop(const Context & /*context*/) {}
|
||||||
|
|
||||||
virtual ~IDatabase() {}
|
virtual ~IDatabase() {}
|
||||||
};
|
};
|
||||||
|
@ -632,7 +632,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create,
|
|||||||
else
|
else
|
||||||
{
|
{
|
||||||
res = StorageFactory::instance().get(create,
|
res = StorageFactory::instance().get(create,
|
||||||
data_path,
|
data_path + escapeForFileName(table_name) + "/",
|
||||||
table_name,
|
table_name,
|
||||||
create.database,
|
create.database,
|
||||||
context,
|
context,
|
||||||
|
@ -132,7 +132,7 @@ BlockIO InterpreterDropQuery::executeToTable(
|
|||||||
database_and_table.first->removeTable(context, database_and_table.second->getTableName());
|
database_and_table.first->removeTable(context, database_and_table.second->getTableName());
|
||||||
database_and_table.second->is_dropped = true;
|
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 it is not virtual database like Dictionary then drop remaining data dir
|
||||||
if (!database_data_path.empty())
|
if (!database_data_path.empty())
|
||||||
@ -269,7 +269,7 @@ BlockIO InterpreterDropQuery::executeToDatabase(String & database_name, ASTDropQ
|
|||||||
database->shutdown();
|
database->shutdown();
|
||||||
|
|
||||||
/// Delete the database.
|
/// Delete the database.
|
||||||
database->drop();
|
database->drop(context);
|
||||||
|
|
||||||
/// Old ClickHouse versions did not store database.sql files
|
/// Old ClickHouse versions did not store database.sql files
|
||||||
Poco::File database_metadata_file(context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql");
|
Poco::File database_metadata_file(context.getPath() + "metadata/" + escapeForFileName(database_name) + ".sql");
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <Common/typeid_cast.h>
|
#include <Common/typeid_cast.h>
|
||||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||||
#include <Common/ThreadPool.h>
|
#include <Common/ThreadPool.h>
|
||||||
|
#include <Common/escapeForFileName.h>
|
||||||
#include <Interpreters/Context.h>
|
#include <Interpreters/Context.h>
|
||||||
#include <Interpreters/ExternalDictionariesLoader.h>
|
#include <Interpreters/ExternalDictionariesLoader.h>
|
||||||
#include <Interpreters/EmbeddedDictionaries.h>
|
#include <Interpreters/EmbeddedDictionaries.h>
|
||||||
@ -291,7 +292,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const String & database_nam
|
|||||||
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints);
|
auto constraints = InterpreterCreateQuery::getConstraintsDescription(create.columns_list->constraints);
|
||||||
|
|
||||||
StoragePtr table = StorageFactory::instance().get(create,
|
StoragePtr table = StorageFactory::instance().get(create,
|
||||||
data_path,
|
data_path + escapeForFileName(table_name) + "/",
|
||||||
table_name,
|
table_name,
|
||||||
database_name,
|
database_name,
|
||||||
system_context,
|
system_context,
|
||||||
|
@ -310,6 +310,17 @@ public:
|
|||||||
throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
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.
|
/** 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.
|
* 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.
|
* To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata.
|
||||||
|
@ -214,7 +214,7 @@ StorageDistributed::StorageDistributed(
|
|||||||
const String & cluster_name_,
|
const String & cluster_name_,
|
||||||
const Context & context_,
|
const Context & context_,
|
||||||
const ASTPtr & sharding_key_,
|
const ASTPtr & sharding_key_,
|
||||||
const String & data_path_,
|
const String & relative_data_path_,
|
||||||
bool attach_)
|
bool attach_)
|
||||||
: IStorage(ColumnsDescription({
|
: IStorage(ColumnsDescription({
|
||||||
{"_shard_num", std::make_shared<DataTypeUInt32>()},
|
{"_shard_num", std::make_shared<DataTypeUInt32>()},
|
||||||
@ -222,7 +222,7 @@ StorageDistributed::StorageDistributed(
|
|||||||
table_name(table_name_), database_name(database_name_),
|
table_name(table_name_), database_name(database_name_),
|
||||||
remote_database(remote_database_), remote_table(remote_table_),
|
remote_database(remote_database_), remote_table(remote_table_),
|
||||||
global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_),
|
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_);
|
setColumns(columns_);
|
||||||
setConstraints(constraints_);
|
setConstraints(constraints_);
|
||||||
@ -252,9 +252,9 @@ StorageDistributed::StorageDistributed(
|
|||||||
const String & cluster_name_,
|
const String & cluster_name_,
|
||||||
const Context & context_,
|
const Context & context_,
|
||||||
const ASTPtr & sharding_key_,
|
const ASTPtr & sharding_key_,
|
||||||
const String & data_path_,
|
const String & relative_data_path_,
|
||||||
bool attach)
|
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_;
|
remote_table_function_ptr = remote_table_function_ptr_;
|
||||||
}
|
}
|
||||||
@ -649,7 +649,7 @@ void registerStorageDistributed(StorageFactory & factory)
|
|||||||
return StorageDistributed::create(
|
return StorageDistributed::create(
|
||||||
args.database_name, args.table_name, args.columns, args.constraints,
|
args.database_name, args.table_name, args.columns, args.constraints,
|
||||||
remote_database, remote_table, cluster_name,
|
remote_database, remote_table, cluster_name,
|
||||||
args.context, sharding_key, args.data_path,
|
args.context, sharding_key, args.relative_data_path,
|
||||||
args.attach);
|
args.attach);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -172,7 +172,7 @@ protected:
|
|||||||
const String & cluster_name_,
|
const String & cluster_name_,
|
||||||
const Context & context_,
|
const Context & context_,
|
||||||
const ASTPtr & sharding_key_,
|
const ASTPtr & sharding_key_,
|
||||||
const String & data_path_,
|
const String & relative_data_path_,
|
||||||
bool attach_);
|
bool attach_);
|
||||||
|
|
||||||
StorageDistributed(
|
StorageDistributed(
|
||||||
@ -184,7 +184,7 @@ protected:
|
|||||||
const String & cluster_name_,
|
const String & cluster_name_,
|
||||||
const Context & context_,
|
const Context & context_,
|
||||||
const ASTPtr & sharding_key_,
|
const ASTPtr & sharding_key_,
|
||||||
const String & data_path_,
|
const String & relative_data_path_,
|
||||||
bool attach);
|
bool attach);
|
||||||
|
|
||||||
ClusterPtr skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info);
|
ClusterPtr skipUnusedShards(ClusterPtr cluster, const SelectQueryInfo & query_info);
|
||||||
|
@ -40,9 +40,9 @@ void StorageFactory::registerStorage(const std::string & name, Creator creator)
|
|||||||
|
|
||||||
StoragePtr StorageFactory::get(
|
StoragePtr StorageFactory::get(
|
||||||
const ASTCreateQuery & query,
|
const ASTCreateQuery & query,
|
||||||
const String & data_path,
|
const String & relative_data_path,
|
||||||
const String & table_name,
|
const String & table_name,
|
||||||
const String & database_name,
|
const String & database_name, // TODO remove
|
||||||
Context & local_context,
|
Context & local_context,
|
||||||
Context & context,
|
Context & context,
|
||||||
const ColumnsDescription & columns,
|
const ColumnsDescription & columns,
|
||||||
@ -150,9 +150,9 @@ StoragePtr StorageFactory::get(
|
|||||||
.engine_args = args,
|
.engine_args = args,
|
||||||
.storage_def = storage_def,
|
.storage_def = storage_def,
|
||||||
.query = query,
|
.query = query,
|
||||||
.data_path = data_path,
|
.relative_data_path = relative_data_path,
|
||||||
.table_name = table_name,
|
.table_name = table_name,
|
||||||
.database_name = database_name,
|
.database_name = database_name, //TODO remove
|
||||||
.local_context = local_context,
|
.local_context = local_context,
|
||||||
.context = context,
|
.context = context,
|
||||||
.columns = columns,
|
.columns = columns,
|
||||||
|
@ -33,9 +33,11 @@ public:
|
|||||||
ASTs & engine_args;
|
ASTs & engine_args;
|
||||||
ASTStorage * storage_def;
|
ASTStorage * storage_def;
|
||||||
const ASTCreateQuery & query;
|
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 & table_name;
|
||||||
const String & database_name;
|
const String & database_name; //TODO remove
|
||||||
Context & local_context;
|
Context & local_context;
|
||||||
Context & context;
|
Context & context;
|
||||||
const ColumnsDescription & columns;
|
const ColumnsDescription & columns;
|
||||||
@ -48,7 +50,7 @@ public:
|
|||||||
|
|
||||||
StoragePtr get(
|
StoragePtr get(
|
||||||
const ASTCreateQuery & query,
|
const ASTCreateQuery & query,
|
||||||
const String & data_path,
|
const String & relative_data_path,
|
||||||
const String & table_name,
|
const String & table_name,
|
||||||
const String & database_name,
|
const String & database_name,
|
||||||
Context & local_context,
|
Context & local_context,
|
||||||
|
@ -54,7 +54,7 @@ namespace
|
|||||||
/* Recursive directory listing with matched paths as a result.
|
/* Recursive directory listing with matched paths as a result.
|
||||||
* Have the same method in StorageHDFS.
|
* 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("*?{");
|
const size_t first_glob = for_match.find_first_of("*?{");
|
||||||
|
|
||||||
@ -98,13 +98,13 @@ std::vector<std::string> listFilesWithRegexpMatching(const std::string & path_fo
|
|||||||
return result;
|
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 '..').
|
/// 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)
|
if (context_global.getApplicationType() != Context::ApplicationType::SERVER)
|
||||||
return;
|
return;
|
||||||
@ -121,7 +121,7 @@ void checkCreationIsAllowed(Context & context_global, const std::string & db_dir
|
|||||||
StorageFile::StorageFile(
|
StorageFile::StorageFile(
|
||||||
const std::string & table_path_,
|
const std::string & table_path_,
|
||||||
int table_fd_,
|
int table_fd_,
|
||||||
const std::string & db_dir_path,
|
const std::string & relative_table_dir_path,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
const std::string & format_name_,
|
const std::string & format_name_,
|
||||||
@ -135,17 +135,17 @@ StorageFile::StorageFile(
|
|||||||
setColumns(columns_);
|
setColumns(columns_);
|
||||||
setConstraints(constraints_);
|
setConstraints(constraints_);
|
||||||
|
|
||||||
std::string db_dir_path_abs = Poco::Path(db_dir_path).makeAbsolute().makeDirectory().toString();
|
|
||||||
|
|
||||||
if (table_fd < 0) /// Will use file
|
if (table_fd < 0) /// Will use file
|
||||||
{
|
{
|
||||||
|
String table_dir_path = context_global.getPath() + relative_table_dir_path + "/";
|
||||||
use_table_fd = false;
|
use_table_fd = false;
|
||||||
|
|
||||||
if (!table_path_.empty()) /// Is user's file
|
if (!table_path_.empty()) /// Is user's file
|
||||||
{
|
{
|
||||||
|
table_dir_path = Poco::Path(relative_table_dir_path).makeAbsolute().makeDirectory().toString();
|
||||||
Poco::Path poco_path = Poco::Path(table_path_);
|
Poco::Path poco_path = Poco::Path(table_path_);
|
||||||
if (poco_path.isRelative())
|
if (poco_path.isRelative())
|
||||||
poco_path = Poco::Path(db_dir_path_abs, poco_path);
|
poco_path = Poco::Path(table_dir_path, poco_path);
|
||||||
|
|
||||||
const std::string path = poco_path.absolute().toString();
|
const std::string path = poco_path.absolute().toString();
|
||||||
if (path.find_first_of("*?{") == std::string::npos)
|
if (path.find_first_of("*?{") == std::string::npos)
|
||||||
@ -155,15 +155,15 @@ StorageFile::StorageFile(
|
|||||||
else
|
else
|
||||||
paths = listFilesWithRegexpMatching("/", path);
|
paths = listFilesWithRegexpMatching("/", path);
|
||||||
for (const auto & cur_path : paths)
|
for (const auto & cur_path : paths)
|
||||||
checkCreationIsAllowed(context_global, db_dir_path_abs, cur_path);
|
checkCreationIsAllowed(context_global, table_dir_path, cur_path);
|
||||||
is_db_table = false;
|
is_db_table = false;
|
||||||
}
|
}
|
||||||
else /// Is DB's file
|
else /// Is DB's file
|
||||||
{
|
{
|
||||||
if (db_dir_path_abs.empty())
|
if (relative_table_dir_path.empty())
|
||||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||||
|
|
||||||
paths = {getTablePath(db_dir_path_abs, table_name, format_name)};
|
paths = {getTablePath(table_dir_path, format_name)};
|
||||||
is_db_table = true;
|
is_db_table = true;
|
||||||
Poco::File(Poco::Path(paths.back()).parent()).createDirectories();
|
Poco::File(Poco::Path(paths.back()).parent()).createDirectories();
|
||||||
}
|
}
|
||||||
@ -363,7 +363,7 @@ void StorageFile::rename(const String & new_path_to_db, const String & new_datab
|
|||||||
|
|
||||||
std::unique_lock<std::shared_mutex> lock(rwlock);
|
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(Poco::Path(path_new).parent()).createDirectories();
|
||||||
Poco::File(paths[0]).renameTo(path_new);
|
Poco::File(paths[0]).renameTo(path_new);
|
||||||
|
|
||||||
@ -425,7 +425,7 @@ void registerStorageFile(StorageFactory & factory)
|
|||||||
|
|
||||||
return StorageFile::create(
|
return StorageFile::create(
|
||||||
source_path, source_fd,
|
source_path, source_fd,
|
||||||
args.data_path,
|
args.relative_data_path,
|
||||||
args.database_name, args.table_name, format_name, args.columns, args.constraints,
|
args.database_name, args.table_name, format_name, args.columns, args.constraints,
|
||||||
args.context,
|
args.context,
|
||||||
compression_method);
|
compression_method);
|
||||||
|
@ -54,7 +54,7 @@ protected:
|
|||||||
StorageFile(
|
StorageFile(
|
||||||
const std::string & table_path_,
|
const std::string & table_path_,
|
||||||
int table_fd_,
|
int table_fd_,
|
||||||
const std::string & db_dir_path,
|
const std::string & relative_table_dir_path,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
const std::string & format_name_,
|
const std::string & format_name_,
|
||||||
|
@ -28,7 +28,7 @@ namespace ErrorCodes
|
|||||||
}
|
}
|
||||||
|
|
||||||
StorageJoin::StorageJoin(
|
StorageJoin::StorageJoin(
|
||||||
const String & path_,
|
const String & relative_path_,
|
||||||
const String & database_name_,
|
const String & database_name_,
|
||||||
const String & table_name_,
|
const String & table_name_,
|
||||||
const Names & key_names_,
|
const Names & key_names_,
|
||||||
@ -38,8 +38,9 @@ StorageJoin::StorageJoin(
|
|||||||
ASTTableJoin::Strictness strictness_,
|
ASTTableJoin::Strictness strictness_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
bool overwrite)
|
bool overwrite,
|
||||||
: StorageSetOrJoinBase{path_, database_name_, table_name_, columns_, constraints_}
|
const Context & context_)
|
||||||
|
: StorageSetOrJoinBase{relative_path_, database_name_, table_name_, columns_, constraints_, context_}
|
||||||
, key_names(key_names_)
|
, key_names(key_names_)
|
||||||
, use_nulls(use_nulls_)
|
, use_nulls(use_nulls_)
|
||||||
, limits(limits_)
|
, limits(limits_)
|
||||||
@ -162,7 +163,7 @@ void registerStorageJoin(StorageFactory & factory)
|
|||||||
}
|
}
|
||||||
|
|
||||||
return StorageJoin::create(
|
return StorageJoin::create(
|
||||||
args.data_path,
|
args.relative_data_path,
|
||||||
args.database_name,
|
args.database_name,
|
||||||
args.table_name,
|
args.table_name,
|
||||||
key_names,
|
key_names,
|
||||||
@ -172,7 +173,8 @@ void registerStorageJoin(StorageFactory & factory)
|
|||||||
strictness,
|
strictness,
|
||||||
args.columns,
|
args.columns,
|
||||||
args.constraints,
|
args.constraints,
|
||||||
join_any_take_last_row);
|
join_any_take_last_row,
|
||||||
|
args.context);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -60,7 +60,7 @@ private:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
StorageJoin(
|
StorageJoin(
|
||||||
const String & path_,
|
const String & relative_path_,
|
||||||
const String & database_name_,
|
const String & database_name_,
|
||||||
const String & table_name_,
|
const String & table_name_,
|
||||||
const Names & key_names_,
|
const Names & key_names_,
|
||||||
@ -69,7 +69,8 @@ protected:
|
|||||||
ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_,
|
ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
bool overwrite);
|
bool overwrite,
|
||||||
|
const Context & context_);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -211,7 +211,7 @@ Block LogBlockInputStream::readImpl()
|
|||||||
}
|
}
|
||||||
catch (Exception & e)
|
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;
|
throw;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -418,29 +418,30 @@ void LogBlockOutputStream::writeMarks(MarksForColumns && marks)
|
|||||||
}
|
}
|
||||||
|
|
||||||
StorageLog::StorageLog(
|
StorageLog::StorageLog(
|
||||||
const std::string & path_,
|
const std::string & relative_path_,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
size_t max_compress_block_size_)
|
size_t max_compress_block_size_,
|
||||||
: path(path_), table_name(table_name_), database_name(database_name_),
|
const Context & context_)
|
||||||
|
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||||
max_compress_block_size(max_compress_block_size_),
|
max_compress_block_size(max_compress_block_size_),
|
||||||
file_checker(path + escapeForFileName(table_name) + '/' + "sizes.json")
|
file_checker(path + "sizes.json")
|
||||||
{
|
{
|
||||||
setColumns(columns_);
|
setColumns(columns_);
|
||||||
setConstraints(constraints_);
|
setConstraints(constraints_);
|
||||||
|
|
||||||
if (path.empty())
|
if (relative_path_.empty())
|
||||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||||
|
|
||||||
/// create files if they do not exist
|
/// create files if they do not exist
|
||||||
Poco::File(path + escapeForFileName(table_name) + '/').createDirectories();
|
Poco::File(path).createDirectories();
|
||||||
|
|
||||||
for (const auto & column : getColumns().getAllPhysical())
|
for (const auto & column : getColumns().getAllPhysical())
|
||||||
addFiles(column.name, *column.type);
|
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];
|
ColumnData & column_data = files[stream_name];
|
||||||
column_data.column_index = file_count;
|
column_data.column_index = file_count;
|
||||||
column_data.data_file = Poco::File{
|
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);
|
column_names_by_idx.push_back(stream_name);
|
||||||
++file_count;
|
++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);
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||||
|
|
||||||
/// Rename directory with data.
|
/// 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;
|
table_name = new_table_name;
|
||||||
database_name = new_database_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)
|
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 &)
|
void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &)
|
||||||
{
|
{
|
||||||
std::shared_lock<std::shared_mutex> lock(rwlock);
|
std::shared_lock<std::shared_mutex> lock(rwlock);
|
||||||
|
|
||||||
String table_dir = path + escapeForFileName(table_name);
|
String table_dir = path;
|
||||||
|
|
||||||
files.clear();
|
files.clear();
|
||||||
file_count = 0;
|
file_count = 0;
|
||||||
@ -647,8 +649,8 @@ void registerStorageLog(StorageFactory & factory)
|
|||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
return StorageLog::create(
|
return StorageLog::create(
|
||||||
args.data_path, args.database_name, args.table_name, args.columns, args.constraints,
|
args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints,
|
||||||
args.context.getSettings().max_compress_block_size);
|
args.context.getSettings().max_compress_block_size, args.context);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -44,7 +44,7 @@ public:
|
|||||||
|
|
||||||
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
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()}; }
|
Strings getDataPaths() const override { return {fullPath()}; }
|
||||||
|
|
||||||
@ -54,12 +54,13 @@ protected:
|
|||||||
* consisting of the specified columns; Create files if they do not exist.
|
* consisting of the specified columns; Create files if they do not exist.
|
||||||
*/
|
*/
|
||||||
StorageLog(
|
StorageLog(
|
||||||
const std::string & path_,
|
const std::string & relative_path_,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
size_t max_compress_block_size_);
|
size_t max_compress_block_size_,
|
||||||
|
const Context & context_);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
String path;
|
String path;
|
||||||
@ -124,7 +125,7 @@ private:
|
|||||||
*/
|
*/
|
||||||
const Marks & getMarksWithRealRowCount() const;
|
const Marks & getMarksWithRealRowCount() const;
|
||||||
|
|
||||||
std::string getFullPath() const { return path + escapeForFileName(table_name) + '/'; }
|
std::string getFullPath() const { return path; }
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -88,31 +88,33 @@ BlockOutputStreamPtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const
|
|||||||
|
|
||||||
|
|
||||||
StorageSetOrJoinBase::StorageSetOrJoinBase(
|
StorageSetOrJoinBase::StorageSetOrJoinBase(
|
||||||
const String & path_,
|
const String & relative_path_,
|
||||||
const String & database_name_,
|
const String & database_name_,
|
||||||
const String & table_name_,
|
const String & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_)
|
const ConstraintsDescription & constraints_,
|
||||||
|
const Context & context_)
|
||||||
: table_name(table_name_), database_name(database_name_)
|
: table_name(table_name_), database_name(database_name_)
|
||||||
{
|
{
|
||||||
setColumns(columns_);
|
setColumns(columns_);
|
||||||
setConstraints(constraints_);
|
setConstraints(constraints_);
|
||||||
|
|
||||||
if (path_.empty())
|
if (relative_path_.empty())
|
||||||
throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
|
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(
|
StorageSet::StorageSet(
|
||||||
const String & path_,
|
const String & relative_path_,
|
||||||
const String & database_name_,
|
const String & database_name_,
|
||||||
const String & table_name_,
|
const String & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_)
|
const ConstraintsDescription & constraints_,
|
||||||
: StorageSetOrJoinBase{path_, database_name_, table_name_, columns_, constraints_},
|
const Context & context_)
|
||||||
|
: StorageSetOrJoinBase{relative_path_, database_name_, table_name_, columns_, constraints_, context_},
|
||||||
set(std::make_shared<Set>(SizeLimits(), false))
|
set(std::make_shared<Set>(SizeLimits(), false))
|
||||||
{
|
{
|
||||||
Block header = getSampleBlock();
|
Block header = getSampleBlock();
|
||||||
@ -202,10 +204,10 @@ 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_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
|
||||||
{
|
{
|
||||||
/// Rename directory with data.
|
/// 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);
|
Poco::File(path).renameTo(new_path);
|
||||||
|
|
||||||
path = new_path + "/";
|
path = new_path;
|
||||||
table_name = new_table_name;
|
table_name = new_table_name;
|
||||||
database_name = new_database_name;
|
database_name = new_database_name;
|
||||||
}
|
}
|
||||||
@ -220,7 +222,7 @@ void registerStorageSet(StorageFactory & factory)
|
|||||||
"Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)",
|
"Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)",
|
||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
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);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -30,11 +30,12 @@ public:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
StorageSetOrJoinBase(
|
StorageSetOrJoinBase(
|
||||||
const String & path_,
|
const String & relative_path_,
|
||||||
const String & database_name_,
|
const String & database_name_,
|
||||||
const String & table_name_,
|
const String & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_);
|
const ConstraintsDescription & constraints_,
|
||||||
|
const Context & context_);
|
||||||
|
|
||||||
String path;
|
String path;
|
||||||
String table_name;
|
String table_name;
|
||||||
@ -82,11 +83,12 @@ private:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
StorageSet(
|
StorageSet(
|
||||||
const String & path_,
|
const String & relative_path_,
|
||||||
const String & database_name_,
|
const String & database_name_,
|
||||||
const String & table_name_,
|
const String & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_);
|
const ConstraintsDescription & constraints_,
|
||||||
|
const Context & context_);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -121,8 +121,8 @@ private:
|
|||||||
started = true;
|
started = true;
|
||||||
|
|
||||||
data_in.emplace(
|
data_in.emplace(
|
||||||
storage.full_path() + "data.bin", 0, 0,
|
storage.fullPath() + "data.bin", 0, 0,
|
||||||
std::min(static_cast<Poco::File::FileSize>(max_read_buffer_size), Poco::File(storage.full_path() + "data.bin").getSize()));
|
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);
|
block_in.emplace(*data_in, 0, index_begin, index_end);
|
||||||
}
|
}
|
||||||
@ -135,11 +135,11 @@ class StripeLogBlockOutputStream final : public IBlockOutputStream
|
|||||||
public:
|
public:
|
||||||
explicit StripeLogBlockOutputStream(StorageStripeLog & storage_)
|
explicit StripeLogBlockOutputStream(StorageStripeLog & storage_)
|
||||||
: storage(storage_), lock(storage.rwlock),
|
: 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),
|
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),
|
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(
|
StorageStripeLog::StorageStripeLog(
|
||||||
const std::string & path_,
|
const std::string & relative_path_,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
bool attach,
|
bool attach,
|
||||||
size_t max_compress_block_size_)
|
size_t max_compress_block_size_,
|
||||||
: path(path_), table_name(table_name_), database_name(database_name_),
|
const Context & context_)
|
||||||
|
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||||
max_compress_block_size(max_compress_block_size_),
|
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"))
|
log(&Logger::get("StorageStripeLog"))
|
||||||
{
|
{
|
||||||
setColumns(columns_);
|
setColumns(columns_);
|
||||||
setConstraints(constraints_);
|
setConstraints(constraints_);
|
||||||
|
|
||||||
if (path.empty())
|
if (relative_path_.empty())
|
||||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||||
|
|
||||||
String full_path = path + escapeForFileName(table_name) + '/';
|
|
||||||
if (!attach)
|
if (!attach)
|
||||||
{
|
{
|
||||||
/// create files if they do not exist
|
/// create files if they do not exist
|
||||||
if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
|
if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
|
||||||
throwFromErrnoWithPath("Cannot create directory " + full_path, full_path,
|
throwFromErrnoWithPath("Cannot create directory " + path, path,
|
||||||
ErrorCodes::CANNOT_CREATE_DIRECTORY);
|
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);
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||||
|
|
||||||
/// Rename directory with data.
|
/// 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;
|
table_name = new_table_name;
|
||||||
database_name = new_database_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());
|
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)) };
|
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)};
|
std::shared_ptr<const IndexForNativeFormat> index{std::make_shared<IndexForNativeFormat>(index_in, column_names_set)};
|
||||||
|
|
||||||
BlockInputStreams res;
|
BlockInputStreams res;
|
||||||
@ -301,11 +302,11 @@ void StorageStripeLog::truncate(const ASTPtr &, const Context &, TableStructureW
|
|||||||
|
|
||||||
std::shared_lock<std::shared_mutex> lock(rwlock);
|
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.remove(true);
|
||||||
file.createDirectories();
|
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);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
return StorageStripeLog::create(
|
return StorageStripeLog::create(
|
||||||
args.data_path, args.database_name, args.table_name, args.columns, args.constraints,
|
args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints,
|
||||||
args.attach, args.context.getSettings().max_compress_block_size);
|
args.attach, args.context.getSettings().max_compress_block_size, args.context);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -51,9 +51,9 @@ public:
|
|||||||
};
|
};
|
||||||
using Files_t = std::map<String, ColumnData>;
|
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;
|
void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override;
|
||||||
|
|
||||||
@ -71,13 +71,14 @@ private:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
StorageStripeLog(
|
StorageStripeLog(
|
||||||
const std::string & path_,
|
const std::string & relative_path_,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
bool attach,
|
bool attach,
|
||||||
size_t max_compress_block_size_);
|
size_t max_compress_block_size_,
|
||||||
|
const Context & context_);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -323,30 +323,30 @@ void TinyLogBlockOutputStream::write(const Block & block)
|
|||||||
|
|
||||||
|
|
||||||
StorageTinyLog::StorageTinyLog(
|
StorageTinyLog::StorageTinyLog(
|
||||||
const std::string & path_,
|
const std::string & relative_path_,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
bool attach,
|
bool attach,
|
||||||
size_t max_compress_block_size_)
|
size_t max_compress_block_size_,
|
||||||
: path(path_), table_name(table_name_), database_name(database_name_),
|
const Context & context_)
|
||||||
|
: path(context_.getPath() + relative_path_), table_name(table_name_), database_name(database_name_),
|
||||||
max_compress_block_size(max_compress_block_size_),
|
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"))
|
log(&Logger::get("StorageTinyLog"))
|
||||||
{
|
{
|
||||||
setColumns(columns_);
|
setColumns(columns_);
|
||||||
setConstraints(constraints_);
|
setConstraints(constraints_);
|
||||||
|
|
||||||
if (path.empty())
|
if (relative_path_.empty())
|
||||||
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
|
||||||
|
|
||||||
String full_path = path + escapeForFileName(table_name) + '/';
|
|
||||||
if (!attach)
|
if (!attach)
|
||||||
{
|
{
|
||||||
/// create files if they do not exist
|
/// create files if they do not exist
|
||||||
if (0 != mkdir(full_path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
|
if (0 != mkdir(path.c_str(), S_IRWXU | S_IRWXG | S_IRWXO) && errno != EEXIST)
|
||||||
throwFromErrnoWithPath("Cannot create directory " + full_path, full_path,
|
throwFromErrnoWithPath("Cannot create directory " + path, path,
|
||||||
ErrorCodes::CANNOT_CREATE_DIRECTORY);
|
ErrorCodes::CANNOT_CREATE_DIRECTORY);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -369,7 +369,7 @@ void StorageTinyLog::addFiles(const String & column_name, const IDataType & type
|
|||||||
ColumnData column_data;
|
ColumnData column_data;
|
||||||
files.insert(std::make_pair(stream_name, column_data));
|
files.insert(std::make_pair(stream_name, column_data));
|
||||||
files[stream_name].data_file = Poco::File(
|
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);
|
std::unique_lock<std::shared_mutex> lock(rwlock);
|
||||||
|
|
||||||
/// Rename directory with data.
|
/// 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;
|
table_name = new_table_name;
|
||||||
database_name = new_database_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)
|
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);
|
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.remove(true);
|
||||||
file.createDirectories();
|
file.createDirectories();
|
||||||
|
|
||||||
files.clear();
|
files.clear();
|
||||||
file_checker = FileChecker{path + escapeForFileName(table_name) + '/' + "sizes.json"};
|
file_checker = FileChecker{path + "sizes.json"};
|
||||||
|
|
||||||
for (const auto &column : getColumns().getAllPhysical())
|
for (const auto &column : getColumns().getAllPhysical())
|
||||||
addFiles(column.name, *column.type);
|
addFiles(column.name, *column.type);
|
||||||
@ -453,8 +454,8 @@ void registerStorageTinyLog(StorageFactory & factory)
|
|||||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||||
|
|
||||||
return StorageTinyLog::create(
|
return StorageTinyLog::create(
|
||||||
args.data_path, args.database_name, args.table_name, args.columns, args.constraints,
|
args.relative_data_path, args.database_name, args.table_name, args.columns, args.constraints,
|
||||||
args.attach, args.context.getSettings().max_compress_block_size);
|
args.attach, args.context.getSettings().max_compress_block_size, args.context);
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -50,7 +50,7 @@ public:
|
|||||||
};
|
};
|
||||||
using Files_t = std::map<String, ColumnData>;
|
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()}; }
|
Strings getDataPaths() const override { return {fullPath()}; }
|
||||||
|
|
||||||
@ -75,13 +75,14 @@ private:
|
|||||||
|
|
||||||
protected:
|
protected:
|
||||||
StorageTinyLog(
|
StorageTinyLog(
|
||||||
const std::string & path_,
|
const std::string & relative_path_,
|
||||||
const std::string & database_name_,
|
const std::string & database_name_,
|
||||||
const std::string & table_name_,
|
const std::string & table_name_,
|
||||||
const ColumnsDescription & columns_,
|
const ColumnsDescription & columns_,
|
||||||
const ConstraintsDescription & constraints_,
|
const ConstraintsDescription & constraints_,
|
||||||
bool attach,
|
bool attach,
|
||||||
size_t max_compress_block_size_);
|
size_t max_compress_block_size_,
|
||||||
|
const Context & context_);
|
||||||
};
|
};
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -9,6 +9,7 @@ namespace DB
|
|||||||
|
|
||||||
NamesAndTypesList StorageSystemDatabases::getNamesAndTypes()
|
NamesAndTypesList StorageSystemDatabases::getNamesAndTypes()
|
||||||
{
|
{
|
||||||
|
//FIXME data_path
|
||||||
return {
|
return {
|
||||||
{"name", std::make_shared<DataTypeString>()},
|
{"name", std::make_shared<DataTypeString>()},
|
||||||
{"engine", std::make_shared<DataTypeString>()},
|
{"engine", std::make_shared<DataTypeString>()},
|
||||||
|
@ -25,11 +25,14 @@ try
|
|||||||
names_and_types.emplace_back("a", std::make_shared<DataTypeUInt64>());
|
names_and_types.emplace_back("a", std::make_shared<DataTypeUInt64>());
|
||||||
names_and_types.emplace_back("b", std::make_shared<DataTypeUInt8>());
|
names_and_types.emplace_back("b", std::make_shared<DataTypeUInt8>());
|
||||||
|
|
||||||
StoragePtr table = StorageLog::create("./", "test", "test", ColumnsDescription{names_and_types}, ConstraintsDescription{}, 1048576);
|
|
||||||
table->startup();
|
|
||||||
|
|
||||||
auto context = Context::createGlobal();
|
auto context = Context::createGlobal();
|
||||||
context.makeGlobalContext();
|
context.makeGlobalContext();
|
||||||
|
context.setPath("./");
|
||||||
|
|
||||||
|
StoragePtr table = StorageLog::create("./", "test", "test", ColumnsDescription{names_and_types}, ConstraintsDescription{}, 1048576, context);
|
||||||
|
table->startup();
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/// write into it
|
/// write into it
|
||||||
{
|
{
|
||||||
|
Loading…
Reference in New Issue
Block a user