提交 6232e1cf 编写于 作者: A Alexey Arno

dbms: Added stricter checkings for coordinated requests originating from...

dbms: Added stricter checkings for coordinated requests originating from shards and a more reliable cluster naming scheme. [#METR-18510]
上级 fc2c3c1f
......@@ -16,7 +16,7 @@ namespace DB
class Cluster
{
public:
Cluster(const Settings & settings, const String & cluster_short_name, const String & cluster_name);
Cluster(const Settings & settings, const String & cluster_name);
/// Построить кластер по именам шардов и реплик. Локальные обрабатываются так же как удаленные.
Cluster(const Settings & settings, std::vector<std::vector<String>> names,
......@@ -100,8 +100,13 @@ public:
private:
void initMisc();
/// Create a unique name based on the list of addresses and ports.
/// We need it in order to be able to perform resharding requests
/// on tables that have the distributed engine.
void assignName();
private:
/// Название кластера, если существует.
/// Название кластера.
String name;
/// Описание шардов кластера.
ShardsInfo shards_info;
......
......@@ -129,11 +129,11 @@ private:
/// Принудительно завершить поток.
void abortPollingIfRequested();
void abortCoordinatorIfRequested();
void abortCoordinatorIfRequested(const std::string & coordinator_id);
void abortRecoveryIfRequested();
void abortJobIfRequested();
Status getCoordinatorStatus();
Status getCoordinatorStatus(const std::string & coordinator_id);
/// Зарегистрировать задачу в соответствующий координатор.
void attachJob();
......@@ -148,7 +148,7 @@ private:
/// Функции, которые создают необходимые объекты для синхронизации
/// распределённых задач.
zkutil::RWLock createLock();
zkutil::RWLock createCoordinatorLock();
zkutil::RWLock createCoordinatorLock(const std::string & coordinator_id);
zkutil::Barrier createCheckBarrier(const std::string & coordinator_id);
zkutil::SingleBarrier createOptOutBarrier(const std::string & coordinator_id, size_t count);
zkutil::SingleBarrier createRecoveryBarrier(const ReshardingJob & job);
......@@ -168,7 +168,6 @@ private:
std::string distributed_online_path;
std::string distributed_lock_path;
std::string coordination_path;
std::string current_coordinator_id;
Context & context;
Logger * log;
......
......@@ -326,17 +326,18 @@ namespace ErrorCodes
extern const int DUPLICATE_SHARD_PATHS = 320;
extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE = 321;
extern const int RESHARDING_BUSY_CLUSTER = 322;
extern const int RESHARDING_NO_SUCH_COORDINATOR = 323;
extern const int RESHARDING_NO_COORDINATOR_MEMBERSHIP = 324;
extern const int RESHARDING_ALREADY_SUBSCRIBED = 325;
extern const int RESHARDING_REMOTE_NODE_UNAVAILABLE = 326;
extern const int RESHARDING_REMOTE_NODE_ERROR = 327;
extern const int RESHARDING_COORDINATOR_DELETED = 328;
extern const int RESHARDING_DISTRIBUTED_JOB_ON_HOLD = 329;
extern const int RESHARDING_INVALID_QUERY = 330;
extern const int RESHARDING_INITIATOR_CHECK_FAILED = 331;
extern const int RWLOCK_ALREADY_HELD = 332;
extern const int BARRIER_TIMEOUT = 333;
extern const int RESHARDING_BUSY_SHARD = 323;
extern const int RESHARDING_NO_SUCH_COORDINATOR = 324;
extern const int RESHARDING_NO_COORDINATOR_MEMBERSHIP = 325;
extern const int RESHARDING_ALREADY_SUBSCRIBED = 326;
extern const int RESHARDING_REMOTE_NODE_UNAVAILABLE = 327;
extern const int RESHARDING_REMOTE_NODE_ERROR = 328;
extern const int RESHARDING_COORDINATOR_DELETED = 329;
extern const int RESHARDING_DISTRIBUTED_JOB_ON_HOLD = 330;
extern const int RESHARDING_INVALID_QUERY = 331;
extern const int RESHARDING_INITIATOR_CHECK_FAILED = 332;
extern const int RWLOCK_ALREADY_HELD = 333;
extern const int BARRIER_TIMEOUT = 334;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
......@@ -119,13 +119,12 @@ Clusters::Clusters(const Settings & settings, const String & config_name)
for (const auto & key : config_keys)
impl.emplace(std::piecewise_construct,
std::forward_as_tuple(key),
std::forward_as_tuple(settings, key, config_name + "." + key));
std::forward_as_tuple(settings, config_name + "." + key));
}
/// Реализация класса Cluster
Cluster::Cluster(const Settings & settings, const String & cluster_short_name, const String & cluster_name)
: name(cluster_short_name)
Cluster::Cluster(const Settings & settings, const String & cluster_name)
{
Poco::Util::AbstractConfiguration & config = Poco::Util::Application::instance().config();
Poco::Util::AbstractConfiguration::Keys config_keys;
......@@ -304,43 +303,6 @@ Cluster::Cluster(const Settings & settings, std::vector<std::vector<String>> nam
++current_shard_num;
}
/// Create a unique name based on the list of addresses.
/// We need it in order to be able to perform resharding requests
/// with the remote table function.
std::vector<std::string> elements;
for (const auto & address : addresses)
{
elements.push_back(address.host_name);
elements.push_back(address.resolved_address.host().toString());
}
for (const auto & addresses : addresses_with_failover)
{
for (const auto & address : addresses)
{
elements.push_back(address.host_name);
elements.push_back(address.resolved_address.host().toString());
}
}
std::sort(elements.begin(), elements.end());
unsigned char hash[SHA512_DIGEST_LENGTH];
SHA512_CTX ctx;
SHA512_Init(&ctx);
for (const auto & host : elements)
SHA512_Update(&ctx, reinterpret_cast<const void *>(host.data()), host.size());
SHA512_Final(hash, &ctx);
{
WriteBufferFromString buf(name);
HexWriteBuffer hex_buf(buf);
hex_buf.write(reinterpret_cast<const char *>(hash), sizeof(hash));
}
initMisc();
}
......@@ -379,6 +341,48 @@ void Cluster::initMisc()
break;
}
}
assignName();
}
void Cluster::assignName()
{
std::vector<std::string> elements;
if (!addresses.empty())
{
for (const auto & address : addresses)
elements.push_back(address.host_name + ":" + toString(address.port));
}
else if (!addresses_with_failover.empty())
{
for (const auto & addresses : addresses_with_failover)
{
for (const auto & address : addresses)
elements.push_back(address.host_name + ":" + toString(address.port));
}
}
else
throw Exception("Cluster: ill-formed cluster", ErrorCodes::LOGICAL_ERROR);
std::sort(elements.begin(), elements.end());
unsigned char hash[SHA512_DIGEST_LENGTH];
SHA512_CTX ctx;
SHA512_Init(&ctx);
for (const auto & host : elements)
SHA512_Update(&ctx, reinterpret_cast<const void *>(host.data()), host.size());
SHA512_Final(hash, &ctx);
{
WriteBufferFromString buf(name);
HexWriteBuffer hex_buf(buf);
hex_buf.write(reinterpret_cast<const char *>(hash), sizeof(hash));
}
}
}
......@@ -45,6 +45,7 @@ namespace ErrorCodes
extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
extern const int RESHARDING_BUSY_CLUSTER;
extern const int RESHARDING_BUSY_SHARD;
extern const int RESHARDING_NO_SUCH_COORDINATOR;
extern const int RESHARDING_NO_COORDINATOR_MEMBERSHIP;
extern const int RESHARDING_ALREADY_SUBSCRIBED;
......@@ -133,6 +134,8 @@ private:
/// representations, of all the nodes of the cluster; used to check if a given node
/// is a member of the cluster;
///
/// /shards: the list of shards that have subscribed;
///
/// /check_barrier: when all the participating nodes have checked
/// that they can perform resharding operations, proceed further;
///
......@@ -803,7 +806,7 @@ std::string ReshardingWorker::createCoordinator(const Cluster & cluster)
{
auto effective_cluster_name = zookeeper->get(coordination_path + "/" + coordinator + "/cluster");
if (effective_cluster_name == cluster_name)
throw Exception("The cluster " + cluster_name + " is currently busy with another "
throw Exception("The cluster specified for this table is currently busy with another "
"distributed job. Please try later", ErrorCodes::RESHARDING_BUSY_CLUSTER);
}
......@@ -824,6 +827,9 @@ std::string ReshardingWorker::createCoordinator(const Cluster & cluster)
toString(cluster.getRemoteShardCount() + cluster.getLocalShardCount()),
zkutil::CreateMode::Persistent);
(void) zookeeper->create(getCoordinatorPath(coordinator_id) + "/shards",
"", zkutil::CreateMode::Persistent);
(void) zookeeper->create(getCoordinatorPath(coordinator_id) + "/status",
"", zkutil::CreateMode::Persistent);
setStatus(coordinator_id, STATUS_OK);
......@@ -836,31 +842,39 @@ std::string ReshardingWorker::createCoordinator(const Cluster & cluster)
(void) zookeeper->create(getCoordinatorPath(coordinator_id) + "/cluster_addresses",
"", zkutil::CreateMode::Persistent);
std::vector<std::string> cluster_addresses;
const auto & addresses = cluster.getShardsAddresses();
for (const auto & address : addresses)
auto publish_address = [&](const std::string & host, size_t shard_no)
{
cluster_addresses.push_back(address.host_name);
cluster_addresses.push_back(address.resolved_address.host().toString());
}
int32_t code = zookeeper->tryCreate(getCoordinatorPath(coordinator_id) + "/cluster_addresses/"
+ host, toString(shard_no), zkutil::CreateMode::Persistent);
if ((code != ZOK) && (code != ZNODEEXISTS))
throw zkutil::KeeperException(code);
};
const auto & addresses_with_failover = cluster.getShardsWithFailoverAddresses();
for (const auto & addresses : addresses_with_failover)
if (!cluster.getShardsAddresses().empty())
{
for (const auto & address : addresses)
size_t shard_no = 0;
for (const auto & address : cluster.getShardsAddresses())
{
cluster_addresses.push_back(address.host_name);
cluster_addresses.push_back(address.resolved_address.host().toString());
publish_address(address.host_name, shard_no);
publish_address(address.resolved_address.host().toString(), shard_no);
++shard_no;
}
}
for (const auto & host : cluster_addresses)
else if (!cluster.getShardsWithFailoverAddresses().empty())
{
int32_t code = zookeeper->tryCreate(getCoordinatorPath(coordinator_id) + "/cluster_addresses/"
+ host, "", zkutil::CreateMode::Persistent);
if ((code != ZOK) && (code != ZNODEEXISTS))
throw zkutil::KeeperException(code);
size_t shard_no = 0;
for (const auto & addresses : cluster.getShardsWithFailoverAddresses())
{
for (const auto & address : addresses)
{
publish_address(address.host_name, shard_no);
publish_address(address.resolved_address.host().toString(), shard_no);
}
++shard_no;
}
}
else
throw Exception("ReshardingWorker: ill-formed cluster", ErrorCodes::LOGICAL_ERROR);
return coordinator_id;
}
......@@ -883,20 +897,38 @@ UInt64 ReshardingWorker::subscribe(const std::string & coordinator_id, const std
{
auto zookeeper = context.getZooKeeper();
current_coordinator_id = coordinator_id;
if (!zookeeper->exists(getCoordinatorPath(coordinator_id)))
throw Exception("Coordinator " + coordinator_id + " does not exist",
ErrorCodes::RESHARDING_NO_SUCH_COORDINATOR);
auto current_host = getFQDNOrHostName();
/// Make sure that this shard is not busy in another distributed job.
{
auto lock = createCoordinatorLock();
auto lock = createLock();
zkutil::RWLock::Guard<zkutil::RWLock::Read> guard{lock};
/// Make sure that the query ALTER TABLE RESHARD with the "COORDINATE WITH" tag
/// is not bogus.
auto coordinators = zookeeper->getChildren(coordination_path);
for (const auto & coordinator : coordinators)
{
if (coordinator == coordinator_id)
continue;
auto cluster_addresses = zookeeper->getChildren(coordination_path + "/" + coordinator
+ "/cluster_addresses");
if (std::find(cluster_addresses.begin(), cluster_addresses.end(), current_host)
!= cluster_addresses.end())
throw Exception("This shard is already busy with another distributed job",
ErrorCodes::RESHARDING_BUSY_SHARD);
}
}
if (!zookeeper->exists(getCoordinatorPath(coordinator_id)))
throw Exception("Coordinator " + coordinator_id + " does not exist",
ErrorCodes::RESHARDING_NO_SUCH_COORDINATOR);
{
auto lock = createCoordinatorLock(coordinator_id);
zkutil::RWLock::Guard<zkutil::RWLock::Write> guard{lock};
auto current_host = getFQDNOrHostName();
/// Make sure that the query ALTER TABLE RESHARD with the "COORDINATE WITH" tag
/// is not bogus.
auto cluster_addresses = zookeeper->getChildren(getCoordinatorPath(coordinator_id)
+ "/cluster_addresses");
......@@ -906,18 +938,25 @@ UInt64 ReshardingWorker::subscribe(const std::string & coordinator_id, const std
+ coordinator_id,
ErrorCodes::RESHARDING_NO_COORDINATOR_MEMBERSHIP);
int32_t code = zookeeper->tryCreate(getCoordinatorPath(coordinator_id) + "/status/" + current_host,
"", zkutil::CreateMode::Persistent);
/// Check that the coordinator recognizes our query.
auto query_hash = zookeeper->get(getCoordinatorPath(coordinator_id) + "/query_hash");
if (computeHash(query) != query_hash)
throw Exception("Coordinator " + coordinator_id + " does not handle this query",
ErrorCodes::RESHARDING_INVALID_QUERY);
/// Access granted. Now perform subscription.
auto my_shard_no = zookeeper->get(getCoordinatorPath(coordinator_id) + "/cluster_addresses/"
+ current_host);
int32_t code = zookeeper->tryCreate(getCoordinatorPath(coordinator_id) + "/shards/"
+ toString(my_shard_no), "", zkutil::CreateMode::Persistent);
if (code == ZNODEEXISTS)
throw Exception("Already subscribed to coordinator " + coordinator_id,
throw Exception("This shard has already subscribed to coordinator " + coordinator_id,
ErrorCodes::RESHARDING_ALREADY_SUBSCRIBED);
else if (code != ZOK)
throw zkutil::KeeperException(code);
auto query_hash = zookeeper->get(getCoordinatorPath(coordinator_id) + "/query_hash");
if (computeHash(query) != query_hash)
throw Exception("Coordinator " + coordinator_id + " does not handle this query",
ErrorCodes::RESHARDING_INVALID_QUERY);
zookeeper->create(getCoordinatorPath(coordinator_id) + "/status/"
+ current_host, "", zkutil::CreateMode::Persistent);
setStatus(coordinator_id, current_host, STATUS_OK);
}
......@@ -926,7 +965,7 @@ UInt64 ReshardingWorker::subscribe(const std::string & coordinator_id, const std
/// to avoid any possible conflict when uploading resharded partitions.
UInt64 block_number;
{
auto lock = createCoordinatorLock();
auto lock = createCoordinatorLock(coordinator_id);
zkutil::RWLock::Guard<zkutil::RWLock::Write> guard{lock};
auto current_block_number = zookeeper->get(getCoordinatorPath(coordinator_id) + "/increment");
......@@ -945,9 +984,13 @@ UInt64 ReshardingWorker::subscribe(const std::string & coordinator_id, const std
void ReshardingWorker::unsubscribe(const std::string & coordinator_id)
{
/// Note: we don't remove this shard from the /shards znode because
/// it can subscribe to a distributed job only if its cluster is not
/// currently busy with any distributed job.
auto zookeeper = context.getZooKeeper();
auto lock = createCoordinatorLock();
auto lock = createCoordinatorLock(coordinator_id);
zkutil::RWLock::Guard<zkutil::RWLock::Write> guard{lock};
auto current_host = getFQDNOrHostName();
......@@ -965,7 +1008,7 @@ void ReshardingWorker::addPartitions(const std::string & coordinator_id,
{
auto zookeeper = context.getZooKeeper();
auto lock = createCoordinatorLock();
auto lock = createCoordinatorLock(coordinator_id);
zkutil::RWLock::Guard<zkutil::RWLock::Write> guard{lock};
auto current_host = getFQDNOrHostName();
......@@ -987,7 +1030,7 @@ ReshardingWorker::PartitionList::iterator ReshardingWorker::categorizePartitions
PartitionList uncoordinated_partition_list;
{
auto lock = createCoordinatorLock();
auto lock = createCoordinatorLock(coordinator_id);
zkutil::RWLock::Guard<zkutil::RWLock::Write> guard{lock};
auto current_host = getFQDNOrHostName();
......@@ -1023,7 +1066,7 @@ ReshardingWorker::PartitionList::iterator ReshardingWorker::categorizePartitions
size_t ReshardingWorker::getPartitionCount(const std::string & coordinator_id)
{
auto lock = createCoordinatorLock();
auto lock = createCoordinatorLock(coordinator_id);
zkutil::RWLock::Guard<zkutil::RWLock::Read> guard{lock};
return getPartitionCountUnlocked(coordinator_id);
......@@ -1037,7 +1080,7 @@ size_t ReshardingWorker::getPartitionCountUnlocked(const std::string & coordinat
size_t ReshardingWorker::getNodeCount(const std::string & coordinator_id)
{
auto lock = createCoordinatorLock();
auto lock = createCoordinatorLock(coordinator_id);
zkutil::RWLock::Guard<zkutil::RWLock::Read> guard{lock};
auto zookeeper = context.getZooKeeper();
......@@ -1069,11 +1112,11 @@ void ReshardingWorker::setStatus(const std::string & coordinator_id, const std::
toString(static_cast<UInt64>(status)));
}
ReshardingWorker::Status ReshardingWorker::getCoordinatorStatus()
ReshardingWorker::Status ReshardingWorker::getCoordinatorStatus(const std::string & coordinator_id)
{
auto zookeeper = context.getZooKeeper();
auto status_str = zookeeper->get(getCoordinatorPath(current_coordinator_id) + "/status");
auto status_str = zookeeper->get(getCoordinatorPath(coordinator_id) + "/status");
return static_cast<Status>(std::stoull(status_str));
}
......@@ -1146,8 +1189,6 @@ void ReshardingWorker::attachJob()
if (!current_job.isCoordinated())
return;
current_coordinator_id = current_job.coordinator_id;
auto zookeeper = context.getZooKeeper();
/// Check if the corresponding coordinator exists. If it doesn't, throw an exception,
......@@ -1202,7 +1243,7 @@ void ReshardingWorker::detachJob()
bool delete_coordinator = false;
{
auto lock = createCoordinatorLock();
auto lock = createCoordinatorLock(current_job.coordinator_id);
zkutil::RWLock::Guard<zkutil::RWLock::Write> guard{lock};
auto children = zookeeper->getChildren(getPartitionPath(current_job) + "/nodes");
......@@ -1242,13 +1283,13 @@ void ReshardingWorker::abortPollingIfRequested()
throw Exception("Cancelled resharding", ErrorCodes::ABORTED);
}
void ReshardingWorker::abortCoordinatorIfRequested()
void ReshardingWorker::abortCoordinatorIfRequested(const std::string & coordinator_id)
{
bool must_abort;
try
{
must_abort = must_stop || (getCoordinatorStatus() != STATUS_OK);
must_abort = must_stop || (getCoordinatorStatus(coordinator_id) != STATUS_OK);
}
catch (...)
{
......@@ -1347,13 +1388,14 @@ zkutil::RWLock ReshardingWorker::createLock()
return lock;
}
zkutil::RWLock ReshardingWorker::createCoordinatorLock()
zkutil::RWLock ReshardingWorker::createCoordinatorLock(const std::string & coordinator_id)
{
auto zookeeper = context.getZooKeeper();
zkutil::RWLock lock(zookeeper, getCoordinatorPath(current_coordinator_id) + "/lock");
zkutil::RWLock lock(zookeeper, getCoordinatorPath(coordinator_id) + "/lock");
zkutil::RWLock::CancellationHook hook = std::bind(&ReshardingWorker::abortCoordinatorIfRequested, this);
zkutil::RWLock::CancellationHook hook = std::bind(&ReshardingWorker::abortCoordinatorIfRequested, this,
coordinator_id);
lock.setCancellationHook(hook);
return lock;
......@@ -1367,7 +1409,9 @@ zkutil::Barrier ReshardingWorker::createCheckBarrier(const std::string & coordin
zkutil::Barrier check_barrier{zookeeper, getCoordinatorPath(coordinator_id) + "/check_barrier",
std::stoull(node_count)};
zkutil::Barrier::CancellationHook hook = std::bind(&ReshardingWorker::abortCoordinatorIfRequested, this);
zkutil::Barrier::CancellationHook hook = std::bind(&ReshardingWorker::abortCoordinatorIfRequested, this,
coordinator_id
);
check_barrier.setCancellationHook(hook);
return check_barrier;
......@@ -1381,7 +1425,8 @@ zkutil::SingleBarrier ReshardingWorker::createOptOutBarrier(const std::string &
zkutil::SingleBarrier opt_out_barrier{zookeeper, getCoordinatorPath(coordinator_id)
+ "/opt_out_barrier", count};
zkutil::SingleBarrier::CancellationHook hook = std::bind(&ReshardingWorker::abortCoordinatorIfRequested, this);
zkutil::SingleBarrier::CancellationHook hook = std::bind(&ReshardingWorker::abortCoordinatorIfRequested, this,
coordinator_id);
opt_out_barrier.setCancellationHook(hook);
return opt_out_barrier;
......
......@@ -3463,11 +3463,11 @@ void StorageReplicatedMergeTree::reshardPartitions(ASTPtr query, const String &
if (!resharding_worker.isStarted())
throw Exception("Resharding background thread is not running", ErrorCodes::RESHARDING_NO_WORKER);
bool is_coordinated = !coordinator.isNull();
bool has_coordinator = !coordinator.isNull();
std::string coordinator_id;
UInt64 block_number = 0;
if (is_coordinated)
if (has_coordinator)
{
coordinator_id = coordinator.get<const String &>();
block_number = resharding_worker.subscribe(coordinator_id, queryToString(query));
......@@ -3533,7 +3533,7 @@ void StorageReplicatedMergeTree::reshardPartitions(ASTPtr query, const String &
if (partition_list.empty())
{
if (!is_coordinated)
if (!has_coordinator)
throw Exception("No existing partition found", ErrorCodes::PARTITION_DOESNT_EXIST);
}
else
......@@ -3552,7 +3552,7 @@ void StorageReplicatedMergeTree::reshardPartitions(ASTPtr query, const String &
}
}
if (is_coordinated)
if (has_coordinator)
{
size_t old_node_count = resharding_worker.getNodeCount(coordinator_id);
resharding_worker.addPartitions(coordinator_id, partition_list);
......@@ -3596,7 +3596,7 @@ void StorageReplicatedMergeTree::reshardPartitions(ASTPtr query, const String &
if (uncoordinated_begin == partition_list.cbegin())
{
coordinator_id.clear();
is_coordinated = false;
has_coordinator = false;
}
}
else
......@@ -3635,7 +3635,7 @@ void StorageReplicatedMergeTree::reshardPartitions(ASTPtr query, const String &
}
catch (const Exception & ex)
{
if (is_coordinated)
if (has_coordinator)
{
if ((ex.code() == ErrorCodes::RESHARDING_NO_SUCH_COORDINATOR) ||
(ex.code() == ErrorCodes::RESHARDING_NO_COORDINATOR_MEMBERSHIP) ||
......@@ -3665,7 +3665,7 @@ void StorageReplicatedMergeTree::reshardPartitions(ASTPtr query, const String &
}
catch (...)
{
if (is_coordinated)
if (has_coordinator)
{
try
{
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册