Fix hung check

This commit is contained in:
alesapin 2022-09-11 13:37:39 +02:00
parent 01e745bbbf
commit ac53830000
3 changed files with 36 additions and 6 deletions

View File

@ -4,6 +4,7 @@
#include <Disks/SingleDiskVolume.h>
#include <Disks/createVolume.h>
#include <IO/HTTPCommon.h>
#include <IO/S3Common.h>
#include <Server/HTTP/HTMLForm.h>
#include <Server/HTTP/HTTPServerResponse.h>
#include <Storages/MergeTree/MergeTreeDataPartInMemory.h>
@ -572,11 +573,22 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart(
{
return downloadPartToDiskRemoteMeta(part_name, replica_path, to_detached, tmp_prefix, disk, *in, throttler);
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::S3_ERROR && e.code() != ErrorCodes::ZERO_COPY_REPLICATION_ERROR)
throw;
if (const auto * s3_exception = dynamic_cast<const S3Exception *>(&e))
{
/// It doesn't make sense to retry Access Denied or No Such Key
if (!s3_exception->isRetryableError())
{
tryLogCurrentException(log, fmt::format("while fetching part: {}", part_name));
throw;
}
}
LOG_WARNING(log, fmt::runtime(e.message() + " Will retry fetching part without zero-copy."));
/// It's important to release session from HTTP pool. Otherwise it's possible to get deadlock

View File

@ -822,6 +822,15 @@ void StorageReplicatedMergeTree::createReplica(const StorageMetadataPtr & metada
} while (code == Coordination::Error::ZBADVERSION);
}
zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperIfTableShutDown() const
{
if (zookeeper_name == default_zookeeper_name)
return getContext()->getZooKeeper();
else
return getContext()->getAuxiliaryZooKeeper(zookeeper_name);
}
void StorageReplicatedMergeTree::drop()
{
/// There is also the case when user has configured ClickHouse to wrong ZooKeeper cluster
@ -833,11 +842,7 @@ void StorageReplicatedMergeTree::drop()
{
/// Table can be shut down, restarting thread is not active
/// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice.
zkutil::ZooKeeperPtr zookeeper;
if (zookeeper_name == default_zookeeper_name)
zookeeper = getContext()->getZooKeeper();
else
zookeeper = getContext()->getAuxiliaryZooKeeper(zookeeper_name);
zkutil::ZooKeeperPtr zookeeper = getZooKeeperIfTableShutDown();
/// If probably there is metadata in ZooKeeper, we don't allow to drop the table.
if (!zookeeper)
@ -7579,7 +7584,16 @@ std::pair<bool, NameSet> StorageReplicatedMergeTree::unlockSharedData(const IMer
return std::make_pair(true, NameSet{});
}
return unlockSharedDataByID(part.getUniqueId(), getTableSharedID(), part.name, replica_name, part.data_part_storage->getDiskType(), getZooKeeper(), *getSettings(), log,
/// We remove parts during table shutdown. If exception happen restarting thread will be already turned
/// off and nobody will reconnect our zookeeper connection. In this case we use zookeeper connection from
/// context.
zkutil::ZooKeeperPtr zookeeper;
if (shutdown_called.load())
zookeeper = getZooKeeperIfTableShutDown();
else
zookeeper = getZooKeeper();
return unlockSharedDataByID(part.getUniqueId(), getTableSharedID(), part.name, replica_name, part.data_part_storage->getDiskType(), zookeeper, *getSettings(), log,
zookeeper_path);
}

View File

@ -363,6 +363,10 @@ private:
zkutil::ZooKeeperPtr tryGetZooKeeper() const;
zkutil::ZooKeeperPtr getZooKeeper() const;
/// Get connection from global context and reconnect if needed
/// NOTE: use it only when table is shut down, in all other cases
/// use getZooKeeper() because it managed by restarting thread.
zkutil::ZooKeeperPtr getZooKeeperIfTableShutDown() const;
zkutil::ZooKeeperPtr getZooKeeperAndAssertNotReadonly() const;
void setZooKeeper();