mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
extract common base for DatabaseMemory and DatabaseOrdinary [#CLICKHOUSE-3664]
This commit is contained in:
parent
cf3c24d07d
commit
10b669a833
@ -8,75 +8,22 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
|
||||
}
|
||||
|
||||
DatabaseMemory::DatabaseMemory(String name_)
|
||||
: DatabaseWithOwnTablesBase(std::move(name_))
|
||||
, log(&Logger::get("DatabaseMemory(" + name + ")"))
|
||||
{}
|
||||
|
||||
void DatabaseMemory::loadTables(
|
||||
Context & /*context*/,
|
||||
ThreadPool * /*thread_pool*/,
|
||||
bool /*has_force_restore_data_flag*/)
|
||||
{
|
||||
log = &Logger::get("DatabaseMemory(" + name + ")");
|
||||
|
||||
/// Nothing to load.
|
||||
}
|
||||
|
||||
bool DatabaseMemory::isTableExist(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseMemory::tryGetTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
return {};
|
||||
return it->second;
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseMemory::getIterator(const Context & /*context*/)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return std::make_unique<DatabaseSnapshotIterator>(tables);
|
||||
}
|
||||
|
||||
bool DatabaseMemory::empty(const Context & /*context*/) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return tables.empty();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseMemory::detachTable(const String & table_name)
|
||||
{
|
||||
StoragePtr res;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
res = it->second;
|
||||
tables.erase(it);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void DatabaseMemory::attachTable(const String & table_name, const StoragePtr & table)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (!tables.emplace(table_name, table).second)
|
||||
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
void DatabaseMemory::createTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name,
|
||||
@ -125,26 +72,6 @@ ASTPtr DatabaseMemory::getCreateQuery(
|
||||
throw Exception("There is no CREATE TABLE query for DatabaseMemory tables", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY);
|
||||
}
|
||||
|
||||
void DatabaseMemory::shutdown()
|
||||
{
|
||||
/// You can not hold a lock during shutdown.
|
||||
/// Because inside `shutdown` function tables can work with database, and mutex is not recursive.
|
||||
|
||||
Tables tables_snapshot;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
tables_snapshot = tables;
|
||||
}
|
||||
|
||||
for (const auto & kv: tables_snapshot)
|
||||
{
|
||||
kv.second->shutdown();
|
||||
}
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
tables.clear();
|
||||
}
|
||||
|
||||
void DatabaseMemory::drop()
|
||||
{
|
||||
/// Additional actions to delete database are not required.
|
||||
|
@ -1,8 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <mutex>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
@ -16,18 +14,10 @@ namespace DB
|
||||
* All tables are created by calling code.
|
||||
* TODO: Maybe DatabaseRuntime is more suitable class name.
|
||||
*/
|
||||
class DatabaseMemory : public IDatabase
|
||||
class DatabaseMemory : public DatabaseWithOwnTablesBase
|
||||
{
|
||||
protected:
|
||||
const String name;
|
||||
mutable std::mutex mutex;
|
||||
Tables tables;
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
public:
|
||||
|
||||
DatabaseMemory(const String & name_) : name(name_) {}
|
||||
DatabaseMemory(String name_);
|
||||
|
||||
String getEngineName() const override { return "Memory"; }
|
||||
|
||||
@ -36,18 +26,6 @@ public:
|
||||
ThreadPool * thread_pool,
|
||||
bool has_force_restore_data_flag) override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context) override;
|
||||
|
||||
bool isTableExist(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
void createTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -58,9 +36,6 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) override;
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
void renameTable(
|
||||
const Context & context,
|
||||
const String & table_name,
|
||||
@ -81,8 +56,10 @@ public:
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
void shutdown() override;
|
||||
void drop() override;
|
||||
|
||||
private:
|
||||
Poco::Logger * log;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -93,8 +93,11 @@ static void loadTable(
|
||||
}
|
||||
|
||||
|
||||
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_) + "/")
|
||||
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) + "/")
|
||||
, log(&Logger::get("DatabaseOrdinary (" + name + ")"))
|
||||
{
|
||||
Poco::File(data_path).createDirectories();
|
||||
}
|
||||
@ -105,8 +108,6 @@ void DatabaseOrdinary::loadTables(
|
||||
ThreadPool * thread_pool,
|
||||
bool has_force_restore_data_flag)
|
||||
{
|
||||
log = &Logger::get("DatabaseOrdinary (" + name + ")");
|
||||
|
||||
using FileNames = std::vector<std::string>;
|
||||
FileNames file_names;
|
||||
|
||||
@ -457,7 +458,6 @@ void DatabaseOrdinary::drop()
|
||||
/// No additional removal actions are required.
|
||||
}
|
||||
|
||||
|
||||
void DatabaseOrdinary::alterTable(
|
||||
const Context & context,
|
||||
const String & name,
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Databases/DatabasesCommon.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -10,14 +10,10 @@ namespace DB
|
||||
* It stores tables list in filesystem using list of .sql files,
|
||||
* that contain declaration of table represented by SQL ATTACH TABLE query.
|
||||
*/
|
||||
class DatabaseOrdinary : public DatabaseMemory
|
||||
class DatabaseOrdinary : public DatabaseWithOwnTablesBase
|
||||
{
|
||||
protected:
|
||||
const String metadata_path;
|
||||
const String data_path;
|
||||
|
||||
public:
|
||||
DatabaseOrdinary(const String & name_, const String & metadata_path_, const Context & context);
|
||||
DatabaseOrdinary(String name_, const String & metadata_path_, const Context & context);
|
||||
|
||||
String getEngineName() const override { return "Ordinary"; }
|
||||
|
||||
@ -64,6 +60,10 @@ public:
|
||||
void drop() override;
|
||||
|
||||
private:
|
||||
const String metadata_path;
|
||||
const String data_path;
|
||||
Poco::Logger * log;
|
||||
|
||||
void startupTables(ThreadPool * thread_pool);
|
||||
};
|
||||
|
||||
|
@ -16,6 +16,9 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
||||
extern const int TABLE_ALREADY_EXISTS;
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -80,4 +83,78 @@ std::pair<String, StoragePtr> createTableFromDefinition(
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
bool DatabaseWithOwnTablesBase::isTableExist(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return tables.find(table_name) != tables.end();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::tryGetTable(
|
||||
const Context & /*context*/,
|
||||
const String & table_name) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
return {};
|
||||
return it->second;
|
||||
}
|
||||
|
||||
DatabaseIteratorPtr DatabaseWithOwnTablesBase::getIterator(const Context & /*context*/)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return std::make_unique<DatabaseSnapshotIterator>(tables);
|
||||
}
|
||||
|
||||
bool DatabaseWithOwnTablesBase::empty(const Context & /*context*/) const
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
return tables.empty();
|
||||
}
|
||||
|
||||
StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name)
|
||||
{
|
||||
StoragePtr res;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
auto it = tables.find(table_name);
|
||||
if (it == tables.end())
|
||||
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
res = it->second;
|
||||
tables.erase(it);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table)
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
if (!tables.emplace(table_name, table).second)
|
||||
throw Exception("Table " + name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
|
||||
void DatabaseWithOwnTablesBase::shutdown()
|
||||
{
|
||||
/// You can not hold a lock during shutdown.
|
||||
/// Because inside `shutdown` function tables can work with database, and mutex is not recursive.
|
||||
|
||||
Tables tables_snapshot;
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
tables_snapshot = tables;
|
||||
}
|
||||
|
||||
for (const auto & kv: tables_snapshot)
|
||||
{
|
||||
kv.second->shutdown();
|
||||
}
|
||||
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
tables.clear();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -70,4 +70,36 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
/// A base class for databases that manage their own list of tables.
|
||||
class DatabaseWithOwnTablesBase : public IDatabase
|
||||
{
|
||||
public:
|
||||
bool isTableExist(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
StoragePtr tryGetTable(
|
||||
const Context & context,
|
||||
const String & table_name) const override;
|
||||
|
||||
bool empty(const Context & context) const override;
|
||||
|
||||
|
||||
void attachTable(const String & table_name, const StoragePtr & table) override;
|
||||
|
||||
StoragePtr detachTable(const String & table_name) override;
|
||||
|
||||
DatabaseIteratorPtr getIterator(const Context & context) override;
|
||||
|
||||
void shutdown() override;
|
||||
|
||||
protected:
|
||||
String name;
|
||||
|
||||
mutable std::mutex mutex;
|
||||
Tables tables;
|
||||
|
||||
DatabaseWithOwnTablesBase(String name_) : name(std::move(name_)) { }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -6,7 +6,7 @@
|
||||
#include <Poco/String.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Poco/NullChannel.h>
|
||||
#include <Databases/DatabaseOrdinary.h>
|
||||
#include <Databases/DatabaseMemory.h>
|
||||
#include <Storages/System/attachSystemTables.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
|
Loading…
Reference in New Issue
Block a user