提交 c818beed 编写于 作者: V Vitaliy Lyudvichenko

Make data_path creation for Databases optional. [#CLICKHOUSE-2]

上级 ac17c591
......@@ -171,4 +171,9 @@ void DatabaseDictionary::drop()
/// Additional actions to delete database are not required.
}
String DatabaseDictionary::getDataPath(const Context & context) const
{
return {};
}
}
......@@ -93,6 +93,8 @@ public:
const Context & context,
const String & table_name) const override;
String getDataPath(const Context & context) const override;
void shutdown() override;
void drop() override;
};
......
......@@ -15,11 +15,11 @@ namespace ErrorCodes
DatabasePtr DatabaseFactory::get(
const String & engine_name,
const String & database_name,
const String & path,
const String & metadata_path,
Context & context)
{
if (engine_name == "Ordinary")
return std::make_shared<DatabaseOrdinary>(database_name, path);
return std::make_shared<DatabaseOrdinary>(database_name, metadata_path, context);
else if (engine_name == "Memory")
return std::make_shared<DatabaseMemory>(database_name);
else if (engine_name == "Dictionary")
......
......@@ -13,7 +13,7 @@ public:
static DatabasePtr get(
const String & engine_name,
const String & database_name,
const String & path,
const String & metadata_path,
Context & context);
};
......
......@@ -152,4 +152,9 @@ void DatabaseMemory::drop()
/// Additional actions to delete database are not required.
}
String DatabaseMemory::getDataPath(const Context & context) const
{
return {};
}
}
......@@ -84,6 +84,8 @@ public:
const Context & context,
const String & table_name) const override;
String getDataPath(const Context & context) const override;
void shutdown() override;
void drop() override;
};
......
......@@ -90,10 +90,11 @@ static void loadTable(
}
DatabaseOrdinary::DatabaseOrdinary(
const String & name_, const String & path_)
: DatabaseMemory(name_), path(path_)
DatabaseOrdinary::DatabaseOrdinary(const String & name_, const String & metadata_path, const Context & context)
: DatabaseMemory(name_), metadata_path(metadata_path)
{
data_path = context.getPath() + "data/" + escapeForFileName(name) + "/";
Poco::File(data_path).createDirectory();
}
......@@ -108,7 +109,7 @@ void DatabaseOrdinary::loadTables(
FileNames file_names;
Poco::DirectoryIterator dir_end;
for (Poco::DirectoryIterator dir_it(path); dir_it != dir_end; ++dir_it)
for (Poco::DirectoryIterator dir_it(metadata_path); dir_it != dir_end; ++dir_it)
{
/// For '.svn', '.gitignore' directory and similar.
if (dir_it.name().at(0) == '.')
......@@ -130,7 +131,7 @@ void DatabaseOrdinary::loadTables(
if (endsWith(dir_it.name(), ".sql"))
file_names.push_back(dir_it.name());
else
throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + path,
throw Exception("Incorrect file extension: " + dir_it.name() + " in metadata directory " + metadata_path,
ErrorCodes::INCORRECT_FILE_NAME);
}
......@@ -162,7 +163,7 @@ void DatabaseOrdinary::loadTables(
watch.restart();
}
loadTable(context, path, *this, name, data_path, table, has_force_restore_data_flag);
loadTable(context, metadata_path, *this, name, data_path, table, has_force_restore_data_flag);
}
};
......@@ -269,7 +270,7 @@ void DatabaseOrdinary::createTable(
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
String table_metadata_path = getTableMetadataPath(path, table_name);
String table_metadata_path = getTableMetadataPath(metadata_path, table_name);
String table_metadata_tmp_path = table_metadata_path + ".tmp";
String statement;
......@@ -312,7 +313,7 @@ void DatabaseOrdinary::removeTable(
{
StoragePtr res = detachTable(table_name);
String table_metadata_path = getTableMetadataPath(path, table_name);
String table_metadata_path = getTableMetadataPath(metadata_path, table_name);
try
{
......@@ -374,7 +375,7 @@ void DatabaseOrdinary::renameTable(
throw Exception{e};
}
ASTPtr ast = getCreateQueryImpl(path, table_name);
ASTPtr ast = getCreateQueryImpl(metadata_path, table_name);
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
ast_create_query.table = to_table_name;
......@@ -388,7 +389,7 @@ time_t DatabaseOrdinary::getTableMetadataModificationTime(
const Context & context,
const String & table_name)
{
String table_metadata_path = getTableMetadataPath(path, table_name);
String table_metadata_path = getTableMetadataPath(metadata_path, table_name);
Poco::File meta_file(table_metadata_path);
if (meta_file.exists())
......@@ -406,7 +407,7 @@ ASTPtr DatabaseOrdinary::getCreateQuery(
const Context & context,
const String & table_name) const
{
ASTPtr ast = getCreateQueryImpl(path, table_name);
ASTPtr ast = getCreateQueryImpl(metadata_path, table_name);
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
ast_create_query.attach = false;
......@@ -454,8 +455,8 @@ void DatabaseOrdinary::alterTable(
/// Read the definition of the table and replace the necessary parts with new ones.
String table_name_escaped = escapeForFileName(name);
String table_metadata_tmp_path = path + "/" + table_name_escaped + ".sql.tmp";
String table_metadata_path = path + "/" + table_name_escaped + ".sql";
String table_metadata_tmp_path = metadata_path + "/" + table_name_escaped + ".sql.tmp";
String table_metadata_path = metadata_path + "/" + table_name_escaped + ".sql";
String statement;
{
......@@ -498,4 +499,9 @@ void DatabaseOrdinary::alterTable(
}
}
String DatabaseOrdinary::getDataPath(const Context & context) const
{
return data_path;
}
}
......@@ -13,10 +13,11 @@ namespace DB
class DatabaseOrdinary : public DatabaseMemory
{
protected:
const String path;
const String metadata_path;
String data_path;
public:
DatabaseOrdinary(const String & name_, const String & path_);
DatabaseOrdinary(const String & name_, const String & metadata_path, const Context & context);
String getEngineName() const override { return "Ordinary"; }
......@@ -58,6 +59,8 @@ public:
const Context & context,
const String & table_name) const override;
String getDataPath(const Context & context) const override;
void shutdown() override;
void drop() override;
......
......@@ -129,6 +129,9 @@ public:
const Context & context,
const String & name) const = 0;
/// Returns path for persistent data storage if the database supports it, empty string otherwise
virtual String getDataPath(const Context & context) const = 0;
/// Ask all tables to complete the background threads they are using and delete all table objects.
virtual void shutdown() = 0;
......
......@@ -151,6 +151,13 @@ ClusterPtr Clusters::getCluster(const std::string & cluster_name) const
}
void Clusters::setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster)
{
std::lock_guard<std::mutex> lock(mutex);
impl[cluster_name] = cluster;
}
void Clusters::updateClusters(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name)
{
Poco::Util::AbstractConfiguration::Keys config_keys;
......@@ -180,6 +187,7 @@ Clusters::Impl Clusters::getContainer() const
return impl;
}
/// Implementation of `Cluster` class
Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & cluster_name)
......
......@@ -172,6 +172,7 @@ public:
Clusters & operator=(const Clusters &) = delete;
ClusterPtr getCluster(const std::string & cluster_name) const;
void setCluster(const String & cluster_name, const ClusterPtr & cluster);
void updateClusters(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & config_name);
......
......@@ -878,7 +878,7 @@ ASTPtr Context::getCreateQuery(const String & database_name, const String & tabl
auto lock = getLock();
String db = resolveDatabase(database_name, current_database);
assertTableExists(db, table_name);
assertDatabaseExists(db);
return shared->databases[db]->getCreateQuery(*this, table_name);
}
......@@ -1359,6 +1359,17 @@ void Context::setClustersConfig(const ConfigurationPtr & config, const String &
}
void Context::setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster)
{
std::lock_guard<std::mutex> lock(shared->clusters_mutex);
if (!shared->clusters)
throw Exception("Clusters are not set", ErrorCodes::LOGICAL_ERROR);
shared->clusters->setCluster(cluster_name, cluster);
}
Compiler & Context::getCompiler()
{
auto lock = getLock();
......
......@@ -319,6 +319,8 @@ public:
std::shared_ptr<Cluster> getCluster(const std::string & cluster_name) const;
std::shared_ptr<Cluster> tryGetCluster(const std::string & cluster_name) const;
void setClustersConfig(const ConfigurationPtr & config, const String & config_name = "remote_servers");
/// Sets custom cluster, but doesn't update configuration
void setCluster(const String & cluster_name, const std::shared_ptr<Cluster> & cluster);
Compiler & getCompiler();
QueryLog & getQueryLog();
......
......@@ -103,13 +103,10 @@ BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
String database_name_escaped = escapeForFileName(database_name);
/// Create directories for tables data and metadata.
/// Create directories for tables metadata.
String path = context.getPath();
String data_path = path + "data/" + database_name_escaped + "/";
String metadata_path = path + "metadata/" + database_name_escaped + "/";
Poco::File(metadata_path).createDirectory();
Poco::File(data_path).createDirectory();
DatabasePtr database = DatabaseFactory::get(database_engine_name, database_name, metadata_path, context);
......@@ -463,13 +460,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
String current_database = context.getCurrentDatabase();
String database_name = create.database.empty() ? current_database : create.database;
String database_name_escaped = escapeForFileName(database_name);
String table_name = create.table;
String table_name_escaped = escapeForFileName(table_name);
String data_path = path + "data/" + database_name_escaped + "/";
String metadata_path = path + "metadata/" + database_name_escaped + "/" + table_name_escaped + ".sql";
// If this is a stub ATTACH query, read the query definition from the database
if (create.attach && !create.storage && !create.columns)
{
......@@ -515,9 +508,13 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
{
std::unique_ptr<DDLGuard> guard;
String data_path;
DatabasePtr database;
if (!create.is_temporary)
{
context.assertDatabaseExists(database_name);
database = context.getDatabase(database_name);
data_path = database->getDataPath(context);
/** If the table already exists, and the request specifies IF NOT EXISTS,
* then we allow concurrent CREATE queries (which do nothing).
......@@ -544,7 +541,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
if (create.is_temporary)
context.getSessionContext().addExternalTable(table_name, res);
else
context.getDatabase(database_name)->createTable(context, table_name, res, query_ptr);
database->createTable(context, table_name, res, query_ptr);
}
res->startup();
......
......@@ -135,7 +135,7 @@ void loadMetadataSystem(Context & context)
Poco::File(global_path + "data/" SYSTEM_DATABASE).createDirectories();
Poco::File(global_path + "metadata/" SYSTEM_DATABASE).createDirectories();
auto system_database = std::make_shared<DatabaseOrdinary>(SYSTEM_DATABASE, global_path + "metadata/" SYSTEM_DATABASE);
auto system_database = std::make_shared<DatabaseOrdinary>(SYSTEM_DATABASE, global_path + "metadata/" SYSTEM_DATABASE, context);
context.addDatabase(SYSTEM_DATABASE, system_database);
}
......
......@@ -51,6 +51,8 @@ namespace ErrorCodes
extern const int RESHARDING_NO_WORKER;
extern const int RESHARDING_INVALID_PARAMETERS;
extern const int RESHARDING_INITIATOR_CHECK_FAILED;
extern const int BAD_ARGUMENTS;
extern const int READONLY;
}
......@@ -223,16 +225,24 @@ BlockInputStreams StorageDistributed::read(
BlockOutputStreamPtr StorageDistributed::write(const ASTPtr & query, const Settings & settings)
{
auto cluster = context.getCluster(cluster_name);
if (owned_cluster && context.getApplicationType() != Context::ApplicationType::LOCAL)
throw Exception(
"Method write is not supported by storage " + getName() +
" created via a table function", ErrorCodes::READONLY);
/// TODO: !path.empty() can be replaced by !owned_cluster or !cluster_name.empty() ?
bool write_enabled = !path.empty() && (((cluster->getLocalShardCount() + cluster->getRemoteShardCount()) < 2) || has_sharding_key);
auto cluster = (owned_cluster) ? owned_cluster : context.getCluster(cluster_name);
if (!write_enabled)
throw Exception{
bool is_sharding_key_ok = has_sharding_key || ((cluster->getLocalShardCount() + cluster->getRemoteShardCount()) < 2);
if (!is_sharding_key_ok)
throw Exception(
"Method write is not supported by storage " + getName() +
" with more than one shard and no sharding key provided",
ErrorCodes::STORAGE_REQUIRES_PARAMETER};
ErrorCodes::STORAGE_REQUIRES_PARAMETER);
if (path.empty() && !settings.insert_distributed_sync.value)
throw Exception(
"Data path should be set for storage " + getName() +
" to enable asynchronous inserts", ErrorCodes::BAD_ARGUMENTS);
/// DistributedBlockOutputStream will not own cluster, but will own ConnectionPools of the cluster
return std::make_shared<DistributedBlockOutputStream>(
......
......@@ -607,7 +607,7 @@ StoragePtr StorageFactory::get(
args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(args[0], local_context);
args[1] = evaluateConstantExpressionAsLiteral(args[1], local_context);
String source_database = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<String>();
String source_database = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<String>();
String table_name_regexp = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
return StorageMerge::create(
......@@ -640,7 +640,7 @@ StoragePtr StorageFactory::get(
args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(args[1], local_context);
args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(args[2], local_context);
String remote_database = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
String remote_database = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
String remote_table = static_cast<const ASTLiteral &>(*args[2]).value.safeGet<String>();
const auto & sharding_key = args.size() == 4 ? args[3] : nullptr;
......@@ -686,7 +686,7 @@ StoragePtr StorageFactory::get(
args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(args[1], local_context);
String destination_database = static_cast<const ASTLiteral &>(*args[0]).value.safeGet<String>();
String destination_table = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
String destination_table = static_cast<const ASTLiteral &>(*args[1]).value.safeGet<String>();
UInt64 num_buckets = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), typeid_cast<ASTLiteral &>(*args[2]).value);
......
......@@ -20,6 +20,8 @@ namespace ErrorCodes
extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR;
extern const int CANNOT_SEEK_THROUGH_FILE;
extern const int DATABASE_ACCESS_DENIED;
extern const int INCORRECT_FILE_NAME;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
};
......@@ -49,6 +51,9 @@ StorageFile::StorageFile(
: IStorage(materialized_columns_, alias_columns_, column_defaults_),
table_name(table_name_), format_name(format_name_), columns(columns_), context_global(context_), table_fd(table_fd_)
{
if (columns->empty())
throw Exception("Empty list of columns passed to storage " + getName() + " constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
if (table_fd < 0) /// Will use file
{
use_table_fd = false;
......@@ -61,6 +66,9 @@ StorageFile::StorageFile(
}
else /// Is DB's file
{
if (db_dir_path.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
path = getTablePath(db_dir_path, table_name, format_name);
is_db_table = true;
Poco::File(Poco::Path(path).parent()).createDirectories();
......
......@@ -46,6 +46,7 @@ namespace ErrorCodes
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
extern const int DUPLICATE_COLUMN;
extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT;
extern const int INCORRECT_FILE_NAME;
}
......@@ -579,6 +580,9 @@ StorageLog::StorageLog(
if (columns->empty())
throw Exception("Empty list of columns passed to StorageLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
if (path.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
/// create files if they do not exist
Poco::File(path + escapeForFileName(name) + '/').createDirectories();
......
......@@ -27,6 +27,7 @@ namespace ErrorCodes
extern const int ABORTED;
extern const int BAD_ARGUMENTS;
extern const int INCORRECT_DATA;
extern const int INCORRECT_FILE_NAME;
}
......@@ -59,6 +60,9 @@ StorageMergeTree::StorageMergeTree(
reader(data), writer(data), merger(data, context.getBackgroundPool()),
log(&Logger::get(database_name_ + "." + table_name + " (StorageMergeTree)"))
{
if (path_.empty())
throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
data.loadDataParts(has_force_restore_data_flag);
if (!attach)
......
......@@ -109,6 +109,8 @@ namespace ErrorCodes
extern const int RECEIVED_ERROR_TOO_MANY_REQUESTS;
extern const int TOO_MUCH_FETCHES;
extern const int BAD_DATA_PART_NAME;
extern const int PART_IS_TEMPORARILY_LOCKED;
extern const int INCORRECT_FILE_NAME;
}
......@@ -209,6 +211,9 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
shutdown_event(false), part_check_thread(*this),
log(&Logger::get(database_name + "." + table_name + " (StorageReplicatedMergeTree)"))
{
if (path_.empty())
throw Exception("ReplicatedMergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
replica_path = zookeeper_path + "/replicas/" + replica_name;
......
......@@ -15,6 +15,12 @@ namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_FILE_NAME;
}
class SetOrJoinBlockOutputStream : public IBlockOutputStream
{
public:
......@@ -81,8 +87,12 @@ StorageSetOrJoinBase::StorageSetOrJoinBase(
const NamesAndTypesList & alias_columns_,
const ColumnDefaults & column_defaults_)
: IStorage{materialized_columns_, alias_columns_, column_defaults_},
path(path_ + escapeForFileName(name_) + '/'), name(name_), columns(columns_)
name(name_), columns(columns_)
{
if (path_.empty())
throw Exception("Join and Set storages require data path", ErrorCodes::INCORRECT_FILE_NAME);
path = path_ + escapeForFileName(name_) + '/';
}
......
......@@ -37,6 +37,7 @@ namespace ErrorCodes
{
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
extern const int CANNOT_CREATE_DIRECTORY;
extern const int INCORRECT_FILE_NAME;
}
......@@ -190,6 +191,9 @@ StorageStripeLog::StorageStripeLog(
if (columns->empty())
throw Exception("Empty list of columns passed to StorageStripeLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
if (path.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
String full_path = path + escapeForFileName(name) + '/';
if (!attach)
{
......
......@@ -49,6 +49,7 @@ namespace ErrorCodes
extern const int CANNOT_READ_ALL_DATA;
extern const int DUPLICATE_COLUMN;
extern const int LOGICAL_ERROR;
extern const int INCORRECT_FILE_NAME;
}
......@@ -454,6 +455,9 @@ StorageTinyLog::StorageTinyLog(
if (columns->empty())
throw Exception("Empty list of columns passed to StorageTinyLog constructor", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
if (path.empty())
throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME);
String full_path = path + escapeForFileName(name) + '/';
if (!attach)
{
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册