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
|
|
|
{
|
2021-04-10 23:33:54 +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);
|
2020-12-15 14:46:13 +00:00
|
|
|
|
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))
|
2020-12-15 14:46:13 +00:00
|
|
|
{
|
|
|
|
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(
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr local_context,
|
2019-09-25 09:35:56 +00:00
|
|
|
const String & table_name,
|
|
|
|
const StoragePtr & table,
|
|
|
|
const ASTPtr & query)
|
|
|
|
{
|
2021-04-04 09:23:40 +00:00
|
|
|
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);
|
2021-04-10 23:33:54 +00:00
|
|
|
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.
|
2019-11-06 16:05:04 +00:00
|
|
|
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())
|
2019-11-06 16:05:04 +00:00
|
|
|
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(
|
2021-04-10 23:33:54 +00:00
|
|
|
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
|
|
|
{
|
2021-04-04 09:23:40 +00:00
|
|
|
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
|
2021-04-10 23:33:54 +00:00
|
|
|
DatabaseOnDisk::dropTable(local_context, table_name, no_delay);
|
2019-09-25 09:35:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void DatabaseLazy::renameTable(
|
2021-04-10 23:33:54 +00:00
|
|
|
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
|
|
|
{
|
2021-04-04 09:23:40 +00:00
|
|
|
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
|
2021-04-10 23:33:54 +00:00
|
|
|
DatabaseOnDisk::renameTable(local_context, table_name, to_database, to_table_name, exchange, dictionary);
|
2019-09-25 09:35:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-11-06 16:05:04 +00:00
|
|
|
time_t DatabaseLazy::getObjectMetadataModificationTime(const String & table_name) const
|
2019-09-25 09:35:56 +00:00
|
|
|
{
|
2019-11-06 16:05:04 +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(
|
2021-04-10 23:33:54 +00:00
|
|
|
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);
|
|
|
|
}
|
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
bool DatabaseLazy::isTableExist(const String & table_name) const
|
2019-09-25 09:35:56 +00:00
|
|
|
{
|
2021-04-04 09:23:40 +00:00
|
|
|
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
|
2019-11-06 16:05:04 +00:00
|
|
|
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
|
|
|
}
|
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
StoragePtr DatabaseLazy::tryGetTable(const String & table_name) const
|
2019-09-25 09:35:56 +00:00
|
|
|
{
|
2021-04-04 09:23:40 +00:00
|
|
|
SCOPE_EXIT_MEMORY_SAFE({ clearExpiredTables(); });
|
2019-09-26 09:20:46 +00:00
|
|
|
{
|
2019-11-06 16:05:04 +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
|
|
|
}
|
|
|
|
|
2021-08-03 09:18:09 +00:00
|
|
|
DatabaseTablesIteratorPtr DatabaseLazy::getTablesIterator(ContextPtr, const FilterByNameFunction & filter_by_table_name) const
|
2019-09-25 09:35:56 +00:00
|
|
|
{
|
2019-11-06 16:05:04 +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
|
|
|
}
|
|
|
|
|
2020-04-23 16:51:48 +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));
|
2019-11-06 16:05:04 +00:00
|
|
|
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),
|
2019-11-06 16:05:04 +00:00
|
|
|
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));
|
2019-11-06 16:05:04 +00:00
|
|
|
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
|
|
|
{
|
2019-11-06 16:05:04 +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
|
|
|
}
|
|
|
|
|
2019-11-06 16:05:04 +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
|
|
|
{
|
2021-04-04 09:23:40 +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
|
|
|
|
2021-05-09 11:59:49 +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;
|
2021-04-10 23:33:54 +00:00
|
|
|
auto context_copy = Context::createCopy(context); /// some tables can change context, but not LogTables
|
2019-10-10 17:33:01 +00:00
|
|
|
|
2021-04-10 23:33:54 +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
|
|
|
{
|
2019-11-06 16:05:04 +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
|
|
|
}
|
|
|
|
}
|
2020-01-02 07:16:02 +00:00
|
|
|
catch (Exception & e)
|
2019-09-26 09:20:46 +00:00
|
|
|
{
|
2020-01-02 07:16:02 +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
|
|
|
|
{
|
2019-11-06 16:05:04 +00:00
|
|
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
2021-08-03 09:18:09 +00:00
|
|
|
DatabaseLazyIterator::DatabaseLazyIterator(const DatabaseLazy & database_, Strings && table_names_)
|
2021-07-05 19:07:56 +00:00
|
|
|
: 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;
|
2020-04-23 16:51:48 +00:00
|
|
|
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)
|
2020-04-23 16:51:48 +00:00
|
|
|
current_storage = database.tryGetTable(*iterator);
|
2019-09-26 07:16:31 +00:00
|
|
|
return current_storage;
|
2019-09-25 09:35:56 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
}
|