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
|
|
|
|
2021-01-18 14:09:39 +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
|
|
|
|
2020-12-04 20:12:32 +00:00
|
|
|
struct MetadataTransaction;
|
|
|
|
using MetadataTransactionPtr = std::shared_ptr<MetadataTransaction>;
|
|
|
|
|
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
|
|
|
|
{
|
|
|
|
String query;
|
|
|
|
std::vector<HostID> hosts;
|
|
|
|
String initiator; // optional
|
|
|
|
|
|
|
|
static constexpr int CURRENT_VERSION = 1;
|
|
|
|
|
|
|
|
String toString() const;
|
|
|
|
|
|
|
|
void parse(const String & data);
|
|
|
|
};
|
|
|
|
|
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;
|
|
|
|
|
2021-02-01 19:29:47 +00:00
|
|
|
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
|
|
|
//MetadataTransactionPtr txn;
|
|
|
|
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) {}
|
2020-12-03 18:14:27 +00:00
|
|
|
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;
|
|
|
|
|
2020-12-04 20:12:32 +00:00
|
|
|
virtual std::unique_ptr<Context> makeQueryContext(Context & from_context);
|
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(); }
|
|
|
|
|
|
|
|
};
|
|
|
|
|
|
|
|
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;
|
2020-12-04 20:12:32 +00:00
|
|
|
std::unique_ptr<Context> makeQueryContext(Context & from_context) override;
|
2020-11-20 16:06:27 +00:00
|
|
|
|
2020-11-27 14:04:03 +00:00
|
|
|
static String getLogEntryName(UInt32 log_entry_number);
|
|
|
|
static UInt32 getLogEntryNumber(const String & log_entry_name);
|
|
|
|
|
|
|
|
DatabaseReplicated * database;
|
2020-11-03 13:47:26 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
|
2020-11-20 16:06:27 +00:00
|
|
|
struct MetadataTransaction
|
|
|
|
{
|
2020-12-04 20:12:32 +00:00
|
|
|
enum State
|
|
|
|
{
|
|
|
|
CREATED,
|
|
|
|
COMMITED,
|
|
|
|
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;
|
|
|
|
|
|
|
|
void addOps(Coordination::Requests & other_ops)
|
|
|
|
{
|
|
|
|
std::move(ops.begin(), ops.end(), std::back_inserter(other_ops));
|
2021-01-28 19:02:39 +00:00
|
|
|
ops.clear();
|
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
|
|
|
|
2020-11-20 16:06:27 +00:00
|
|
|
};
|
|
|
|
|
2020-11-03 13:47:26 +00:00
|
|
|
}
|