ClickHouse/src/Databases/DatabaseReplicated.h

102 lines
4.1 KiB
C++
Raw Normal View History

2020-04-05 12:18:51 +00:00
#pragma once
#include <Databases/DatabaseAtomic.h>
2021-02-19 23:41:58 +00:00
#include <Databases/DatabaseReplicatedSettings.h>
2020-04-05 12:18:51 +00:00
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Core/BackgroundSchedulePool.h>
#include <DataStreams/BlockIO.h>
#include <Interpreters/Context.h>
2020-04-05 12:18:51 +00:00
2020-06-27 13:39:41 +00:00
2020-04-05 12:18:51 +00:00
namespace DB
{
2020-11-27 14:04:03 +00:00
class DatabaseReplicatedDDLWorker;
2020-11-19 10:34:45 +00:00
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
2021-02-08 09:14:17 +00:00
class Cluster;
using ClusterPtr = std::shared_ptr<Cluster>;
class DatabaseReplicated : public DatabaseAtomic
2020-04-05 12:18:51 +00:00
{
public:
2020-10-27 09:19:45 +00:00
DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid,
const String & zookeeper_path_, const String & shard_name_, const String & replica_name_,
2021-02-19 23:41:58 +00:00
DatabaseReplicatedSettings db_settings_,
ContextPtr context);
2020-06-27 13:39:41 +00:00
~DatabaseReplicated() override;
2021-02-15 20:00:59 +00:00
String getEngineName() const override { return "Replicated"; }
2021-02-19 23:41:58 +00:00
/// If current query is initial, then the following methods add metadata updating ZooKeeper operations to current ZooKeeperMetadataTransaction.
void dropTable(ContextPtr, const String & table_name, bool no_delay) override;
void renameTable(ContextPtr context, const String & table_name, IDatabase & to_database,
2020-11-29 11:45:32 +00:00
const String & to_table_name, bool exchange, bool dictionary) override;
void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table,
const String & table_metadata_tmp_path, const String & table_metadata_path,
ContextPtr query_context) override;
2020-11-29 11:45:32 +00:00
void commitAlterTable(const StorageID & table_id,
const String & table_metadata_tmp_path, const String & table_metadata_path,
const String & statement, ContextPtr query_context) override;
void detachTablePermanently(ContextPtr context, const String & table_name) override;
void removeDetachedPermanentlyFlag(ContextPtr context, const String & table_name, const String & table_metadata_path, bool attach) const override;
2020-11-29 11:45:32 +00:00
2021-02-15 20:00:59 +00:00
/// Try to execute DLL query on current host as initial query. If query is succeed,
/// then it will be executed on all replicas.
BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context);
2020-04-05 12:18:51 +00:00
2021-02-04 19:41:44 +00:00
void stopReplication();
2021-02-08 09:14:17 +00:00
String getFullReplicaName() const;
static std::pair<String, String> parseFullReplicaName(const String & name);
2021-02-15 20:00:59 +00:00
/// Returns cluster consisting of database replicas
2021-02-08 09:14:17 +00:00
ClusterPtr getCluster() const;
2020-11-27 14:04:03 +00:00
void drop(ContextPtr /*context*/) override;
2021-02-15 20:00:59 +00:00
2021-09-13 19:11:16 +00:00
void loadStoredObjects(ContextMutablePtr context, bool force_restore, bool force_attach, bool skip_startup_tables) override;
2021-08-31 08:53:48 +00:00
2021-09-13 19:11:16 +00:00
void beforeLoadingMetadata(ContextMutablePtr context, bool force_restore, bool force_attach) override;
2021-08-31 08:53:48 +00:00
void startupTables(ThreadPool & thread_pool, bool force_restore, bool force_attach) override;
2021-02-15 20:00:59 +00:00
void shutdown() override;
2020-11-27 14:04:03 +00:00
friend struct DatabaseReplicatedTask;
friend class DatabaseReplicatedDDLWorker;
2020-04-05 12:18:51 +00:00
private:
2021-02-19 23:41:58 +00:00
void tryConnectToZooKeeperAndInitDatabase(bool force_attach);
2020-11-19 10:34:45 +00:00
bool createDatabaseNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper);
void createReplicaNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper);
2020-05-11 12:55:17 +00:00
void checkQueryValid(const ASTPtr & query, ContextPtr query_context) const;
2021-03-16 20:01:20 +00:00
2021-02-09 15:14:20 +00:00
void recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 max_log_ptr);
std::map<String, String> tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr);
2020-11-19 10:34:45 +00:00
2020-11-27 14:04:03 +00:00
ASTPtr parseQueryFromMetadataInZooKeeper(const String & node_name, const String & query);
String readMetadataFile(const String & table_name) const;
2020-11-27 14:04:03 +00:00
ClusterPtr getClusterImpl() const;
void setCluster(ClusterPtr && new_cluster);
2021-05-31 13:31:03 +00:00
void createEmptyLogEntry(const ZooKeeperPtr & current_zookeeper);
2020-10-27 09:19:45 +00:00
String zookeeper_path;
String shard_name;
String replica_name;
2020-11-13 18:35:45 +00:00
String replica_path;
2021-02-19 23:41:58 +00:00
DatabaseReplicatedSettings db_settings;
2020-10-27 09:19:45 +00:00
2020-04-05 12:18:51 +00:00
zkutil::ZooKeeperPtr getZooKeeper() const;
2021-02-15 00:04:46 +00:00
std::atomic_bool is_readonly = true;
2020-11-27 14:04:03 +00:00
std::unique_ptr<DatabaseReplicatedDDLWorker> ddl_worker;
mutable ClusterPtr cluster;
2020-04-05 12:18:51 +00:00
};
}