ClickHouse/src/Interpreters/DDLTask.h

197 lines
5.2 KiB
C++
Raw Normal View History

2020-11-03 13:47:26 +00:00
#pragma once
#include <Core/Types.h>
#include <Interpreters/Cluster.h>
2020-11-20 16:06:27 +00:00
#include <Common/ZooKeeper/Types.h>
2020-11-03 13:47:26 +00:00
2020-11-27 14:04:03 +00:00
namespace Poco
{
class Logger;
}
2020-11-03 13:47:26 +00:00
namespace zkutil
{
class ZooKeeper;
}
2020-11-03 13:47:26 +00:00
namespace DB
{
class ASTQueryWithOnCluster;
2020-11-20 16:06:27 +00:00
using ZooKeeperPtr = std::shared_ptr<zkutil::ZooKeeper>;
2020-11-27 14:04:03 +00:00
class DatabaseReplicated;
2020-11-03 13:47:26 +00:00
2021-02-19 23:41:58 +00:00
class ZooKeeperMetadataTransaction;
using ZooKeeperMetadataTransactionPtr = std::shared_ptr<ZooKeeperMetadataTransaction>;
2020-12-04 20:12:32 +00:00
2020-11-03 13:47:26 +00:00
struct HostID
{
String host_name;
UInt16 port;
HostID() = default;
explicit HostID(const Cluster::Address & address)
: host_name(address.host_name), port(address.port) {}
static HostID fromString(const String & host_port_str);
String toString() const
{
return Cluster::Address::toString(host_name, port);
}
String readableString() const
{
return host_name + ":" + DB::toString(port);
}
bool isLocalAddress(UInt16 clickhouse_port) const;
static String applyToString(const HostID & host_id)
{
return host_id.toString();
}
};
struct DDLLogEntry
{
UInt64 version = 1;
2020-11-03 13:47:26 +00:00
String query;
std::vector<HostID> hosts;
String initiator; // optional
std::optional<SettingsChanges> settings;
2020-11-03 13:47:26 +00:00
void setSettingsIfRequired(const Context & context);
2020-11-03 13:47:26 +00:00
String toString() const;
void parse(const String & data);
void assertVersion() const;
2020-11-03 13:47:26 +00:00
};
2020-11-27 14:04:03 +00:00
struct DDLTaskBase
{
const String entry_name;
const String entry_path;
DDLLogEntry entry;
String host_id_str;
ASTPtr query;
bool is_initial_query = false;
2020-11-27 14:04:03 +00:00
bool is_circular_replicated = false;
bool execute_on_leader = false;
2020-11-03 13:47:26 +00:00
2020-12-04 20:12:32 +00:00
Coordination::Requests ops;
2020-11-27 14:04:03 +00:00
ExecutionStatus execution_status;
bool was_executed = false;
2021-01-26 17:51:25 +00:00
std::atomic_bool completely_processed = false;
2020-11-29 11:45:32 +00:00
DDLTaskBase(const String & name, const String & path) : entry_name(name), entry_path(path) {}
DDLTaskBase(const DDLTaskBase &) = delete;
2020-11-29 11:45:32 +00:00
virtual ~DDLTaskBase() = default;
void parseQueryFromEntry(const Context & context);
2020-11-27 14:04:03 +00:00
virtual String getShardID() const = 0;
2021-02-19 23:41:58 +00:00
virtual std::unique_ptr<Context> makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper);
2020-11-27 14:04:03 +00:00
inline String getActiveNodePath() const { return entry_path + "/active/" + host_id_str; }
inline String getFinishedNodePath() const { return entry_path + "/finished/" + host_id_str; }
inline String getShardNodePath() const { return entry_path + "/shards/" + getShardID(); }
2021-02-09 15:14:20 +00:00
static String getLogEntryName(UInt32 log_entry_number);
static UInt32 getLogEntryNumber(const String & log_entry_name);
2020-11-27 14:04:03 +00:00
};
struct DDLTask : public DDLTaskBase
2020-11-03 13:47:26 +00:00
{
2020-11-27 14:04:03 +00:00
DDLTask(const String & name, const String & path) : DDLTaskBase(name, path) {}
bool findCurrentHostID(const Context & global_context, Poco::Logger * log);
void setClusterInfo(const Context & context, Poco::Logger * log);
2020-11-03 13:47:26 +00:00
2020-11-27 14:04:03 +00:00
String getShardID() const override;
private:
bool tryFindHostInCluster();
bool tryFindHostInClusterViaResolving(const Context & context);
HostID host_id;
2020-11-03 13:47:26 +00:00
String cluster_name;
ClusterPtr cluster;
Cluster::Address address_in_cluster;
size_t host_shard_num;
size_t host_replica_num;
2020-11-27 14:04:03 +00:00
};
2020-11-03 13:47:26 +00:00
2020-11-27 14:04:03 +00:00
struct DatabaseReplicatedTask : public DDLTaskBase
{
DatabaseReplicatedTask(const String & name, const String & path, DatabaseReplicated * database_);
2020-11-03 13:47:26 +00:00
2020-11-27 14:04:03 +00:00
String getShardID() const override;
2021-02-19 23:41:58 +00:00
std::unique_ptr<Context> makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper) override;
2020-11-20 16:06:27 +00:00
2020-11-27 14:04:03 +00:00
DatabaseReplicated * database;
2020-11-03 13:47:26 +00:00
};
2021-02-19 23:41:58 +00:00
/// The main purpose of ZooKeeperMetadataTransaction is to execute all zookeeper operation related to query
/// in a single transaction when we performed all required checks and ready to "commit" changes.
/// For example, create ALTER_METADATA entry in ReplicatedMergeTree log,
/// create path/to/entry/finished/host_id node in distributed DDL queue to mark query as executed and
/// update metadata in path/to/replicated_database/metadata/table_name
/// It's used for DatabaseReplicated.
/// TODO we can also use it for ordinary ON CLUSTER queries
class ZooKeeperMetadataTransaction
2020-11-20 16:06:27 +00:00
{
2020-12-04 20:12:32 +00:00
enum State
{
CREATED,
2021-02-08 19:36:17 +00:00
COMMITTED,
2020-12-04 20:12:32 +00:00
FAILED
};
State state = CREATED;
2020-11-20 16:06:27 +00:00
ZooKeeperPtr current_zookeeper;
String zookeeper_path;
2020-11-29 11:45:32 +00:00
bool is_initial_query;
2020-11-20 16:06:27 +00:00
Coordination::Requests ops;
2021-02-19 23:41:58 +00:00
public:
ZooKeeperMetadataTransaction(const ZooKeeperPtr & current_zookeeper_, const String & zookeeper_path_, bool is_initial_query_)
: current_zookeeper(current_zookeeper_)
, zookeeper_path(zookeeper_path_)
, is_initial_query(is_initial_query_)
{
}
bool isInitialQuery() const { return is_initial_query; }
bool isExecuted() const { return state != CREATED; }
String getDatabaseZooKeeperPath() const { return zookeeper_path; }
void addOp(Coordination::RequestPtr && op)
{
assert(!isExecuted());
ops.emplace_back(op);
}
2021-02-08 19:36:17 +00:00
void moveOpsTo(Coordination::Requests & other_ops)
2020-11-20 16:06:27 +00:00
{
2021-02-19 23:41:58 +00:00
assert(!isExecuted());
2020-11-20 16:06:27 +00:00
std::move(ops.begin(), ops.end(), std::back_inserter(other_ops));
2021-01-28 19:02:39 +00:00
ops.clear();
2021-02-08 19:36:17 +00:00
state = COMMITTED;
2020-11-20 16:06:27 +00:00
}
2020-11-29 11:45:32 +00:00
void commit();
2020-12-04 20:12:32 +00:00
~ZooKeeperMetadataTransaction() { assert(isExecuted() || std::uncaught_exceptions()); }
2020-11-20 16:06:27 +00:00
};
2020-11-03 13:47:26 +00:00
}