2020-04-05 12:18:51 +00:00
|
|
|
#pragma once
|
|
|
|
|
2020-05-13 14:44:01 +00:00
|
|
|
#include <Databases/DatabaseAtomic.h>
|
2020-04-05 12:18:51 +00:00
|
|
|
#include <Common/ZooKeeper/ZooKeeper.h>
|
2020-05-13 17:00:47 +00:00
|
|
|
#include <Core/BackgroundSchedulePool.h>
|
2020-07-04 16:32:23 +00:00
|
|
|
#include <DataStreams/BlockIO.h>
|
|
|
|
#include <DataStreams/OneBlockInputStream.h>
|
2020-10-20 16:14:54 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2020-11-05 09:52:23 +00:00
|
|
|
#include <DataStreams/BlockIO.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-05 09:52:23 +00:00
|
|
|
|
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>;
|
2020-11-05 09:52:23 +00:00
|
|
|
|
2021-02-08 09:14:17 +00:00
|
|
|
class Cluster;
|
|
|
|
using ClusterPtr = std::shared_ptr<Cluster>;
|
|
|
|
|
2020-05-27 15:04:10 +00:00
|
|
|
/** DatabaseReplicated engine
|
|
|
|
* supports replication of metadata
|
|
|
|
* via DDL log being written to ZooKeeper
|
|
|
|
* and executed on all of the replicas
|
|
|
|
* for a given database.
|
|
|
|
*
|
|
|
|
* One Clickhouse server can have multiple
|
|
|
|
* replicated databases running and updating
|
|
|
|
* at the same time.
|
|
|
|
*
|
|
|
|
* The engine has two parameters ZooKeeper path and
|
|
|
|
* replica name.
|
|
|
|
* The same ZooKeeper path corresponds to the same
|
2020-06-20 15:39:58 +00:00
|
|
|
* database. Replica names MUST be different for all replicas
|
2020-05-27 15:04:10 +00:00
|
|
|
* of the same database.
|
|
|
|
*
|
|
|
|
* Using this engine, creation of Replicated tables
|
|
|
|
* requires no ZooKeeper path and replica name parameters.
|
|
|
|
* Table's replica name is the same as database replica name.
|
2020-06-20 15:39:58 +00:00
|
|
|
* Table's ZooKeeper path is a concatenation of database
|
2020-05-27 15:04:10 +00:00
|
|
|
* ZooKeeper path, /tables/, and UUID of the table.
|
2020-04-05 12:18:51 +00:00
|
|
|
*/
|
2020-05-13 14:44:01 +00:00
|
|
|
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_,
|
2020-11-27 14:04:03 +00:00
|
|
|
const Context & context);
|
2020-06-27 13:39:41 +00:00
|
|
|
|
2020-11-05 09:52:23 +00:00
|
|
|
~DatabaseReplicated() override;
|
|
|
|
|
2020-11-29 11:45:32 +00:00
|
|
|
void dropTable(const Context &, const String & table_name, bool no_delay) override;
|
|
|
|
void renameTable(const Context & context, const String & table_name, IDatabase & to_database,
|
|
|
|
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,
|
|
|
|
const Context & query_context) override;
|
|
|
|
void commitAlterTable(const StorageID & table_id,
|
|
|
|
const String & table_metadata_tmp_path, const String & table_metadata_path,
|
|
|
|
const String & statement, const Context & query_context) override;
|
2021-02-08 09:46:30 +00:00
|
|
|
void createDictionary(const Context & context,
|
|
|
|
const String & dictionary_name,
|
|
|
|
const ASTPtr & query) override;
|
|
|
|
void removeDictionary(const Context & context, const String & dictionary_name) override;
|
2020-11-29 11:45:32 +00:00
|
|
|
|
2020-06-20 15:39:58 +00:00
|
|
|
void drop(const Context & /*context*/) override;
|
|
|
|
|
2020-05-05 14:16:59 +00:00
|
|
|
String getEngineName() const override { return "Replicated"; }
|
2020-04-05 12:18:51 +00:00
|
|
|
|
2021-02-04 19:41:44 +00:00
|
|
|
BlockIO propose(const ASTPtr & query, const Context & query_context);
|
2020-04-05 12:18:51 +00:00
|
|
|
|
2021-02-04 19:41:44 +00:00
|
|
|
void stopReplication();
|
2020-11-13 18:35:45 +00:00
|
|
|
void shutdown() override;
|
|
|
|
|
2021-01-26 17:51:25 +00:00
|
|
|
void loadStoredObjects(Context & context, bool has_force_restore_data_flag, bool force_attach) override;
|
2020-07-04 16:32:23 +00:00
|
|
|
|
2021-02-08 09:14:17 +00:00
|
|
|
String getFullReplicaName() const;
|
|
|
|
static std::pair<String, String> parseFullReplicaName(const String & name);
|
|
|
|
|
|
|
|
ClusterPtr getCluster() const;
|
2020-11-27 14:04:03 +00:00
|
|
|
|
|
|
|
//FIXME
|
|
|
|
friend struct DatabaseReplicatedTask;
|
|
|
|
friend class DatabaseReplicatedDDLWorker;
|
2020-04-05 12:18:51 +00:00
|
|
|
private:
|
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
|
|
|
|
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);
|
|
|
|
|
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;
|
2020-10-27 09:19:45 +00:00
|
|
|
|
2020-04-05 12:18:51 +00:00
|
|
|
zkutil::ZooKeeperPtr getZooKeeper() const;
|
|
|
|
|
2020-11-27 14:04:03 +00:00
|
|
|
std::unique_ptr<DatabaseReplicatedDDLWorker> ddl_worker;
|
2020-04-05 12:18:51 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|