Merge pull request #13866 from ClickHouse/remove-useless-code-zkutil

Remove useless code around zkutil
This commit is contained in:
alexey-milovidov 2020-08-19 12:32:08 +03:00 committed by GitHub
commit 1a33f6d0b8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 95 additions and 476 deletions

View File

@ -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)

View File

@ -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;
}

View File

@ -1,8 +0,0 @@
#include <common/DateLUT.h>
/// Позволяет проверить время инициализации DateLUT.
int main(int, char **)
{
DateLUT::instance();
return 0;
}

View File

@ -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");
};
}

View File

@ -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();
}

View File

@ -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;
};
}

View File

@ -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;
}

View File

@ -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>;
}

View File

@ -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)

View File

@ -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;
}

View File

@ -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;
}

View File

@ -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

View File

@ -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);

View File

@ -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 (...)