ClickHouse/src/Databases/DatabaseLazy.cpp

342 lines
11 KiB
C++
Raw Normal View History

2019-09-25 09:35:56 +00:00
#include <Core/Settings.h>
#include <Databases/DatabaseLazy.h>
2019-10-02 12:58:13 +00:00
#include <Databases/DatabaseOnDisk.h>
2019-09-25 09:35:56 +00:00
#include <Databases/DatabasesCommon.h>
2019-10-03 09:31:59 +00:00
#include <Interpreters/Context.h>
2019-09-25 09:35:56 +00:00
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTCreateQuery.h>
#include <Storages/IStorage.h>
#include <common/logger_useful.h>
2021-06-15 19:55:21 +00:00
#include <common/scope_guard_safe.h>
2019-10-08 08:10:55 +00:00
#include <iomanip>
2021-04-30 19:38:07 +00:00
#include <filesystem>
2019-09-25 09:35:56 +00:00
2021-04-30 19:38:07 +00:00
namespace fs = std::filesystem;
2019-09-25 09:35:56 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int TABLE_ALREADY_EXISTS;
extern const int UNKNOWN_TABLE;
extern const int UNSUPPORTED_METHOD;
extern const int LOGICAL_ERROR;
}
2021-09-04 22:59:44 +00:00
DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, time_t expiration_time_, ContextPtr context_, ASTPtr storage_def_)
: DatabaseOnDisk(name_, metadata_path_, "data/" + escapeForFileName(name_) + "/", "DatabaseLazy (" + name_ + ")", context_, storage_def_)
2019-09-25 12:08:41 +00:00
, expiration_time(expiration_time_)
2019-09-25 09:35:56 +00:00
{
}
2019-10-10 17:33:01 +00:00
void DatabaseLazy::loadStoredObjects(
2021-08-29 18:03:33 +00:00
ContextMutablePtr local_context, bool /* has_force_restore_data_flag */, bool /*force_attach*/, bool /* skip_startup_tables */)
2019-09-25 09:35:56 +00:00
{
iterateMetadataFiles(local_context, [this](const String & file_name)
2019-10-05 10:21:04 +00:00
{
2019-10-03 08:27:43 +00:00
const std::string table_name = file_name.substr(0, file_name.size() - 4);
2021-04-30 19:38:07 +00:00
fs::path detached_permanently_flag = fs::path(getMetadataPath()) / (file_name + detached_suffix);
if (fs::exists(detached_permanently_flag))
{
LOG_DEBUG(log, "Skipping permanently detached table {}.", backQuote(table_name));
return;
}
2020-03-20 12:45:06 +00:00
attachTable(table_name, nullptr, {});
2019-09-26 11:09:26 +00:00
});
2019-09-25 09:35:56 +00:00
}
void DatabaseLazy::createTable(
ContextPtr local_context,
2019-09-25 09:35:56 +00:00
const String & table_name,
const StoragePtr & table,
const ASTPtr & query)
{
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
2019-10-01 10:24:09 +00:00
if (!endsWith(table->getName(), "Log"))
throw Exception("Lazy engine can be used only with *Log tables.", ErrorCodes::UNSUPPORTED_METHOD);
DatabaseOnDisk::createTable(local_context, table_name, table, query);
2019-10-03 14:18:17 +00:00
/// DatabaseOnDisk::createTable renames file, so we need to get new metadata_modification_time.
std::lock_guard lock(mutex);
2019-10-03 14:18:17 +00:00
auto it = tables_cache.find(table_name);
if (it != tables_cache.end())
it->second.metadata_modification_time = DatabaseOnDisk::getObjectMetadataModificationTime(table_name);
2019-09-25 09:35:56 +00:00
}
2020-01-22 11:30:11 +00:00
void DatabaseLazy::dropTable(
ContextPtr local_context,
2020-03-20 00:07:52 +00:00
const String & table_name,
bool no_delay)
2019-09-25 09:35:56 +00:00
{
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
DatabaseOnDisk::dropTable(local_context, table_name, no_delay);
2019-09-25 09:35:56 +00:00
}
void DatabaseLazy::renameTable(
ContextPtr local_context,
2019-09-25 09:35:56 +00:00
const String & table_name,
IDatabase & to_database,
2020-03-31 20:38:05 +00:00
const String & to_table_name,
2020-07-16 21:41:26 +00:00
bool exchange,
bool dictionary)
2019-09-25 09:35:56 +00:00
{
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
DatabaseOnDisk::renameTable(local_context, table_name, to_database, to_table_name, exchange, dictionary);
2019-09-25 09:35:56 +00:00
}
time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name) const
2019-09-25 09:35:56 +00:00
{
std::lock_guard lock(mutex);
2019-09-26 09:20:46 +00:00
auto it = tables_cache.find(table_name);
if (it != tables_cache.end())
return it->second.metadata_modification_time;
2020-07-07 12:11:58 +00:00
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
2019-09-25 09:35:56 +00:00
}
void DatabaseLazy::alterTable(
ContextPtr /* context */,
2020-03-20 12:45:06 +00:00
const StorageID & /*table_id*/,
2019-12-26 18:17:05 +00:00
const StorageInMemoryMetadata & /* metadata */)
2019-09-25 09:35:56 +00:00
{
2019-12-26 13:35:36 +00:00
clearExpiredTables();
2019-09-25 09:35:56 +00:00
throw Exception("ALTER query is not supported for Lazy database.", ErrorCodes::UNSUPPORTED_METHOD);
}
bool DatabaseLazy::isTableExist(const String & table_name) const
2019-09-25 09:35:56 +00:00
{
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
std::lock_guard lock(mutex);
2019-09-26 09:20:46 +00:00
return tables_cache.find(table_name) != tables_cache.end();
2019-09-25 09:35:56 +00:00
}
StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const
2019-09-25 09:35:56 +00:00
{
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
2019-09-26 09:20:46 +00:00
{
std::lock_guard lock(mutex);
2019-09-26 09:20:46 +00:00
auto it = tables_cache.find(table_name);
2019-10-01 10:24:09 +00:00
if (it == tables_cache.end())
2020-01-15 11:52:30 +00:00
return {};
2019-10-01 10:24:09 +00:00
if (it->second.table)
2019-09-26 12:34:58 +00:00
{
2019-10-03 09:20:21 +00:00
cache_expiration_queue.erase(it->second.expiration_iterator);
2019-09-26 12:34:58 +00:00
it->second.last_touched = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
2019-10-03 09:20:21 +00:00
it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), it->second.last_touched, table_name);
2019-09-26 12:34:58 +00:00
2019-09-26 09:20:46 +00:00
return it->second.table;
2019-09-26 12:34:58 +00:00
}
2019-09-26 09:20:46 +00:00
}
2019-10-05 10:21:04 +00:00
2020-04-01 22:41:29 +00:00
return loadTable(table_name);
2019-09-25 09:35:56 +00:00
}
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const
2019-09-25 09:35:56 +00:00
{
std::lock_guard lock(mutex);
2019-09-25 12:08:41 +00:00
Strings filtered_tables;
2019-09-26 11:09:26 +00:00
for (const auto & [table_name, cached_table] : tables_cache)
{
2019-10-01 10:48:19 +00:00
if (!filter_by_table_name || filter_by_table_name(table_name))
2019-09-26 11:09:26 +00:00
filtered_tables.push_back(table_name);
}
2019-09-25 12:08:41 +00:00
std::sort(filtered_tables.begin(), filtered_tables.end());
2020-04-01 22:41:29 +00:00
return std::make_unique<DatabaseLazyIterator>(*this, std::move(filtered_tables));
2019-09-25 09:35:56 +00:00
}
bool DatabaseLazy::empty() const
2019-09-25 09:35:56 +00:00
{
2019-09-26 09:20:46 +00:00
return tables_cache.empty();
2019-09-25 09:35:56 +00:00
}
2019-11-11 11:34:03 +00:00
void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table, const String &)
2019-09-26 07:16:31 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Attach table {}.", backQuote(table_name));
std::lock_guard lock(mutex);
2019-09-26 12:34:58 +00:00
time_t current_time = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
2019-10-05 10:21:04 +00:00
2019-10-03 09:20:21 +00:00
auto [it, inserted] = tables_cache.emplace(std::piecewise_construct,
2019-09-26 09:20:46 +00:00
std::forward_as_tuple(table_name),
std::forward_as_tuple(table, current_time, DatabaseOnDisk::getObjectMetadataModificationTime(table_name)));
2019-10-03 09:20:21 +00:00
if (!inserted)
2020-07-07 12:11:58 +00:00
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
2019-10-03 09:20:21 +00:00
it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name);
2019-09-26 07:16:31 +00:00
}
StoragePtr DatabaseLazy::detachTable(const String & table_name)
{
StoragePtr res;
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Detach table {}.", backQuote(table_name));
std::lock_guard lock(mutex);
2019-09-26 07:16:31 +00:00
auto it = tables_cache.find(table_name);
if (it == tables_cache.end())
2020-07-07 12:11:58 +00:00
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
2019-09-26 09:20:46 +00:00
res = it->second.table;
2019-10-03 12:49:22 +00:00
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
2019-09-26 07:16:31 +00:00
tables_cache.erase(it);
}
return res;
}
void DatabaseLazy::shutdown()
{
2019-09-26 09:20:46 +00:00
TablesCache tables_snapshot;
2019-09-26 07:16:31 +00:00
{
std::lock_guard lock(mutex);
2019-09-26 07:16:31 +00:00
tables_snapshot = tables_cache;
}
for (const auto & kv : tables_snapshot)
{
2019-10-01 10:24:09 +00:00
if (kv.second.table)
2021-05-13 07:03:00 +00:00
kv.second.table->flushAndShutdown();
2019-09-26 07:16:31 +00:00
}
std::lock_guard lock(mutex);
2019-09-26 07:16:31 +00:00
tables_cache.clear();
}
DatabaseLazy::~DatabaseLazy()
{
try
{
shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
2020-04-01 22:41:29 +00:00
StoragePtr DatabaseLazy::loadTable(const String & table_name) const
2019-09-26 09:20:46 +00:00
{
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
2019-09-26 12:39:37 +00:00
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Load table {} to cache.", backQuote(table_name));
2019-10-01 10:24:09 +00:00
const String table_metadata_path = fs::path(getMetadataPath()) / (escapeForFileName(table_name) + ".sql");
2019-09-26 09:20:46 +00:00
try
{
StoragePtr table;
auto context_copy = Context::createCopy(context); /// some tables can change context, but not LogTables
2019-10-10 17:33:01 +00:00
auto ast = parseQueryFromMetadata(log, getContext(), table_metadata_path, /*throw_on_error*/ true, /*remove_empty*/false);
2019-10-10 17:33:01 +00:00
if (ast)
2019-11-01 12:47:55 +00:00
{
2020-04-22 05:39:31 +00:00
const auto & ast_create = ast->as<const ASTCreateQuery &>();
2019-12-25 16:13:48 +00:00
String table_data_path_relative = getTableDataPath(ast_create);
2020-07-07 12:11:58 +00:00
table = createTableFromAST(ast_create, getDatabaseName(), table_data_path_relative, context_copy, false).second;
2019-11-01 12:47:55 +00:00
}
2019-10-10 17:33:01 +00:00
if (!ast || !endsWith(table->getName(), "Log"))
2019-09-26 09:20:46 +00:00
throw Exception("Only *Log tables can be used with Lazy database engine.", ErrorCodes::LOGICAL_ERROR);
2021-08-29 18:03:33 +00:00
table->startup();
2019-09-26 09:20:46 +00:00
{
std::lock_guard lock(mutex);
2019-09-26 11:09:26 +00:00
auto it = tables_cache.find(table_name);
2019-09-26 12:34:58 +00:00
if (it == tables_cache.end())
2020-07-07 12:11:58 +00:00
throw Exception("Table " + backQuote(database_name) + "." + backQuote(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
2019-09-26 12:34:58 +00:00
2019-10-03 12:49:22 +00:00
if (it->second.expiration_iterator != cache_expiration_queue.end())
cache_expiration_queue.erase(it->second.expiration_iterator);
2019-09-26 12:34:58 +00:00
it->second.last_touched = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
2019-10-03 09:20:21 +00:00
it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), it->second.last_touched, table_name);
2019-09-26 12:34:58 +00:00
return it->second.table = table;
2019-09-26 09:20:46 +00:00
}
}
catch (Exception & e)
2019-09-26 09:20:46 +00:00
{
e.addMessage("Cannot create table from metadata file " + table_metadata_path);
throw;
2019-09-26 09:20:46 +00:00
}
}
2019-09-26 07:16:31 +00:00
void DatabaseLazy::clearExpiredTables() const
{
std::lock_guard lock(mutex);
2019-10-03 09:20:21 +00:00
auto time_now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now());
2019-10-03 12:49:22 +00:00
CacheExpirationQueue expired_tables;
auto expired_it = cache_expiration_queue.begin();
while (expired_it != cache_expiration_queue.end() && (time_now - expired_it->last_touched) >= expiration_time)
++expired_it;
expired_tables.splice(expired_tables.end(), cache_expiration_queue, cache_expiration_queue.begin(), expired_it);
CacheExpirationQueue busy_tables;
while (!expired_tables.empty())
2019-09-26 12:34:58 +00:00
{
2019-10-03 12:49:22 +00:00
String table_name = expired_tables.front().table_name;
2019-10-03 09:20:21 +00:00
auto it = tables_cache.find(table_name);
2019-10-02 12:58:13 +00:00
2019-10-03 13:40:53 +00:00
if (!it->second.table || it->second.table.unique())
2019-10-03 12:49:22 +00:00
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Drop table {} from cache.", backQuote(it->first));
2019-10-01 10:24:09 +00:00
it->second.table.reset();
2019-10-03 13:40:53 +00:00
expired_tables.erase(it->second.expiration_iterator);
2019-10-03 12:49:22 +00:00
it->second.expiration_iterator = cache_expiration_queue.end();
}
else
{
2020-05-23 22:24:01 +00:00
LOG_DEBUG(log, "Table {} is busy.", backQuote(it->first));
2019-10-03 12:49:22 +00:00
busy_tables.splice(busy_tables.end(), expired_tables, it->second.expiration_iterator);
}
2019-09-26 12:34:58 +00:00
}
2019-10-03 12:49:22 +00:00
cache_expiration_queue.splice(cache_expiration_queue.begin(), busy_tables, busy_tables.begin(), busy_tables.end());
2019-09-26 07:16:31 +00:00
}
DatabaseLazyIterator::DatabaseLazyIterator(const DatabaseLazy & database_, Strings && table_names_)
: IDatabaseTablesIterator(database_.database_name)
, database(database_)
2019-09-25 12:08:41 +00:00
, table_names(std::move(table_names_))
, iterator(table_names.begin())
2019-09-26 07:16:31 +00:00
, current_storage(nullptr)
2019-09-25 12:08:41 +00:00
{
}
2019-09-25 09:35:56 +00:00
void DatabaseLazyIterator::next()
{
2019-09-26 07:16:31 +00:00
current_storage.reset();
2019-10-01 10:48:19 +00:00
++iterator;
while (isValid() && !database.isTableExist(*iterator))
2019-09-26 11:09:26 +00:00
++iterator;
2019-09-25 09:35:56 +00:00
}
bool DatabaseLazyIterator::isValid() const
{
2019-09-25 12:08:41 +00:00
return iterator != table_names.end();
2019-09-25 09:35:56 +00:00
}
const String & DatabaseLazyIterator::name() const
{
2019-09-25 12:08:41 +00:00
return *iterator;
2019-09-25 09:35:56 +00:00
}
2019-09-26 07:16:31 +00:00
const StoragePtr & DatabaseLazyIterator::table() const
2019-09-25 09:35:56 +00:00
{
2019-09-26 07:16:31 +00:00
if (!current_storage)
current_storage = database.tryGetTable(*iterator);
2019-09-26 07:16:31 +00:00
return current_storage;
2019-09-25 09:35:56 +00:00
}
}