ClickHouse/src/Databases/TablesLoader.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

94 lines
2.8 KiB
C++
Raw Normal View History

#pragma once
2021-08-31 08:53:48 +00:00
#include <map>
2022-07-20 20:54:43 +00:00
#include <mutex>
2021-08-31 08:53:48 +00:00
#include <unordered_map>
#include <unordered_set>
2022-07-20 20:54:43 +00:00
#include <Core/QualifiedTableName.h>
#include <Core/Types.h>
#include <Databases/LoadingStrictnessLevel.h>
#include <Databases/TablesDependencyGraph.h>
2022-07-20 20:54:43 +00:00
#include <Interpreters/Context_fwd.h>
#include <Parsers/IAST_fwd.h>
#include <Common/Stopwatch.h>
2023-04-26 18:25:39 +00:00
#include <Common/AsyncLoader.h>
2021-08-31 08:53:48 +00:00
namespace Poco
{
2022-06-28 11:29:07 +00:00
class Logger; // NOLINT(cppcoreguidelines-virtual-class-destructor)
2021-08-31 08:53:48 +00:00
}
class AtomicStopwatch;
2021-08-31 08:53:48 +00:00
namespace DB
{
2021-08-31 08:53:48 +00:00
class IDatabase;
using DatabasePtr = std::shared_ptr<IDatabase>;
2021-09-13 19:11:16 +00:00
struct ParsedTableMetadata
{
String path;
ASTPtr ast;
};
using ParsedMetadata = std::map<QualifiedTableName, ParsedTableMetadata>;
struct ParsedTablesMetadata
{
String default_database;
std::mutex mutex;
2021-09-13 19:11:16 +00:00
ParsedMetadata parsed_tables;
2021-09-01 19:42:49 +00:00
/// For logging
size_t total_dictionaries = 0;
};
2021-09-01 19:42:49 +00:00
/// Loads tables (and dictionaries) from specified databases
/// taking into account dependencies between them.
class TablesLoader
{
public:
2021-09-01 19:42:49 +00:00
using Databases = std::map<String, DatabasePtr>;
2022-07-20 20:54:43 +00:00
TablesLoader(ContextMutablePtr global_context_, Databases databases_, LoadingStrictnessLevel strictness_mode_);
2021-09-01 19:42:49 +00:00
TablesLoader() = delete;
/// Create tasks for async loading of all tables in `databases` after specified jobs `load_after`.
2023-05-01 11:56:00 +00:00
[[nodiscard]] LoadTaskPtrs loadTablesAsync(LoadJobSet load_after = {});
/// Create tasks for async startup of all tables in `databases` after specified jobs `startup_after`.
/// Note that for every table startup an extra dependency on that table loading will be added along with `startup_after`.
/// Must be called only after `loadTablesAsync()`.
2023-05-01 11:56:00 +00:00
[[nodiscard]] LoadTaskPtrs startupTablesAsync(LoadJobSet startup_after = {});
/// Set of goal jobs for the whole loading process. Useful for scheduling more tasks after table loading
2023-05-01 11:56:00 +00:00
LoadJobSet goals();
private:
ContextMutablePtr global_context;
Databases databases;
2022-07-20 20:54:43 +00:00
LoadingStrictnessLevel strictness_mode;
2021-09-01 19:42:49 +00:00
Strings databases_to_load;
2021-09-13 19:11:16 +00:00
ParsedTablesMetadata metadata;
TablesDependencyGraph referential_dependencies;
TablesDependencyGraph loading_dependencies;
TablesDependencyGraph all_loading_dependencies;
Poco::Logger * log;
std::atomic<size_t> tables_processed{0};
2021-09-01 19:42:49 +00:00
AtomicStopwatch stopwatch;
2023-04-26 18:25:39 +00:00
AsyncLoader & async_loader;
2023-05-01 11:56:00 +00:00
std::unordered_map<UUID, LoadTaskPtr> load_table; /// table uuid -> load task
LoadTaskPtrs load_tasks; /// Tasks to load all tables
LoadTaskPtrs startup_tasks; /// Tasks to startup all tables and databases after loading
void buildDependencyGraph();
void removeUnresolvableDependencies();
2023-04-26 18:25:39 +00:00
void loadTablesInTopologicalOrder();
void startLoadingTables(ContextMutablePtr load_context, const std::vector<StorageID> & tables_to_load, size_t level);
};
}