mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge pull request #13866 from ClickHouse/remove-useless-code-zkutil
Remove useless code around zkutil
This commit is contained in:
commit
1a33f6d0b8
@ -1,19 +1,15 @@
|
||||
include (${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake)
|
||||
|
||||
add_executable (date_lut_init date_lut_init.cpp)
|
||||
add_executable (date_lut2 date_lut2.cpp)
|
||||
add_executable (date_lut3 date_lut3.cpp)
|
||||
add_executable (date_lut4 date_lut4.cpp)
|
||||
add_executable (date_lut_default_timezone date_lut_default_timezone.cpp)
|
||||
add_executable (local_date_time_comparison local_date_time_comparison.cpp)
|
||||
add_executable (realloc-perf allocator.cpp)
|
||||
|
||||
set(PLATFORM_LIBS ${CMAKE_DL_LIBS})
|
||||
|
||||
target_link_libraries (date_lut_init PRIVATE common ${PLATFORM_LIBS})
|
||||
target_link_libraries (date_lut2 PRIVATE common ${PLATFORM_LIBS})
|
||||
target_link_libraries (date_lut3 PRIVATE common ${PLATFORM_LIBS})
|
||||
target_link_libraries (date_lut4 PRIVATE common ${PLATFORM_LIBS})
|
||||
target_link_libraries (date_lut_default_timezone PRIVATE common ${PLATFORM_LIBS})
|
||||
target_link_libraries (local_date_time_comparison PRIVATE common)
|
||||
target_link_libraries (realloc-perf PRIVATE common)
|
||||
|
@ -1,20 +0,0 @@
|
||||
#include <iostream>
|
||||
#include <common/DateLUT.h>
|
||||
|
||||
|
||||
int main(int, char **)
|
||||
{
|
||||
/** В DateLUT был глюк - для времён из дня 1970-01-01, возвращался номер часа больше 23. */
|
||||
static const time_t time = 66130;
|
||||
|
||||
const auto & date_lut = DateLUT::instance();
|
||||
|
||||
std::cerr << date_lut.toHour(time) << std::endl;
|
||||
std::cerr << date_lut.toDayNum(time) << std::endl;
|
||||
|
||||
const auto * values = reinterpret_cast<const DateLUTImpl::Values *>(&date_lut);
|
||||
|
||||
std::cerr << values[0].date << ", " << time_t(values[1].date - values[0].date) << std::endl;
|
||||
|
||||
return 0;
|
||||
}
|
@ -1,8 +0,0 @@
|
||||
#include <common/DateLUT.h>
|
||||
|
||||
/// Позволяет проверить время инициализации DateLUT.
|
||||
int main(int, char **)
|
||||
{
|
||||
DateLUT::instance();
|
||||
return 0;
|
||||
}
|
@ -1,49 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include "ZooKeeperHolder.h"
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
|
||||
class Increment
|
||||
{
|
||||
public:
|
||||
Increment(ZooKeeperHolderPtr zk_holder_, const std::string & path_)
|
||||
: zookeeper_holder(zk_holder_), path(path_)
|
||||
{
|
||||
zookeeper_holder->getZooKeeper()->createAncestors(path);
|
||||
}
|
||||
|
||||
size_t get()
|
||||
{
|
||||
LOG_TRACE(log, "Get increment");
|
||||
|
||||
size_t result = 0;
|
||||
std::string result_str;
|
||||
Coordination::Stat stat;
|
||||
|
||||
bool success = false;
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
do
|
||||
{
|
||||
if (zookeeper->tryGet(path, result_str, &stat))
|
||||
{
|
||||
result = std::stol(result_str) + 1;
|
||||
success = zookeeper->trySet(path, std::to_string(result), stat.version) == Coordination::Error::ZOK;
|
||||
}
|
||||
else
|
||||
{
|
||||
success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == Coordination::Error::ZOK;
|
||||
}
|
||||
}
|
||||
while (!success);
|
||||
|
||||
return result;
|
||||
}
|
||||
private:
|
||||
zkutil::ZooKeeperHolderPtr zookeeper_holder;
|
||||
std::string path;
|
||||
Poco::Logger * log = &Poco::Logger::get("zkutil::Increment");
|
||||
};
|
||||
|
||||
}
|
@ -1,76 +0,0 @@
|
||||
#include "KeeperException.h"
|
||||
#include "Lock.h"
|
||||
|
||||
|
||||
using namespace zkutil;
|
||||
|
||||
bool Lock::tryLock()
|
||||
{
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
if (locked)
|
||||
{
|
||||
/// проверим, что нода создана и я ее владелец
|
||||
if (tryCheck() != Status::LOCKED_BY_ME)
|
||||
locked.reset();
|
||||
}
|
||||
else
|
||||
{
|
||||
std::string dummy;
|
||||
Coordination::Error code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy);
|
||||
|
||||
if (code == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
locked.reset();
|
||||
}
|
||||
else if (code == Coordination::Error::ZOK)
|
||||
{
|
||||
locked = std::make_unique<ZooKeeperHandler>(zookeeper);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Coordination::Exception(code);
|
||||
}
|
||||
}
|
||||
return bool(locked);
|
||||
}
|
||||
|
||||
void Lock::unlock()
|
||||
{
|
||||
if (locked)
|
||||
{
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
if (tryCheck() == Status::LOCKED_BY_ME)
|
||||
zookeeper->remove(lock_path, -1);
|
||||
locked.reset();
|
||||
}
|
||||
}
|
||||
|
||||
Lock::Status Lock::tryCheck() const
|
||||
{
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
|
||||
Status lock_status;
|
||||
Coordination::Stat stat;
|
||||
std::string dummy;
|
||||
bool result = zookeeper->tryGet(lock_path, dummy, &stat);
|
||||
if (!result)
|
||||
lock_status = UNLOCKED;
|
||||
else
|
||||
{
|
||||
if (stat.ephemeralOwner == zookeeper->getClientID())
|
||||
lock_status = LOCKED_BY_ME;
|
||||
else
|
||||
lock_status = LOCKED_BY_OTHER;
|
||||
}
|
||||
|
||||
if (locked && lock_status != LOCKED_BY_ME)
|
||||
LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: {}/{}", lock_path, lock_message);
|
||||
|
||||
return lock_status;
|
||||
}
|
||||
|
||||
void Lock::unlockAssumeLockNodeRemovedManually()
|
||||
{
|
||||
locked.reset();
|
||||
}
|
||||
|
@ -1,78 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include "ZooKeeperHolder.h"
|
||||
#include <common/logger_useful.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
class Lock
|
||||
{
|
||||
public:
|
||||
/// lock_prefix - относительный путь до блокировки в ZK. Начинается со слеша
|
||||
/// lock_name - имя ноды блокировки в ZK
|
||||
Lock(
|
||||
zkutil::ZooKeeperHolderPtr zookeeper_holder_,
|
||||
const std::string & lock_prefix_,
|
||||
const std::string & lock_name_,
|
||||
const std::string & lock_message_ = "",
|
||||
bool create_parent_path_ = false)
|
||||
:
|
||||
zookeeper_holder(zookeeper_holder_),
|
||||
lock_path(lock_prefix_ + "/" + lock_name_),
|
||||
lock_message(lock_message_),
|
||||
log(&Poco::Logger::get("zkutil::Lock"))
|
||||
{
|
||||
auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
if (create_parent_path_)
|
||||
zookeeper->createAncestors(lock_prefix_);
|
||||
|
||||
zookeeper->createIfNotExists(lock_prefix_, "");
|
||||
}
|
||||
|
||||
Lock(const Lock &) = delete;
|
||||
Lock(Lock && lock) = default;
|
||||
Lock & operator=(const Lock &) = delete;
|
||||
|
||||
~Lock()
|
||||
{
|
||||
try
|
||||
{
|
||||
unlock();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
DB::tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
enum Status
|
||||
{
|
||||
UNLOCKED,
|
||||
LOCKED_BY_ME,
|
||||
LOCKED_BY_OTHER,
|
||||
};
|
||||
|
||||
/// проверяет создана ли эфемерная нода и кто ее владелец.
|
||||
Status tryCheck() const;
|
||||
|
||||
void unlock();
|
||||
void unlockAssumeLockNodeRemovedManually();
|
||||
|
||||
bool tryLock();
|
||||
|
||||
/// путь к ноде блокировки в zookeeper
|
||||
const std::string & getPath() { return lock_path; }
|
||||
|
||||
private:
|
||||
zkutil::ZooKeeperHolderPtr zookeeper_holder;
|
||||
/// пока храним указатель на хендлер, никто не может переиницализировать сессию с zookeeper
|
||||
using ZooKeeperHandler = zkutil::ZooKeeperHolder::UnstorableZookeeperHandler;
|
||||
std::unique_ptr<ZooKeeperHandler> locked;
|
||||
|
||||
std::string lock_path;
|
||||
std::string lock_message;
|
||||
Poco::Logger * log;
|
||||
|
||||
};
|
||||
}
|
@ -1,84 +0,0 @@
|
||||
#include "ZooKeeperHolder.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
using namespace zkutil;
|
||||
|
||||
ZooKeeperHolder::UnstorableZookeeperHandler ZooKeeperHolder::getZooKeeper()
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
return UnstorableZookeeperHandler(ptr);
|
||||
}
|
||||
|
||||
void ZooKeeperHolder::initFromInstance(const ZooKeeper::Ptr & zookeeper_ptr)
|
||||
{
|
||||
ptr = zookeeper_ptr;
|
||||
}
|
||||
|
||||
bool ZooKeeperHolder::replaceZooKeeperSessionToNewOne()
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
|
||||
if (ptr.unique())
|
||||
{
|
||||
ptr = ptr->startNewSession();
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_ERROR(log, "replaceZooKeeperSessionToNewOne(): Fail to replace zookeeper session to new one because handlers for old zookeeper session still exists.");
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
bool ZooKeeperHolder::isSessionExpired() const
|
||||
{
|
||||
return ptr ? ptr->expired() : false;
|
||||
}
|
||||
|
||||
|
||||
std::string ZooKeeperHolder::nullptr_exception_message =
|
||||
"UnstorableZookeeperHandler::zk_ptr is nullptr. "
|
||||
"ZooKeeperHolder should be initialized before sending any request to ZooKeeper";
|
||||
|
||||
ZooKeeperHolder::UnstorableZookeeperHandler::UnstorableZookeeperHandler(ZooKeeper::Ptr zk_ptr_)
|
||||
: zk_ptr(zk_ptr_)
|
||||
{
|
||||
}
|
||||
|
||||
ZooKeeper * ZooKeeperHolder::UnstorableZookeeperHandler::operator->()
|
||||
{
|
||||
if (zk_ptr == nullptr)
|
||||
throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return zk_ptr.get();
|
||||
}
|
||||
|
||||
const ZooKeeper * ZooKeeperHolder::UnstorableZookeeperHandler::operator->() const
|
||||
{
|
||||
if (zk_ptr == nullptr)
|
||||
throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::LOGICAL_ERROR);
|
||||
return zk_ptr.get();
|
||||
}
|
||||
|
||||
ZooKeeper & ZooKeeperHolder::UnstorableZookeeperHandler::operator*()
|
||||
{
|
||||
if (zk_ptr == nullptr)
|
||||
throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::LOGICAL_ERROR);
|
||||
return *zk_ptr;
|
||||
}
|
||||
|
||||
const ZooKeeper & ZooKeeperHolder::UnstorableZookeeperHandler::operator*() const
|
||||
{
|
||||
if (zk_ptr == nullptr)
|
||||
throw DB::Exception(nullptr_exception_message, DB::ErrorCodes::LOGICAL_ERROR);
|
||||
return *zk_ptr;
|
||||
}
|
@ -1,86 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include "ZooKeeper.h"
|
||||
#include <mutex>
|
||||
#include <boost/noncopyable.hpp>
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
class Lock;
|
||||
|
||||
class ZooKeeperHolder : public boost::noncopyable
|
||||
{
|
||||
friend class zkutil::Lock;
|
||||
|
||||
protected:
|
||||
class UnstorableZookeeperHandler;
|
||||
|
||||
public:
|
||||
ZooKeeperHolder() = default;
|
||||
|
||||
/// вызывать из одного потока - не thread safe
|
||||
template <typename... Args>
|
||||
void init(Args &&... args);
|
||||
/// был ли класс инициализирован
|
||||
bool isInitialized() const { return ptr != nullptr; }
|
||||
|
||||
/// Workaround for zkutil::Lock
|
||||
void initFromInstance(const ZooKeeper::Ptr & zookeeper_ptr);
|
||||
|
||||
UnstorableZookeeperHandler getZooKeeper();
|
||||
bool replaceZooKeeperSessionToNewOne();
|
||||
|
||||
bool isSessionExpired() const;
|
||||
|
||||
protected:
|
||||
/** Хендлер для подсчета количества используемых ссылок на ZooKeeper
|
||||
*
|
||||
* Запрещается переинициализировать ZooKeeper пока, хранится хотя бы один хендлер на него.
|
||||
* Большинство классов должны хранить хендлер на стеке и не хранить как член класса.
|
||||
* Т.е. хранить holder и запрашивать хендлер перед каждым использованием.
|
||||
* Поэтому класс специально объявлен, как protected.
|
||||
*
|
||||
* Исключение - классы, работающие с эфимерными нодами. Пример: zkutil::Lock
|
||||
*
|
||||
* Как использовать:
|
||||
* auto zookeeper = zookeeper_holder->getZooKeeper();
|
||||
* zookeeper->get(path);
|
||||
*/
|
||||
class UnstorableZookeeperHandler
|
||||
{
|
||||
public:
|
||||
UnstorableZookeeperHandler(ZooKeeper::Ptr zk_ptr_);
|
||||
|
||||
explicit operator bool() const { return bool(zk_ptr); }
|
||||
bool operator==(std::nullptr_t) const { return zk_ptr == nullptr; }
|
||||
bool operator!=(std::nullptr_t) const { return !(*this == nullptr); }
|
||||
|
||||
/// в случае nullptr методы разыменования кидают исключение,
|
||||
/// с более подробным текстом, чем просто nullptr
|
||||
ZooKeeper * operator->();
|
||||
const ZooKeeper * operator->() const;
|
||||
ZooKeeper & operator*();
|
||||
const ZooKeeper & operator*() const;
|
||||
|
||||
private:
|
||||
ZooKeeper::Ptr zk_ptr;
|
||||
};
|
||||
|
||||
private:
|
||||
mutable std::mutex mutex;
|
||||
ZooKeeper::Ptr ptr;
|
||||
|
||||
Poco::Logger * log = &Poco::Logger::get("ZooKeeperHolder");
|
||||
|
||||
static std::string nullptr_exception_message;
|
||||
};
|
||||
|
||||
template <typename... Args>
|
||||
void ZooKeeperHolder::init(Args &&... args)
|
||||
{
|
||||
ptr = std::make_shared<ZooKeeper>(std::forward<Args>(args)...);
|
||||
}
|
||||
|
||||
using ZooKeeperHolderPtr = std::shared_ptr<ZooKeeperHolder>;
|
||||
|
||||
}
|
@ -4,18 +4,12 @@ target_link_libraries(zkutil_test_commands PRIVATE clickhouse_common_zookeeper)
|
||||
add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp)
|
||||
target_link_libraries(zkutil_test_commands_new_lib PRIVATE clickhouse_common_zookeeper string_utils)
|
||||
|
||||
add_executable(zkutil_test_lock zkutil_test_lock.cpp)
|
||||
target_link_libraries(zkutil_test_lock PRIVATE clickhouse_common_zookeeper)
|
||||
|
||||
add_executable(zkutil_expiration_test zkutil_expiration_test.cpp)
|
||||
target_link_libraries(zkutil_expiration_test PRIVATE clickhouse_common_zookeeper)
|
||||
|
||||
add_executable(zkutil_test_async zkutil_test_async.cpp)
|
||||
target_link_libraries(zkutil_test_async PRIVATE clickhouse_common_zookeeper)
|
||||
|
||||
add_executable(zkutil_zookeeper_holder zkutil_zookeeper_holder.cpp)
|
||||
target_link_libraries(zkutil_zookeeper_holder PRIVATE clickhouse_common_zookeeper)
|
||||
|
||||
add_executable (zk_many_watches_reconnect zk_many_watches_reconnect.cpp)
|
||||
target_link_libraries (zk_many_watches_reconnect PRIVATE clickhouse_common_zookeeper clickhouse_common_config)
|
||||
|
||||
|
@ -1,22 +0,0 @@
|
||||
#include <iostream>
|
||||
#include <Common/ZooKeeper/Lock.h>
|
||||
|
||||
int main()
|
||||
{
|
||||
|
||||
try
|
||||
{
|
||||
auto zookeeper_holder = std::make_shared<zkutil::ZooKeeperHolder>();
|
||||
zookeeper_holder->init("localhost:2181");
|
||||
|
||||
zkutil::Lock l(zookeeper_holder, "/test", "test_lock");
|
||||
std::cout << "check " << l.tryCheck() << std::endl;
|
||||
std::cout << "lock tryLock() " << l.tryLock() << std::endl;
|
||||
std::cout << "check " << l.tryCheck() << std::endl;
|
||||
}
|
||||
catch (const Poco::Exception & e)
|
||||
{
|
||||
std::cout << e.message() << std::endl;
|
||||
}
|
||||
return 0;
|
||||
}
|
@ -1,32 +0,0 @@
|
||||
#include <Common/ZooKeeper/ZooKeeperHolder.h>
|
||||
#include <iostream>
|
||||
|
||||
#include <Poco/Util/Application.h>
|
||||
|
||||
int main()
|
||||
{
|
||||
// Test::initLogger();
|
||||
|
||||
zkutil::ZooKeeperHolder zk_holder;
|
||||
zk_holder.init("localhost:2181");
|
||||
|
||||
{
|
||||
auto zk_handler = zk_holder.getZooKeeper();
|
||||
if (zk_handler)
|
||||
{
|
||||
bool started_new_session = zk_holder.replaceZooKeeperSessionToNewOne();
|
||||
std::cerr << "Started new session: " << started_new_session << "\n";
|
||||
std::cerr << "get / " << zk_handler->get("/") << "\n";
|
||||
}
|
||||
}
|
||||
|
||||
{
|
||||
bool started_new_session = zk_holder.replaceZooKeeperSessionToNewOne();
|
||||
std::cerr << "Started new session: " << started_new_session << "\n";
|
||||
auto zk_handler = zk_holder.getZooKeeper();
|
||||
if (zk_handler != nullptr)
|
||||
std::cerr << "get / " << zk_handler->get("/") << "\n";
|
||||
}
|
||||
|
||||
return 0;
|
||||
}
|
@ -98,10 +98,8 @@ SRCS(
|
||||
UTF8Helpers.cpp
|
||||
WeakHash.cpp
|
||||
ZooKeeper/IKeeper.cpp
|
||||
ZooKeeper/Lock.cpp
|
||||
ZooKeeper/TestKeeper.cpp
|
||||
ZooKeeper/ZooKeeper.cpp
|
||||
ZooKeeper/ZooKeeperHolder.cpp
|
||||
ZooKeeper/ZooKeeperImpl.cpp
|
||||
ZooKeeper/ZooKeeperNodeCache.cpp
|
||||
|
||||
|
@ -1191,7 +1191,7 @@ void SSDCacheStorage::update(DictionarySourcePtr & source_ptr, const std::vector
|
||||
|
||||
if (update_error_count)
|
||||
{
|
||||
/// TODO: юзать старые значения.
|
||||
/// TODO: use old values
|
||||
|
||||
/// We don't have expired data for that `id` so all we can do is to rethrow `last_exception`.
|
||||
std::rethrow_exception(last_update_exception);
|
||||
|
@ -28,7 +28,6 @@
|
||||
#include <Common/randomSeed.h>
|
||||
#include <Common/ZooKeeper/ZooKeeper.h>
|
||||
#include <Common/ZooKeeper/KeeperException.h>
|
||||
#include <Common/ZooKeeper/Lock.h>
|
||||
#include <Common/isLocalAddress.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
@ -196,12 +195,101 @@ struct DDLTask
|
||||
};
|
||||
|
||||
|
||||
static std::unique_ptr<zkutil::Lock> createSimpleZooKeeperLock(
|
||||
const std::shared_ptr<zkutil::ZooKeeper> & zookeeper, const String & lock_prefix, const String & lock_name, const String & lock_message)
|
||||
namespace
|
||||
{
|
||||
auto zookeeper_holder = std::make_shared<zkutil::ZooKeeperHolder>();
|
||||
zookeeper_holder->initFromInstance(zookeeper);
|
||||
return std::make_unique<zkutil::Lock>(std::move(zookeeper_holder), lock_prefix, lock_name, lock_message);
|
||||
|
||||
/** Caveats: usage of locks in ZooKeeper is incorrect in 99% of cases,
|
||||
* and highlights your poor understanding of distributed systems.
|
||||
*
|
||||
* It's only correct if all the operations that are performed under lock
|
||||
* are atomically checking that the lock still holds
|
||||
* or if we ensure that these operations will be undone if lock is lost
|
||||
* (due to ZooKeeper session loss) that's very difficult to achieve.
|
||||
*
|
||||
* It's Ok if every operation that we perform under lock is actually operation in ZooKeeper.
|
||||
*
|
||||
* In 1% of cases when you can correctly use Lock, the logic is complex enough, so you don't need this class.
|
||||
*
|
||||
* TLDR: Don't use this code.
|
||||
* We only have a few cases of it's usage and it will be removed.
|
||||
*/
|
||||
class ZooKeeperLock
|
||||
{
|
||||
public:
|
||||
/// lock_prefix - path where the ephemeral lock node will be created
|
||||
/// lock_name - the name of the ephemeral lock node
|
||||
ZooKeeperLock(
|
||||
const zkutil::ZooKeeperPtr & zookeeper_,
|
||||
const std::string & lock_prefix_,
|
||||
const std::string & lock_name_,
|
||||
const std::string & lock_message_ = "")
|
||||
:
|
||||
zookeeper(zookeeper_),
|
||||
lock_path(lock_prefix_ + "/" + lock_name_),
|
||||
lock_message(lock_message_),
|
||||
log(&Poco::Logger::get("zkutil::Lock"))
|
||||
{
|
||||
zookeeper->createIfNotExists(lock_prefix_, "");
|
||||
}
|
||||
|
||||
~ZooKeeperLock()
|
||||
{
|
||||
try
|
||||
{
|
||||
unlock();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
DB::tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
|
||||
void unlock()
|
||||
{
|
||||
Coordination::Stat stat;
|
||||
std::string dummy;
|
||||
bool result = zookeeper->tryGet(lock_path, dummy, &stat);
|
||||
|
||||
if (result && stat.ephemeralOwner == zookeeper->getClientID())
|
||||
zookeeper->remove(lock_path, -1);
|
||||
else
|
||||
LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: {}/{}", lock_path, lock_message);
|
||||
}
|
||||
|
||||
bool tryLock()
|
||||
{
|
||||
std::string dummy;
|
||||
Coordination::Error code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy);
|
||||
|
||||
if (code == Coordination::Error::ZNODEEXISTS)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
else if (code == Coordination::Error::ZOK)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Coordination::Exception(code);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
zkutil::ZooKeeperPtr zookeeper;
|
||||
|
||||
std::string lock_path;
|
||||
std::string lock_message;
|
||||
Poco::Logger * log;
|
||||
|
||||
};
|
||||
|
||||
std::unique_ptr<ZooKeeperLock> createSimpleZooKeeperLock(
|
||||
const zkutil::ZooKeeperPtr & zookeeper, const String & lock_prefix, const String & lock_name, const String & lock_message)
|
||||
{
|
||||
return std::make_unique<ZooKeeperLock>(zookeeper, lock_prefix, lock_name, lock_message);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
@ -914,8 +1002,6 @@ void DDLWorker::cleanupQueue(Int64 current_time_seconds, const ZooKeeperPtr & zo
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(lock_path, -1));
|
||||
ops.emplace_back(zkutil::makeRemoveRequest(node_path, -1));
|
||||
zookeeper->multi(ops);
|
||||
|
||||
lock->unlockAssumeLockNodeRemovedManually();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
|
Loading…
Reference in New Issue
Block a user