Merge branch 'master' of https://github.com/yandex/ClickHouse into CLICKHOUSE-2720

This commit is contained in:
Ivan Blinkov 2018-04-02 18:33:58 +03:00
commit cc67d849aa
252 changed files with 4619 additions and 2856 deletions

3
.gitmodules vendored
View File

@ -1,6 +1,3 @@
[submodule "contrib/zookeeper"]
path = contrib/zookeeper
url = https://github.com/ClickHouse-Extras/zookeeper.git
[submodule "contrib/poco"]
path = contrib/poco
url = https://github.com/ClickHouse-Extras/poco

View File

@ -258,7 +258,6 @@ include (cmake/find_sparsehash.cmake)
include (cmake/find_rt.cmake)
include (cmake/find_execinfo.cmake)
include (cmake/find_readline_edit.cmake)
include (cmake/find_zookeeper.cmake)
include (cmake/find_re2.cmake)
include (cmake/find_rdkafka.cmake)
include (cmake/find_capnp.cmake)

View File

@ -1,32 +1,34 @@
# Contributing to ClickHouse
## Technical info
Developer guide for writing code for ClickHouse is published on official website alongside the usage and operations documentation:
https://clickhouse.yandex/docs/en/development/index.html
## Legal info
In order for us (YANDEX LLC) to accept patches and other contributions from you, you will have to adopt our Yandex Contributor License Agreement (the "**CLA**"). The current version of the CLA you may find here:
1) https://yandex.ru/legal/cla/?lang=en (in English) and
2) https://yandex.ru/legal/cla/?lang=ru (in Russian).
By adopting the CLA, you state the following:
* You obviously wish and are willingly licensing your contributions to us for our open source projects under the terms of the CLA,
* You has read the terms and conditions of the CLA and agree with them in full,
* You are legally able to provide and license your contributions as stated,
* We may use your contributions for our open source projects and for any other our project too,
* We rely on your assurances concerning the rights of third parties in relation to your contributes.
If you agree with these principles, please read and adopt our CLA. By providing us your contributions, you hereby declare that you has already read and adopt our CLA, and we may freely merge your contributions with our corresponding open source project and use it in further in accordance with terms and conditions of the CLA.
If you have already adopted terms and conditions of the CLA, you are able to provide your contributes. When you submit your pull request, please add the following information into it:
```
I hereby agree to the terms of the CLA available at: [link].
```
Replace the bracketed text as follows:
* [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian).
It is enough to provide us such notification at once.
# Contributing to ClickHouse
## Technical info
Developer guide for writing code for ClickHouse is published on official website alongside the usage and operations documentation:
https://clickhouse.yandex/docs/en/development/index.html
## Legal info
In order for us (YANDEX LLC) to accept patches and other contributions from you, you will have to adopt our Yandex Contributor License Agreement (the "**CLA**"). The current version of the CLA you may find here:
1) https://yandex.ru/legal/cla/?lang=en (in English) and
2) https://yandex.ru/legal/cla/?lang=ru (in Russian).
By adopting the CLA, you state the following:
* You obviously wish and are willingly licensing your contributions to us for our open source projects under the terms of the CLA,
* You has read the terms and conditions of the CLA and agree with them in full,
* You are legally able to provide and license your contributions as stated,
* We may use your contributions for our open source projects and for any other our project too,
* We rely on your assurances concerning the rights of third parties in relation to your contributes.
If you agree with these principles, please read and adopt our CLA. By providing us your contributions, you hereby declare that you has already read and adopt our CLA, and we may freely merge your contributions with our corresponding open source project and use it in further in accordance with terms and conditions of the CLA.
If you have already adopted terms and conditions of the CLA, you are able to provide your contributes. When you submit your pull request, please add the following information into it:
```
I hereby agree to the terms of the CLA available at: [link].
```
Replace the bracketed text as follows:
* [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian).
It is enough to provide us such notification at once.
If you don't agree with the CLA, you still can open a pull request to provide your contributions.

View File

@ -1,5 +1,6 @@
# ClickHouse
ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time.
[Read more...](https://clickhouse.yandex/)
Learn more about ClickHouse at [https://clickhouse.yandex/](https://clickhouse.yandex/)
[![Build Status](https://travis-ci.org/yandex/ClickHouse.svg?branch=master)](https://travis-ci.org/yandex/ClickHouse)

View File

@ -1,35 +0,0 @@
if (ARCH_ARM)
# bundled have some asm broken for arm, use package libzookeeper-mt-dev
set(USE_INTERNAL_ZOOKEEPER_LIBRARY 0 CACHE BOOL "")
endif ()
option (USE_INTERNAL_ZOOKEEPER_LIBRARY "Set to FALSE to use system zookeeper library instead of bundled" ${NOT_UNBUNDLED})
if (USE_INTERNAL_ZOOKEEPER_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/zookeeper/src/c/CMakeLists.txt")
message (WARNING "submodule contrib/zookeeper is missing. to fix try run: \n git submodule update --init --recursive")
set (USE_INTERNAL_ZOOKEEPER_LIBRARY 0)
endif ()
if (NOT USE_INTERNAL_ZOOKEEPER_LIBRARY)
find_library (ZOOKEEPER_LIBRARY zookeeper_mt)
find_path (ZOOKEEPER_INCLUDE_DIR NAMES zookeeper/zookeeper.h PATHS ${ZOOKEEPER_INCLUDE_PATHS})
set(ZOOKEEPER_INCLUDE_DIR "${ZOOKEEPER_INCLUDE_DIR}/zookeeper")
endif ()
if (ZOOKEEPER_LIBRARY AND ZOOKEEPER_INCLUDE_DIR)
else ()
set (USE_INTERNAL_ZOOKEEPER_LIBRARY 1)
set(WANT_CPPUNIT 0 CACHE BOOL "")
set (ZOOKEEPER_LIBRARY zookeeper)
endif ()
message (STATUS "Using zookeeper: ${ZOOKEEPER_INCLUDE_DIR} : ${ZOOKEEPER_LIBRARY}")
# how to make clickhouse branch of https://github.com/ClickHouse-Extras/zookeeper.git :
# git remote add upstream https://github.com/apache/zookeeper.git
# git checkhout upstream/master
# git branch -D clickhouse
# git checkout -b clickhouse
# git merge clickhouse_misc
# git merge clickhouse_706

View File

@ -25,10 +25,6 @@ if (USE_INTERNAL_DOUBLE_CONVERSION_LIBRARY)
add_subdirectory (double-conversion)
endif ()
if (USE_INTERNAL_ZOOKEEPER_LIBRARY)
add_subdirectory (zookeeper-cmake)
endif ()
if (USE_INTERNAL_CITYHASH_LIBRARY)
add_subdirectory (libcityhash)
endif ()

1
contrib/zookeeper vendored

@ -1 +0,0 @@
Subproject commit 438afae5af36c5be9c82d074f43a9bb19e0797c0

View File

@ -192,9 +192,11 @@ endif()
# cli executable
add_executable(cli ${LIBRARY_DIR}/src/cli.c)
target_link_libraries(cli zookeeper)
target_link_libraries(cli $<$<PLATFORM_ID:Linux>:m>)
# load_gen executable
if(WANT_SYNCAPI AND NOT WIN32)
add_executable(load_gen ${LIBRARY_DIR}/src/load_gen.c)
target_link_libraries(load_gen zookeeper)
target_link_libraries(load_gen $<$<PLATFORM_ID:Linux>:m>)
endif()

2
contrib/zstd vendored

@ -1 +1 @@
Subproject commit f4340f46b2387bc8de7d5320c0b83bb1499933ad
Subproject commit 255597502c3a4ef150abc964e376d4202a8c2929

View File

@ -8,7 +8,7 @@
# Требуется clang, желательно наиболее свежий (trunk).
#
# Используется при сборке пакетов.
# Заголовочные файлы записываются в пакет clickhouse-server-base, в директорию /usr/share/clickhouse/headers.
# Заголовочные файлы записываются в пакет clickhouse-common, в директорию /usr/share/clickhouse/headers.
#
# Если вы хотите установить их самостоятельно, без сборки пакета,
# чтобы clickhouse-server видел их там, где ожидается, выполните:

View File

@ -276,6 +276,8 @@ void ConfigProcessor::doIncludesRecursive(
/// Replace the original contents, not add to it.
bool replace = attributes->getNamedItem("replace");
bool included_something = false;
auto process_include = [&](const Node * include_attr, const std::function<const Node * (const std::string &)> & get_node, const char * error_msg)
{
std::string name = include_attr->getNodeValue();
@ -316,6 +318,8 @@ void ConfigProcessor::doIncludesRecursive(
{
element->setAttributeNode(dynamic_cast<Attr *>(config->importNode(from_attrs->item(i), true)));
}
included_something = true;
}
};
@ -348,11 +352,14 @@ void ConfigProcessor::doIncludesRecursive(
}
}
NodeListPtr children = node->childNodes();
Node * child = nullptr;
for (size_t i = 0; (child = children->item(i)); ++i)
if (included_something)
doIncludesRecursive(config, include_from, node, zk_node_cache, contributing_zk_paths);
else
{
doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths);
NodeListPtr children = node->childNodes();
Node * child = nullptr;
for (size_t i = 0; (child = children->item(i)); ++i)
doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths);
}
}

View File

@ -1,15 +1,10 @@
include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake)
add_headers_and_sources(clickhouse_common_zookeeper .)
add_library(clickhouse_common_zookeeper ${SPLIT_SHARED} ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources})
if (NOT USE_INTERNAL_ZOOKEEPER_LIBRARY)
target_include_directories (clickhouse_common_zookeeper BEFORE PUBLIC ${ZOOKEEPER_INCLUDE_DIR})
endif ()
target_link_libraries (clickhouse_common_zookeeper clickhouse_common_io ${ZOOKEEPER_LIBRARY})
target_link_libraries (clickhouse_common_zookeeper clickhouse_common_io)
if (ENABLE_TESTS)
add_subdirectory (tests)

View File

@ -29,11 +29,11 @@ public:
if (zookeeper->tryGet(path, result_str, &stat))
{
result = std::stol(result_str) + 1;
success = zookeeper->trySet(path, std::to_string(result), stat.version) == ZOK;
success = zookeeper->trySet(path, std::to_string(result), stat.version) == ZooKeeperImpl::ZooKeeper::ZOK;
}
else
{
success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZOK;
success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZooKeeperImpl::ZooKeeper::ZOK;
}
}
while (!success);

View File

@ -23,32 +23,23 @@ namespace zkutil
/// You should reinitialize ZooKeeper session in case of these errors
inline bool isUnrecoverableErrorCode(int32_t zk_return_code)
inline bool isHardwareError(int32_t zk_return_code)
{
return zk_return_code == ZINVALIDSTATE || zk_return_code == ZSESSIONEXPIRED || zk_return_code == ZSESSIONMOVED;
}
/// Errors related with temporary network problems
inline bool isTemporaryErrorCode(int32_t zk_return_code)
{
return zk_return_code == ZCONNECTIONLOSS || zk_return_code == ZOPERATIONTIMEOUT;
}
/// Any error related with network or master election
/// In case of these errors you should retry the query or reinitialize ZooKeeper session (see isUnrecoverable())
inline bool isHardwareErrorCode(int32_t zk_return_code)
{
return isUnrecoverableErrorCode(zk_return_code) || isTemporaryErrorCode(zk_return_code);
return zk_return_code == ZooKeeperImpl::ZooKeeper::ZINVALIDSTATE
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONMOVED
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZOPERATIONTIMEOUT;
}
/// Valid errors sent from server
inline bool isUserError(int32_t zk_return_code)
{
return zk_return_code == ZNONODE
|| zk_return_code == ZBADVERSION
|| zk_return_code == ZNOCHILDRENFOREPHEMERALS
|| zk_return_code == ZNODEEXISTS
|| zk_return_code == ZNOTEMPTY;
return zk_return_code == ZooKeeperImpl::ZooKeeper::ZNONODE
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZBADVERSION
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS
|| zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY;
}
@ -60,12 +51,11 @@ private:
: DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code) { incrementEventCounter(); }
public:
explicit KeeperException(const std::string & msg) : KeeperException(msg, ZOK, 0) {}
KeeperException(const std::string & msg, const int32_t code)
: KeeperException(msg + " (" + zerror(code) + ")", code, 0) {}
explicit KeeperException(const int32_t code) : KeeperException(zerror(code), code, 0) {}
: KeeperException(msg + " (" + ZooKeeperImpl::ZooKeeper::errorMessage(code) + ")", code, 0) {}
explicit KeeperException(const int32_t code) : KeeperException(ZooKeeperImpl::ZooKeeper::errorMessage(code), code, 0) {}
KeeperException(const int32_t code, const std::string & path)
: KeeperException(std::string{zerror(code)} + ", path: " + path, code, 0) {}
: KeeperException(std::string{ZooKeeperImpl::ZooKeeper::errorMessage(code)} + ", path: " + path, code, 0) {}
KeeperException(const KeeperException & exc) : DB::Exception(exc), code(exc.code) { incrementEventCounter(); }
@ -73,23 +63,11 @@ public:
const char * className() const throw() override { return "zkutil::KeeperException"; }
KeeperException * clone() const override { return new KeeperException(*this); }
/// You should reinitialize ZooKeeper session in case of these errors
bool isUnrecoverable() const
{
return isUnrecoverableErrorCode(code);
}
/// Errors related with temporary network problems
bool isTemporaryError() const
{
return isTemporaryErrorCode(code);
}
/// Any error related with network or master election
/// In case of these errors you should retry the query or reinitialize ZooKeeper session (see isUnrecoverable())
/// In case of these errors you should reinitialize ZooKeeper session.
bool isHardwareError() const
{
return isHardwareErrorCode(code);
return zkutil::isHardwareError(code);
}
const int32_t code;
@ -106,15 +84,20 @@ private:
class KeeperMultiException : public KeeperException
{
public:
MultiTransactionInfo info;
Requests requests;
Responses responses;
size_t failed_op_index = 0;
std::string getPathForFirstFailedOp() const;
/// If it is user error throws KeeperMultiException else throws ordinary KeeperException
/// If it is ZOK does nothing
static void check(const MultiTransactionInfo & info);
static void check(int code, const Ops & ops, const OpResultsPtr & op_results);
static void check(int32_t code, const Requests & requests, const Responses & responses);
protected:
KeeperMultiException(const MultiTransactionInfo & info, size_t failed_op_index);
KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses);
private:
size_t getFailedOpIndex(int32_t code, const Responses & responses) const;
};
};

View File

@ -76,41 +76,9 @@ private:
std::string node_path = node->getPath();
node_name = node_path.substr(node_path.find_last_of('/') + 1);
cleanOldEphemeralNodes();
thread = std::thread(&LeaderElection::threadFunction, this);
}
void cleanOldEphemeralNodes()
{
if (identifier.empty())
return;
/** If there are nodes with same identifier, remove them.
* Such nodes could still be alive after failed attempt of removal,
* if it was temporary communication failure, that was continued for more than session timeout,
* but ZK session is still alive for unknown reason, and someone still holds that ZK session.
* See comments in destructor of EphemeralNodeHolder.
*/
Strings brothers = zookeeper.getChildren(path);
for (const auto & brother : brothers)
{
if (brother == node_name)
continue;
std::string brother_path = path + "/" + brother;
std::string brother_identifier = zookeeper.get(brother_path);
if (brother_identifier == identifier)
{
ProfileEvents::increment(ProfileEvents::ObsoleteEphemeralNode);
LOG_WARNING(&Logger::get("LeaderElection"), "Found obsolete ephemeral node for identifier "
+ identifier + ", removing: " + brother_path);
zookeeper.tryRemoveWithRetries(brother_path);
}
}
}
void releaseNode()
{
shutdown = true;

View File

@ -11,31 +11,16 @@ bool Lock::tryLock()
if (tryCheck() != Status::LOCKED_BY_ME)
locked.reset(nullptr);
}
if (!locked)
else
{
size_t attempt;
std::string dummy;
int32_t code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy);
/// TODO: ошибка. можно создать эфемерную ноду, но при этом не получить подтверждения даже после нескольких попыток.
/// тогда все последующие локи будут неуспешные из-за существования ноды.
int32_t code = zookeeper->tryCreateWithRetries(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy, &attempt);
if (code == ZNODEEXISTS)
if (code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
{
if (attempt == 0)
locked.reset(nullptr);
else
{
zkutil::Stat stat;
zookeeper->get(lock_path, &stat);
if (stat.ephemeralOwner == zookeeper->getClientID())
locked.reset(new ZooKeeperHandler(zookeeper));
else
locked.reset(nullptr);
}
locked.reset(nullptr);
}
else if (code == ZOK)
else if (code == ZooKeeperImpl::ZooKeeper::ZOK)
{
locked.reset(new ZooKeeperHandler(zookeeper));
}
@ -52,34 +37,8 @@ void Lock::unlock()
if (locked)
{
auto zookeeper = zookeeper_holder->getZooKeeper();
try
{
if (tryCheck() == Status::LOCKED_BY_ME)
{
size_t attempt;
int32_t code = zookeeper->tryRemoveEphemeralNodeWithRetries(lock_path, -1, &attempt);
if (attempt)
{
if (code != ZOK)
throw zkutil::KeeperException(code);
}
else
{
if (code == ZNONODE)
LOG_ERROR(log, "Node " << lock_path << " has been already removed. Probably due to network error.");
else if (code != ZOK)
throw zkutil::KeeperException(code);
}
}
}
catch (const zkutil::KeeperException & e)
{
/// если сессия находится в невостанавливаемом состоянии, то эфемерные ноды нам больше не принадлежат
/// и лок через таймаут будет отпущен
if (!e.isUnrecoverable())
throw;
}
if (tryCheck() == Status::LOCKED_BY_ME)
zookeeper->remove(lock_path, -1);
locked.reset(nullptr);
}
}
@ -97,47 +56,17 @@ Lock::Status Lock::tryCheck() const
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 reinitialized. Path: " << lock_path << "/" << lock_message);
LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: " << lock_path << "/" << lock_message);
return lock_status;
}
std::string Lock::status2String(Status status)
{
if (status >= END)
throw zkutil::KeeperException("Wrong status code: " + std::to_string(status));
static const char * names[] = {"Unlocked", "Locked by me", "Locked by other"};
return names[status];
}
void Lock::unlockOrMoveIfFailed(std::vector<zkutil::Lock> & failed_to_unlock_locks)
{
try
{
unlock();
}
catch (const zkutil::KeeperException & e)
{
if (e.isTemporaryError())
{
LOG_WARNING(log, "Fail to unlock lock. Move lock to vector to remove later. Path: " << getPath());
failed_to_unlock_locks.emplace_back(std::move(*this));
}
else
throw;
}
}
void Lock::unlockAssumeLockNodeRemovedManually()
{
locked.reset(nullptr);

View File

@ -51,15 +51,12 @@ namespace zkutil
UNLOCKED,
LOCKED_BY_ME,
LOCKED_BY_OTHER,
END
};
std::string status2String(Status status);
/// проверяет создана ли эфемерная нода и кто ее владелец.
Status tryCheck() const;
void unlock();
void unlockOrMoveIfFailed(std::vector<zkutil::Lock> & failed_to_unlock_locks);
void unlockAssumeLockNodeRemovedManually();
bool tryLock();

View File

@ -3,184 +3,17 @@
#include <future>
#include <memory>
#include <vector>
#include <zookeeper.h>
#include <Common/ZooKeeper/ZooKeeperImpl.h>
#include <Poco/Event.h>
namespace zkutil
{
using ACLPtr = const ACL_vector *;
using Stat = ::Stat;
class ZooKeeper;
struct Op
{
public:
Op() : data(new zoo_op_t) {}
virtual ~Op() {}
virtual std::shared_ptr<Op> clone() const = 0;
virtual std::string getPath() const = 0;
virtual std::string describe() const = 0;
std::unique_ptr<zoo_op_t> data;
struct Remove;
struct Create;
struct SetData;
struct Check;
};
using OpPtr = std::shared_ptr<Op>;
struct Op::Remove : public Op
{
Remove(const std::string & path_, int32_t version_) :
path(path_), version(version_)
{
zoo_delete_op_init(data.get(), path.c_str(), version);
}
OpPtr clone() const override
{
return std::make_shared<Remove>(path, version);
}
std::string getPath() const override { return path; }
std::string describe() const override { return "command: remove, path: " + path; }
private:
std::string path;
int32_t version;
};
struct Op::Create : public Op
{
Create(const std::string & path_pattern_, const std::string & value_, ACLPtr acl_, int32_t flags_);
OpPtr clone() const override
{
return std::make_shared<Create>(path_pattern, value, acl, flags);
}
std::string getPathCreated() { return created_path.data(); }
std::string getPath() const override { return path_pattern; }
std::string describe() const override
{
return "command: create"
", path: " + path_pattern +
", value: " + value;
}
private:
std::string path_pattern;
std::string value;
ACLPtr acl;
int32_t flags;
std::vector<char> created_path;
};
struct Op::SetData : public Op
{
SetData(const std::string & path_, const std::string & value_, int32_t version_) :
path(path_), value(value_), version(version_)
{
zoo_set_op_init(data.get(), path.c_str(), value.c_str(), value.size(), version, &stat);
}
OpPtr clone() const override
{
return std::make_shared<SetData>(path, value, version);
}
std::string getPath() const override { return path; }
std::string describe() const override
{
return
"command: set"
", path: " + path +
", value: " + value +
", version: " + std::to_string(data->set_op.version);
}
private:
std::string path;
std::string value;
int32_t version;
Stat stat;
};
struct Op::Check : public Op
{
Check(const std::string & path_, int32_t version_) :
path(path_), version(version_)
{
zoo_check_op_init(data.get(), path.c_str(), version);
}
OpPtr clone() const override
{
return std::make_shared<Check>(path, version);
}
std::string getPath() const override { return path; }
std::string describe() const override { return "command: check, path: " + path; }
private:
std::string path;
int32_t version;
};
using Ops = std::vector<OpPtr>;
/// C++ version of zoo_op_result_t
struct OpResult
{
int err;
std::string value;
std::unique_ptr<Stat> stat;
/// ZooKeeper is required for correct chroot path prefixes handling
explicit OpResult(const zoo_op_result_t & op_result, const ZooKeeper * zookeeper = nullptr);
};
using OpResults = std::vector<OpResult>;
using OpResultsPtr = std::shared_ptr<OpResults>;
using Stat = ZooKeeperImpl::ZooKeeper::Stat;
using Strings = std::vector<std::string>;
/// Simple structure to handle transaction execution results
struct MultiTransactionInfo
{
Ops ops;
int32_t code = ZOK;
OpResultsPtr op_results;
MultiTransactionInfo() = default;
MultiTransactionInfo(int32_t code_, const Ops & ops_, const OpResultsPtr & op_results_)
: ops(ops_), code(code_), op_results(op_results_) {}
bool empty() const
{
return ops.empty();
}
/// Returns failed op if zkutil::isUserError(code) is true
const Op & getFailedOp() const;
};
namespace CreateMode
{
extern const int Persistent;
@ -191,23 +24,39 @@ namespace CreateMode
using EventPtr = std::shared_ptr<Poco::Event>;
class ZooKeeper;
/// Callback to call when the watch fires.
/// Because callbacks are called in the single "completion" thread internal to libzookeeper,
/// they must execute as quickly as possible (preferably just set some notification).
/// Parameters:
/// zookeeper - zookeeper session to which the fired watch belongs
/// type - event type, one of the *_EVENT constants from zookeeper.h
/// state - session connection state, one of the *_STATE constants from zookeeper.h
/// path - znode path to which the change happened. if event == ZOO_SESSION_EVENT it is either NULL or empty string.
using WatchCallback = std::function<void(ZooKeeper & zookeeper, int type, int state, const char * path)>;
using WatchCallback = ZooKeeperImpl::ZooKeeper::WatchCallback;
using Request = ZooKeeperImpl::ZooKeeper::Request;
using Response = ZooKeeperImpl::ZooKeeper::Response;
/// Returns first op which code != ZOK or throws an exception
/// ZooKeeper client sets correct OP codes if the transaction fails because of logical (user) errors like ZNODEEXISTS
/// If it is failed because of network error, for example, OP codes is not set.
/// Therefore you should make zkutil::isUserError() check before the function invocation.
size_t getFailedOpIndex(const OpResultsPtr & op_results, int32_t transaction_return_code);
using RequestPtr = ZooKeeperImpl::ZooKeeper::RequestPtr;
using ResponsePtr = ZooKeeperImpl::ZooKeeper::ResponsePtr;
using Requests = ZooKeeperImpl::ZooKeeper::Requests;
using Responses = ZooKeeperImpl::ZooKeeper::Responses;
using CreateRequest = ZooKeeperImpl::ZooKeeper::CreateRequest;
using RemoveRequest = ZooKeeperImpl::ZooKeeper::RemoveRequest;
using ExistsRequest = ZooKeeperImpl::ZooKeeper::ExistsRequest;
using GetRequest = ZooKeeperImpl::ZooKeeper::GetRequest;
using SetRequest = ZooKeeperImpl::ZooKeeper::SetRequest;
using ListRequest = ZooKeeperImpl::ZooKeeper::ListRequest;
using CheckRequest = ZooKeeperImpl::ZooKeeper::CheckRequest;
using CreateResponse = ZooKeeperImpl::ZooKeeper::CreateResponse;
using RemoveResponse = ZooKeeperImpl::ZooKeeper::RemoveResponse;
using ExistsResponse = ZooKeeperImpl::ZooKeeper::ExistsResponse;
using GetResponse = ZooKeeperImpl::ZooKeeper::GetResponse;
using SetResponse = ZooKeeperImpl::ZooKeeper::SetResponse;
using ListResponse = ZooKeeperImpl::ZooKeeper::ListResponse;
using CheckResponse = ZooKeeperImpl::ZooKeeper::CheckResponse;
RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode);
RequestPtr makeRemoveRequest(const std::string & path, int version);
RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
RequestPtr makeCheckRequest(const std::string & path, int version);
}

File diff suppressed because it is too large Load Diff

View File

@ -29,14 +29,10 @@ namespace zkutil
{
const UInt32 DEFAULT_SESSION_TIMEOUT = 30000;
const UInt32 MEDIUM_SESSION_TIMEOUT = 120000;
const UInt32 BIG_SESSION_TIMEOUT = 600000;
/// Preferred size of multi() command (in number of ops)
constexpr size_t MULTI_BATCH_SIZE = 100;
struct WatchContext;
struct MultiTransactionInfo;
/// ZooKeeper session. The interface is substantially different from the usual libzookeeper API.
///
@ -77,27 +73,15 @@ public:
*/
ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name);
~ZooKeeper();
/// Creates a new session with the same parameters. This method can be used for reconnecting
/// after the session has expired.
/// This object remains unchanged, and the new session is returned.
Ptr startNewSession() const;
/// Returns true, if the session has expired forever.
/// This is possible only if the connection has been established, then lost and re-established
/// again, but too late.
/// In contrast, if, for instance, the server name or port is misconfigured, connection
/// attempts will continue indefinitely, expired() will return false and all method calls
/// will raise ConnectionLoss exception.
/// Also returns true if is_dirty flag is set - a request to close the session ASAP.
/// Returns true, if the session has expired.
bool expired();
ACLPtr getDefaultACL();
void setDefaultACL(ACLPtr new_acl);
/// Create a znode. ACL set by setDefaultACL is used (full access to everybody by default).
/// Create a znode.
/// Throw an exception if something went wrong.
std::string create(const std::string & path, const std::string & data, int32_t mode);
@ -108,12 +92,9 @@ public:
/// In case of other errors throws an exception.
int32_t tryCreate(const std::string & path, const std::string & data, int32_t mode, std::string & path_created);
int32_t tryCreate(const std::string & path, const std::string & data, int32_t mode);
int32_t tryCreateWithRetries(const std::string & path, const std::string & data, int32_t mode,
std::string & path_created, size_t * attempt = nullptr);
/// Create a Persistent node.
/// Does nothing if the node already exists.
/// Retries on ConnectionLoss or OperationTimeout.
void createIfNotExists(const std::string & path, const std::string & data);
/// Creates all non-existent ancestors of the given path with empty contents.
@ -123,42 +104,11 @@ public:
/// Remove the node if the version matches. (if version == -1, remove any version).
void remove(const std::string & path, int32_t version = -1);
/// Removes the node. In case of network errors tries to remove again.
/// ZNONODE error for the second and the following tries is ignored.
void removeWithRetries(const std::string & path, int32_t version = -1);
/// Doesn't throw in the following cases:
/// * The node doesn't exist
/// * Versions don't match
/// * The node has children.
int32_t tryRemove(const std::string & path, int32_t version = -1);
/// Retries in case of network errors, returns ZNONODE if the node is already removed.
int32_t tryRemoveWithRetries(const std::string & path, int32_t version = -1, size_t * attempt = nullptr);
/// The same, but sets is_dirty flag if all removal attempts were unsuccessful.
/// This is needed because the session might still exist after all retries,
/// even if more time than session_timeout has passed.
/// So we do not rely on the ephemeral node being deleted and set is_dirty to
/// try and close the session ASAP.
/** Ridiculously Long Delay to Expire
When disconnects do happen, the common case should be a very* quick
reconnect to another server, but an extended network outage may
introduce a long delay before a client can reconnect to the ZooKeep
er service. Some developers wonder why the ZooKeeper client li
brary doesnt simply decide at some point (perhaps twice the session
timeout) that enough is enough and kill the session itself.
There are two answers to this. First, ZooKeeper leaves this kind of
policy decision up to the developer. Developers can easily implement
such a policy by closing the handle themselves. Second, when a Zoo
Keeper ensemble goes down, time freezes. Thus, when the ensemble is
brought back up, session timeouts are restarted. If processes using
ZooKeeper hang in there, they may find out that the long timeout was
due to an extended ensemble failure that has recovered and pick right
up where they left off without any additional startup delay.
ZooKeeper: Distributed Process Coordination p118
*/
int32_t tryRemoveEphemeralNodeWithRetries(const std::string & path, int32_t version = -1, size_t * attempt = nullptr);
bool exists(const std::string & path, Stat * stat = nullptr, const EventPtr & watch = nullptr);
bool existsWatch(const std::string & path, Stat * stat, const WatchCallback & watch_callback);
@ -195,16 +145,14 @@ public:
/// Performs several operations in a transaction.
/// Throws on every error.
OpResultsPtr multi(const Ops & ops);
Responses multi(const Requests & requests);
/// Throws only if some operation has returned an "unexpected" error
/// - an error that would cause the corresponding try- method to throw.
int32_t tryMulti(const Ops & ops, OpResultsPtr * out_results = nullptr);
/// Use only with read-only operations.
int32_t tryMultiWithRetries(const Ops & ops, OpResultsPtr * out_results = nullptr, size_t * attempt = nullptr);
int32_t tryMulti(const Requests & requests, Responses & responses);
/// Throws nothing, just alias of multiImpl
int32_t tryMultiNoThrow(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr)
int32_t tryMultiNoThrow(const Requests & requests, Responses & responses)
{
return multiImpl(ops, out_op_results, out_info);
return multiImpl(requests, responses);
}
Int64 getClientID();
@ -237,182 +185,47 @@ public:
///
/// Future should not be destroyed before the result is gotten.
template <typename Result, typename... TaskParams>
class Future
{
friend class ZooKeeper;
private:
using Task = std::packaged_task<Result (TaskParams...)>;
using TaskPtr = std::unique_ptr<Task>;
using TaskPtrPtr = std::unique_ptr<TaskPtr>;
std::future<ZooKeeperImpl::ZooKeeper::GetResponse> asyncGet(const std::string & path);
/// Everything is complicated.
///
/// In libzookeeper async interface a function (e.g. zoo_aget)
/// accepts a pointer to a standalone callback function and void* pointer to the context
/// which is then passed to the callback.
/// The caller is responsible for ensuring that the context lives until the callback
/// is finished and we can't simply pass ownership of the context into function object.
/// Instead, we save the context in a Future object and return it to the caller.
/// The context will live until the Future lives.
/// Context data is wrapped in an unique_ptr so that its address (which is passed to
/// libzookeeper) remains unchanged after the Future is returned from the function.
///
/// The second problem is that after std::promise has been fulfilled, and the user
/// has gotten the result from std::future, the Future object can be destroyed
/// before the std::promise::set_value() call that fulfils the promise completes in another
/// thread.
/// See http://stackoverflow.com/questions/10843304/race-condition-in-pthread-once
/// To combat this we use the second unique_ptr. Inside the callback, the void* context
/// is cast to unique_ptr and moved into the local unique_ptr to prolong the lifetime of
/// the context data.
std::future<ZooKeeperImpl::ZooKeeper::GetResponse> asyncTryGet(const std::string & path);
TaskPtrPtr task;
std::future<Result> future;
std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse> asyncExists(const std::string & path);
template <typename... Args>
Future(Args &&... args) :
task(std::make_unique<TaskPtr>(std::make_unique<Task>(std::forward<Args>(args)...))),
future((*task)->get_future()) {}
std::future<ZooKeeperImpl::ZooKeeper::ListResponse> asyncGetChildren(const std::string & path);
public:
Result get()
{
return future.get();
}
Future(Future &&) = default;
Future & operator= (Future &&) = default;
~Future()
{
/// If nobody has waited for the result, we must wait for it before the object is
/// destroyed, because the object contents can still be used in the callback.
if (future.valid())
future.wait();
}
};
struct ValueAndStat
{
std::string value;
Stat stat;
};
using GetFuture = Future<ValueAndStat, int, const char *, int, const Stat *>;
GetFuture asyncGet(const std::string & path);
struct ValueAndStatAndExists
{
std::string value;
Stat stat;
bool exists;
};
using TryGetFuture = Future<ValueAndStatAndExists, int, const char *, int, const Stat *>;
TryGetFuture asyncTryGet(const std::string & path);
struct StatAndExists
{
Stat stat;
bool exists;
};
using ExistsFuture = Future<StatAndExists, int, const Stat *>;
ExistsFuture asyncExists(const std::string & path);
using GetChildrenFuture = Future<Strings, int, const String_vector *>;
GetChildrenFuture asyncGetChildren(const std::string & path);
using RemoveFuture = Future<void, int>;
RemoveFuture asyncRemove(const std::string & path, int32_t version = -1);
std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> asyncRemove(const std::string & path, int32_t version = -1);
/// Doesn't throw in the following cases:
/// * The node doesn't exist
/// * The versions do not match
/// * The node has children
using TryRemoveFuture = Future<int32_t, int>;
TryRemoveFuture asyncTryRemove(const std::string & path, int32_t version = -1);
std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> asyncTryRemove(const std::string & path, int32_t version = -1);
struct OpResultsAndCode
{
OpResultsPtr results;
std::shared_ptr<Ops> ops_ptr;
int code;
};
std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> asyncMulti(const Requests & ops);
using MultiFuture = Future<OpResultsAndCode, int>;
MultiFuture asyncMulti(const Ops & ops);
/// Like the previous one but don't throw any exceptions on future.get()
MultiFuture tryAsyncMulti(const Ops & ops);
std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> tryAsyncMulti(const Requests & ops);
static std::string error2string(int32_t code);
/// Max size of node contents in bytes.
/// In 3.4.5 max node size is 1Mb.
static const size_t MAX_NODE_SIZE = 1048576;
/// Length of the suffix that ZooKeeper adds to sequential nodes.
/// In fact it is smaller, but round it up for convenience.
static const size_t SEQUENTIAL_SUFFIX_SIZE = 64;
zhandle_t * getHandle() { return impl; }
private:
friend struct WatchContext;
friend class EphemeralNodeHolder;
friend struct OpResult;
void init(const std::string & hosts, const std::string & identity, int32_t session_timeout_ms, const std::string & chroot);
void init(const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_, const std::string & chroot_);
void removeChildrenRecursive(const std::string & path);
void tryRemoveChildrenRecursive(const std::string & path);
static WatchCallback callbackForEvent(const EventPtr & event);
WatchContext * createContext(WatchCallback && callback);
static void destroyContext(WatchContext * context);
static void processCallback(zhandle_t * zh, int type, int state, const char * path, void * watcher_ctx);
template <typename T>
int32_t retry(T && operation, size_t * attempt = nullptr)
{
int32_t code = operation();
if (attempt)
*attempt = 0;
for (size_t i = 0; (i < retry_num) && (code == ZOPERATIONTIMEOUT || code == ZCONNECTIONLOSS); ++i)
{
if (attempt)
*attempt = i;
/// If the connection has been lost, wait timeout/3 hoping for connection re-establishment.
static const int MAX_SLEEP_TIME = 10;
if (code == ZCONNECTIONLOSS)
usleep(std::min(session_timeout_ms * 1000u / 3, MAX_SLEEP_TIME * 1000u * 1000u));
LOG_WARNING(log, "Error on attempt " << i << ": " << error2string(code) << ". Retry");
code = operation();
}
return code;
}
/// The following methods don't throw exceptions but return error codes.
int32_t createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created);
int32_t removeImpl(const std::string & path, int32_t version = -1);
int32_t removeImpl(const std::string & path, int32_t version);
int32_t getImpl(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback);
int32_t setImpl(const std::string & path, const std::string & data, int32_t version = -1, Stat * stat = nullptr);
int32_t setImpl(const std::string & path, const std::string & data, int32_t version, Stat * stat);
int32_t getChildrenImpl(const std::string & path, Strings & res, Stat * stat, WatchCallback watch_callback);
int32_t multiImpl(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr);
int32_t multiImpl(const Requests & requests, Responses & responses);
int32_t existsImpl(const std::string & path, Stat * stat_, WatchCallback watch_callback);
MultiFuture asyncMultiImpl(const zkutil::Ops & ops_, bool throw_exception);
std::unique_ptr<ZooKeeperImpl::ZooKeeper> impl;
std::string hosts;
std::string identity;
@ -420,19 +233,8 @@ private:
std::string chroot;
std::mutex mutex;
ACLPtr default_acl;
zhandle_t * impl;
std::unordered_set<WatchContext *> watch_context_store;
/// Retries number in case of OperationTimeout or ConnectionLoss errors.
static constexpr size_t retry_num = 3;
Logger * log = nullptr;
/// If true, there were unsuccessfull attempts to remove ephemeral nodes.
/// It is better to close the session to remove ephemeral nodes with certainty
/// instead of continuing to use re-established session.
bool is_dirty = false;
};
@ -476,11 +278,7 @@ public:
{
try
{
/// Important: if the ZooKeeper is temporarily unavailable, repeated attempts to
/// delete the node are made.
/// Otherwise it is possible that EphemeralNodeHolder is destroyed,
/// but the session has recovered and the node in ZooKeeper remains for the long time.
zookeeper.tryRemoveEphemeralNodeWithRetries(path);
zookeeper.tryRemove(path);
}
catch (const KeeperException & e)
{

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,548 @@
#pragma once
#include <Core/Types.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBufferFromPocoSocket.h>
#include <IO/WriteBufferFromPocoSocket.h>
#include <Poco/Net/StreamSocket.h>
#include <Poco/Net/SocketAddress.h>
#include <map>
#include <mutex>
#include <chrono>
#include <vector>
#include <memory>
#include <thread>
#include <atomic>
#include <cstdint>
#include <optional>
#include <functional>
namespace ZooKeeperImpl
{
using namespace DB;
/** Usage scenario:
* - create an object and issue commands;
* - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap:
* for example, just signal a condvar / fulfull a promise.
* - you also may provide callbacks for watches; they are also invoked in internal thread and must be cheap.
* - whenever you receive SessionExpired exception of method isValid returns false,
* the ZooKeeper instance is no longer usable - you may only destroy it and probably create another.
* - whenever session is expired or ZooKeeper instance is destroying, all callbacks are notified with special event.
* - data for callbacks must be alive when ZooKeeper instance is alive.
*/
class ZooKeeper
{
public:
using Addresses = std::vector<Poco::Net::SocketAddress>;
struct ACL
{
static constexpr int32_t Read = 1;
static constexpr int32_t Write = 2;
static constexpr int32_t Create = 4;
static constexpr int32_t Delete = 8;
static constexpr int32_t Admin = 16;
static constexpr int32_t All = 0x1F;
int32_t permissions;
String scheme;
String id;
void write(WriteBuffer & out) const;
};
using ACLs = std::vector<ACL>;
struct Stat
{
int64_t czxid;
int64_t mzxid;
int64_t ctime;
int64_t mtime;
int32_t version;
int32_t cversion;
int32_t aversion;
int64_t ephemeralOwner;
int32_t dataLength;
int32_t numChildren;
int64_t pzxid;
void read(ReadBuffer & in);
};
using XID = int32_t;
using OpNum = int32_t;
struct Response
{
int32_t error = 0;
virtual ~Response() {}
virtual void readImpl(ReadBuffer &) = 0;
virtual void removeRootPath(const String & /* root_path */) {}
};
using ResponsePtr = std::shared_ptr<Response>;
using Responses = std::vector<ResponsePtr>;
using ResponseCallback = std::function<void(const Response &)>;
struct Request
{
XID xid = 0;
bool has_watch = false;
virtual ~Request() {};
virtual OpNum getOpNum() const = 0;
/// Writes length, xid, op_num, then the rest.
void write(WriteBuffer & out) const;
virtual void writeImpl(WriteBuffer &) const = 0;
virtual ResponsePtr makeResponse() const = 0;
virtual void addRootPath(const String & /* root_path */) {};
virtual String getPath() const = 0;
};
using RequestPtr = std::shared_ptr<Request>;
using Requests = std::vector<RequestPtr>;
struct HeartbeatRequest final : Request
{
OpNum getOpNum() const override { return 11; }
void writeImpl(WriteBuffer &) const override {}
ResponsePtr makeResponse() const override;
String getPath() const override { return {}; }
};
struct HeartbeatResponse final : Response
{
void readImpl(ReadBuffer &) override {}
};
struct WatchResponse final : Response
{
int32_t type = 0;
int32_t state = 0;
String path;
void readImpl(ReadBuffer &) override;
void removeRootPath(const String & root_path) override;
};
using WatchCallback = std::function<void(const WatchResponse &)>;
struct AuthRequest final : Request
{
int32_t type = 0; /// ignored by the server
String scheme;
String data;
OpNum getOpNum() const override { return 100; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
String getPath() const override { return {}; }
};
struct AuthResponse final : Response
{
void readImpl(ReadBuffer &) override {};
};
struct CloseRequest final : Request
{
OpNum getOpNum() const override { return -11; }
void writeImpl(WriteBuffer &) const override {}
ResponsePtr makeResponse() const override;
String getPath() const override { return {}; }
};
struct CloseResponse final : Response
{
void readImpl(ReadBuffer &) override;
};
struct CreateRequest final : Request
{
String path;
String data;
bool is_ephemeral = false;
bool is_sequential = false;
ACLs acls;
OpNum getOpNum() const override { return 1; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct CreateResponse final : Response
{
String path_created;
void readImpl(ReadBuffer &) override;
void removeRootPath(const String & root_path) override;
};
struct RemoveRequest final : Request
{
String path;
int32_t version = -1;
OpNum getOpNum() const override { return 2; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct RemoveResponse final : Response
{
void readImpl(ReadBuffer &) override {}
};
struct ExistsRequest final : Request
{
String path;
OpNum getOpNum() const override { return 3; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct ExistsResponse final : Response
{
Stat stat;
void readImpl(ReadBuffer &) override;
};
struct GetRequest final : Request
{
String path;
OpNum getOpNum() const override { return 4; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct GetResponse final : Response
{
String data;
Stat stat;
void readImpl(ReadBuffer &) override;
};
struct SetRequest final : Request
{
String path;
String data;
int32_t version = -1;
OpNum getOpNum() const override { return 5; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct SetResponse final : Response
{
Stat stat;
void readImpl(ReadBuffer &) override;
};
struct ListRequest final : Request
{
String path;
OpNum getOpNum() const override { return 12; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct ListResponse final : Response
{
std::vector<String> names;
Stat stat;
void readImpl(ReadBuffer &) override;
};
struct CheckRequest final : Request
{
String path;
int32_t version = -1;
OpNum getOpNum() const override { return 13; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct CheckResponse final : Response
{
void readImpl(ReadBuffer &) override {};
};
struct MultiRequest final : Request
{
Requests requests;
OpNum getOpNum() const override { return 14; }
void writeImpl(WriteBuffer &) const override;
ResponsePtr makeResponse() const override;
void addRootPath(const String & root_path) override;
String getPath() const override { return {}; }
};
struct MultiResponse final : Response
{
Responses responses;
MultiResponse(const Requests & requests);
void readImpl(ReadBuffer &) override;
void removeRootPath(const String & root_path) override;
};
/// This response may be received only as an element of responses in MultiResponse.
struct ErrorResponse final : Response
{
void readImpl(ReadBuffer &) override;
};
/** Connection to addresses is performed in order. If you want, shuffle them manually.
* Operation timeout couldn't be greater than session timeout.
* Operation timeout applies independently for network read, network write, waiting for events and synchronization.
*/
ZooKeeper(
const Addresses & addresses,
const String & root_path,
const String & auth_scheme,
const String & auth_data,
Poco::Timespan session_timeout,
Poco::Timespan connection_timeout,
Poco::Timespan operation_timeout);
~ZooKeeper();
/// If expired, you can only destroy the object. All other methods will throw exception.
bool isExpired() const { return expired; }
/// Useful to check owner of ephemeral node.
int64_t getSessionID() const { return session_id; }
using CreateCallback = std::function<void(const CreateResponse &)>;
using RemoveCallback = std::function<void(const RemoveResponse &)>;
using ExistsCallback = std::function<void(const ExistsResponse &)>;
using GetCallback = std::function<void(const GetResponse &)>;
using SetCallback = std::function<void(const SetResponse &)>;
using ListCallback = std::function<void(const ListResponse &)>;
using CheckCallback = std::function<void(const CheckResponse &)>;
using MultiCallback = std::function<void(const MultiResponse &)>;
void create(
const String & path,
const String & data,
bool is_ephemeral,
bool is_sequential,
const ACLs & acls,
CreateCallback callback);
void remove(
const String & path,
int32_t version,
RemoveCallback callback);
void exists(
const String & path,
ExistsCallback callback,
WatchCallback watch);
void get(
const String & path,
GetCallback callback,
WatchCallback watch);
void set(
const String & path,
const String & data,
int32_t version,
SetCallback callback);
void list(
const String & path,
ListCallback callback,
WatchCallback watch);
void check(
const String & path,
int32_t version,
CheckCallback callback);
void multi(
const Requests & requests,
MultiCallback callback);
enum Error
{
ZOK = 0,
/** System and server-side errors.
* This is never thrown by the server, it shouldn't be used other than
* to indicate a range. Specifically error codes greater than this
* value, but lesser than ZAPIERROR, are system errors.
*/
ZSYSTEMERROR = -1,
ZRUNTIMEINCONSISTENCY = -2, /// A runtime inconsistency was found
ZDATAINCONSISTENCY = -3, /// A data inconsistency was found
ZCONNECTIONLOSS = -4, /// Connection to the server has been lost
ZMARSHALLINGERROR = -5, /// Error while marshalling or unmarshalling data
ZUNIMPLEMENTED = -6, /// Operation is unimplemented
ZOPERATIONTIMEOUT = -7, /// Operation timeout
ZBADARGUMENTS = -8, /// Invalid arguments
ZINVALIDSTATE = -9, /// Invliad zhandle state
/** API errors.
* This is never thrown by the server, it shouldn't be used other than
* to indicate a range. Specifically error codes greater than this
* value are API errors.
*/
ZAPIERROR = -100,
ZNONODE = -101, /// Node does not exist
ZNOAUTH = -102, /// Not authenticated
ZBADVERSION = -103, /// Version conflict
ZNOCHILDRENFOREPHEMERALS = -108, /// Ephemeral nodes may not have children
ZNODEEXISTS = -110, /// The node already exists
ZNOTEMPTY = -111, /// The node has children
ZSESSIONEXPIRED = -112, /// The session has been expired by the server
ZINVALIDCALLBACK = -113, /// Invalid callback specified
ZINVALIDACL = -114, /// Invalid ACL specified
ZAUTHFAILED = -115, /// Client authentication failed
ZCLOSING = -116, /// ZooKeeper is closing
ZNOTHING = -117, /// (not error) no server responses to process
ZSESSIONMOVED = -118 /// Session moved to another server, so operation is ignored
};
static const char * errorMessage(int32_t code);
/// For watches.
enum State
{
EXPIRED_SESSION = -112,
AUTH_FAILED = -113,
CONNECTING = 1,
ASSOCIATING = 2,
CONNECTED = 3,
NOTCONNECTED = 999
};
enum Event
{
CREATED = 1,
DELETED = 2,
CHANGED = 3,
CHILD = 4,
SESSION = -1,
NOTWATCHING = -2
};
private:
String root_path;
ACLs default_acls;
Poco::Timespan session_timeout;
Poco::Timespan operation_timeout;
Poco::Net::StreamSocket socket;
std::optional<ReadBufferFromPocoSocket> in;
std::optional<WriteBufferFromPocoSocket> out;
int64_t session_id = 0;
std::atomic<XID> xid {1};
using clock = std::chrono::steady_clock;
struct RequestInfo
{
RequestPtr request;
ResponseCallback callback;
WatchCallback watch;
clock::time_point time;
};
using RequestsQueue = ConcurrentBoundedQueue<RequestPtr>;
RequestsQueue requests{1};
void pushRequest(RequestInfo && request);
using Operations = std::map<XID, RequestInfo>;
Operations operations;
std::mutex operations_mutex;
using WatchCallbacks = std::vector<WatchCallback>;
using Watches = std::map<String /* path */, WatchCallbacks>;
Watches watches;
std::mutex watches_mutex;
std::thread send_thread;
std::thread receive_thread;
std::atomic<bool> expired {false};
void connect(
const Addresses & addresses,
Poco::Timespan connection_timeout);
void sendHandshake();
void receiveHandshake();
void sendAuth(const String & scheme, const String & data);
void receiveEvent();
void sendThread();
void receiveThread();
void close();
/// Call all remaining callbacks and watches, passing errors to them.
void finalize(bool error_send, bool error_receive);
template <typename T>
void write(const T &);
template <typename T>
void read(T &);
};
};

View File

@ -41,20 +41,18 @@ std::optional<std::string> ZooKeeperNodeCache::get(const std::string & path)
if (nonexistent_nodes.count(path))
return std::nullopt;
auto watch_callback = [context=context](zkutil::ZooKeeper & zookeeper, int type, int state, const char * path)
auto watch_callback = [context=context](const ZooKeeperImpl::ZooKeeper::WatchResponse & response)
{
if (!(type != ZOO_SESSION_EVENT || state == ZOO_EXPIRED_SESSION_STATE))
if (!(response.type != ZooKeeperImpl::ZooKeeper::SESSION || response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION))
return;
bool changed = false;
{
std::lock_guard<std::mutex> lock(context->mutex);
if (&zookeeper != context->zookeeper.get())
return;
if (type != ZOO_SESSION_EVENT)
changed = context->invalidated_paths.emplace(path).second;
else if (state == ZOO_EXPIRED_SESSION_STATE)
if (response.type != ZooKeeperImpl::ZooKeeper::SESSION)
changed = context->invalidated_paths.emplace(response.path).second;
else if (response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION)
{
context->zookeeper = nullptr;
context->invalidated_paths.clear();

View File

@ -1,6 +1,9 @@
add_executable(zkutil_test_commands zkutil_test_commands.cpp)
target_link_libraries(zkutil_test_commands clickhouse_common_zookeeper)
add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp)
target_link_libraries(zkutil_test_commands_new_lib clickhouse_common_zookeeper)
add_executable(zkutil_test_lock zkutil_test_lock.cpp)
target_link_libraries(zkutil_test_lock clickhouse_common_zookeeper)
@ -15,3 +18,6 @@ target_link_libraries(zkutil_zookeeper_holder clickhouse_common_zookeeper)
add_executable (zk_many_watches_reconnect zk_many_watches_reconnect.cpp)
target_link_libraries (zk_many_watches_reconnect clickhouse_common_zookeeper clickhouse_common_config)
add_executable (zookeeper_impl zookeeper_impl.cpp)
target_link_libraries (zookeeper_impl clickhouse_common_zookeeper)

View File

@ -1,3 +1,4 @@
#include <Common/typeid_cast.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/StringUtils/StringUtils.h>
#include <iostream>
@ -31,25 +32,24 @@ TEST(zkutil, multi_nice_exception_msg)
{
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
auto acl = zookeeper->getDefaultACL();
zkutil::Ops ops;
zkutil::Requests ops;
ASSERT_NO_THROW(
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
);
try
{
ops.clear();
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/c", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test/zkutil_multi/c", -1));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "BadBoy", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/b", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/c", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi/c", -1));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "BadBoy", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/b", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
FAIL();
@ -69,8 +69,7 @@ TEST(zkutil, multi_nice_exception_msg)
TEST(zkutil, multi_async)
{
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
auto acl = zookeeper->getDefaultACL();
zkutil::Ops ops;
zkutil::Requests ops;
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
@ -81,30 +80,29 @@ TEST(zkutil, multi_async)
{
ops.clear();
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "", zkutil::CreateMode::Persistent));
auto fut = zookeeper->tryAsyncMulti(ops);
ops.clear();
auto res = fut.get();
ASSERT_TRUE(res.code == ZOK);
ASSERT_EQ(res.results->size(), 2);
ASSERT_EQ(res.ops_ptr->size(), 2);
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZOK);
ASSERT_EQ(res.responses.size(), 2);
}
EXPECT_ANY_THROW
(
std::vector<zkutil::ZooKeeper::MultiFuture> futures;
std::vector<std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>> futures;
for (size_t i = 0; i < 10000; ++i)
{
ops.clear();
ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test/zkutil_multi", -1));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Check("/clickhouse_test/zkutil_multi", -1));
ops.emplace_back(new zkutil::Op::SetData("/clickhouse_test/zkutil_multi", "xxx", 42));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi", -1));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", -1));
ops.emplace_back(zkutil::makeSetRequest("/clickhouse_test/zkutil_multi", "xxx", 42));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
futures.emplace_back(zookeeper->asyncMulti(ops));
}
@ -118,16 +116,15 @@ TEST(zkutil, multi_async)
{
ops.clear();
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent));
auto fut = zookeeper->tryAsyncMulti(ops);
ops.clear();
auto res = fut.get();
ASSERT_TRUE(res.code == ZNODEEXISTS);
ASSERT_EQ(res.results->size(), 2);
ASSERT_EQ(res.ops_ptr->size(), 2);
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS);
ASSERT_EQ(res.responses.size(), 2);
}
}
@ -135,9 +132,9 @@ TEST(zkutil, multi_async)
TEST(zkutil, multi_async_libzookeeper_segfault)
{
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", "", 1000);
zkutil::Ops ops;
zkutil::Requests ops;
ops.emplace_back(new zkutil::Op::Check("/clickhouse_test/zkutil_multi", 0));
ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", 0));
/// Uncomment to test
//auto cmd = ShellCommand::execute("sudo service zookeeper restart");
@ -146,7 +143,7 @@ TEST(zkutil, multi_async_libzookeeper_segfault)
auto future = zookeeper->asyncMulti(ops);
auto res = future.get();
EXPECT_TRUE(zkutil::isUnrecoverableErrorCode(res.code));
EXPECT_TRUE(zkutil::isHardwareError(res.error));
}
@ -159,21 +156,20 @@ TEST(zkutil, multi_create_sequential)
zookeeper->createAncestors("/clickhouse_test/");
zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", "", zkutil::DEFAULT_SESSION_TIMEOUT, "/clickhouse_test");
auto acl = zookeeper->getDefaultACL();
zkutil::Ops ops;
zkutil::Requests ops;
String base_path = "/zkutil/multi_create_sequential";
zookeeper->tryRemoveRecursive(base_path);
zookeeper->createAncestors(base_path + "/");
String sequential_node_prefix = base_path + "/queue-";
ops.emplace_back(new zkutil::Op::Create(sequential_node_prefix, "", acl, zkutil::CreateMode::EphemeralSequential));
zkutil::OpResultsPtr results = zookeeper->multi(ops);
zkutil::OpResult & sequential_node_result_op = results->at(0);
ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential));
auto results = zookeeper->multi(ops);
const auto & sequential_node_result_op = typeid_cast<const zkutil::CreateResponse &>(*results.at(0));
EXPECT_FALSE(sequential_node_result_op.value.empty());
EXPECT_GT(sequential_node_result_op.value.length(), sequential_node_prefix.length());
EXPECT_EQ(sequential_node_result_op.value.substr(0, sequential_node_prefix.length()), sequential_node_prefix);
EXPECT_FALSE(sequential_node_result_op.path_created.empty());
EXPECT_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length());
EXPECT_EQ(sequential_node_result_op.path_created.substr(0, sequential_node_prefix.length()), sequential_node_prefix);
}
catch (...)
{

View File

@ -33,18 +33,18 @@ int main(int argc, char ** argv)
while (true)
{
{
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Create>("/test/zk_expiration_test", "hello", zk.getDefaultACL(), zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>("/test/zk_expiration_test", -1));
zkutil::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest("/test/zk_expiration_test", "hello", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeRemoveRequest("/test/zk_expiration_test", -1));
zkutil::MultiTransactionInfo info;
zk.tryMultiNoThrow(ops, nullptr, &info);
zkutil::Responses responses;
int32_t code = zk.tryMultiNoThrow(ops, responses);
std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(info.code) << std::endl;
std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(code) << std::endl;
try
{
if (info.code != ZOK)
std::cout << "Path: " << info.getFailedOp().getPath() << std::endl;
if (code)
std::cout << "Path: " << zkutil::KeeperMultiException(code, ops, responses).getPathForFirstFailedOp() << std::endl;
}
catch (...)
{

View File

@ -23,12 +23,12 @@ try
{
while (true)
{
std::vector<zkutil::ZooKeeper::TryGetFuture> futures;
std::vector<std::future<zkutil::GetResponse>> futures;
for (auto & node : nodes)
futures.push_back(zookeeper.asyncTryGet("/tmp/" + node));
futures.push_back(zookeeper.asyncGet("/tmp/" + node));
for (auto & future : futures)
std::cerr << (future.get().value.empty() ? ',' : '.');
std::cerr << (future.get().data.empty() ? ',' : '.');
}
});
}
@ -41,5 +41,5 @@ try
catch (const Poco::Exception & e)
{
std::cout << e.message() << std::endl;
throw;
return 1;
}

View File

@ -1,41 +1,54 @@
#include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/typeid_cast.h>
#include <iostream>
#include <port/unistd.h>
using namespace zkutil;
int main()
int main(int argc, char ** argv)
try
{
try
if (argc < 2)
{
ZooKeeper zk("mtfilter01t:2181,metrika-test:2181,mtweb01t:2181", "", 5000);
Strings children;
std::cout << "create path" << std::endl;
zk.create("/test", "old", zkutil::CreateMode::Persistent);
zkutil::Stat stat;
zkutil::EventPtr watch = std::make_shared<Poco::Event>();
std::cout << "get path" << std::endl;
zk.get("/test", &stat, watch);
std::cout << "set path" << std::endl;
zk.set("/test", "new");
watch->wait();
std::cout << "watch happened" << std::endl;
std::cout << "remove path" << std::endl;
zk.remove("/test");
Ops ops;
ops.emplace_back(std::make_unique<Op::Create>("/test", "multi1", zk.getDefaultACL(), CreateMode::Persistent));
ops.emplace_back(std::make_unique<Op::SetData>("/test", "multi2", -1));
ops.emplace_back(std::make_unique<Op::Remove>("/test", -1));
std::cout << "multi" << std::endl;
OpResultsPtr res = zk.multi(ops);
std::cout << "path created: " << dynamic_cast<Op::Create &>(*ops[0]).getPathCreated() << std::endl;
}
catch (KeeperException & e)
{
std::cerr << "KeeperException " << e.what() << " " << e.message() << std::endl;
std::cerr << "Usage: ./zkutil_test_commands host:port,host:port...\n";
return 1;
}
ZooKeeper zk(argv[1], "", 5000);
std::cout << "create path" << std::endl;
zk.create("/test", "old", zkutil::CreateMode::Persistent);
zkutil::Stat stat;
zkutil::EventPtr watch = std::make_shared<Poco::Event>();
std::cout << "get path" << std::endl;
zk.get("/test", &stat, watch);
std::cout << "set path" << std::endl;
zk.set("/test", "new");
watch->wait();
std::cout << "watch happened" << std::endl;
std::cout << "remove path" << std::endl;
std::cout << "list path" << std::endl;
Strings children = zk.getChildren("/");
for (const auto & name : children)
std::cerr << "\t" << name << "\n";
zk.remove("/test");
zkutil::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest("/test", "multi1", CreateMode::Persistent));
ops.emplace_back(zkutil::makeSetRequest("/test", "multi2", -1));
ops.emplace_back(zkutil::makeRemoveRequest("/test", -1));
std::cout << "multi" << std::endl;
zkutil::Responses res = zk.multi(ops);
std::cout << "path created: " << typeid_cast<const CreateResponse &>(*res[0]).path_created << std::endl;
return 0;
}
catch (KeeperException & e)
{
std::cerr << "KeeperException " << e.what() << " " << e.message() << std::endl;
return 1;
}

View File

@ -0,0 +1,199 @@
#include <Poco/ConsoleChannel.h>
#include <Poco/Logger.h>
#include <Poco/Event.h>
#include <Common/ZooKeeper/ZooKeeperImpl.h>
#include <Common/typeid_cast.h>
#include <iostream>
#include <boost/algorithm/string.hpp>
using namespace ZooKeeperImpl;
int main(int argc, char ** argv)
try
{
if (argc < 2)
{
std::cerr << "Usage: ./zkutil_test_commands_new_lib host:port,host:port...\n";
return 1;
}
Poco::AutoPtr<Poco::ConsoleChannel> channel(new Poco::ConsoleChannel(std::cerr));
Poco::Logger::root().setChannel(channel);
Poco::Logger::root().setLevel("trace");
std::string addresses_arg = argv[1];
std::vector<std::string> addresses_strings;
boost::split(addresses_strings, addresses_arg, boost::is_any_of(","));
ZooKeeper::Addresses addresses;
addresses.reserve(addresses_strings.size());
for (const auto & address_string : addresses_strings)
addresses.emplace_back(address_string);
ZooKeeper zk(addresses, {}, {}, {}, {5, 0}, {0, 50000}, {0, 50000});
Poco::Event event(true);
std::cout << "create\n";
zk.create("/test", "old", false, false, {},
[&](const ZooKeeper::CreateResponse & response)
{
if (response.error)
std::cerr << "Error (create) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
std::cerr << "Created path: " << response.path_created << '\n';
//event.set();
});
//event.wait();
std::cout << "get\n";
zk.get("/test",
[&](const ZooKeeper::GetResponse & response)
{
if (response.error)
std::cerr << "Error (get) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
std::cerr << "Value: " << response.data << '\n';
//event.set();
},
[](const ZooKeeper::WatchResponse & response)
{
if (response.error)
std::cerr << "Watch (get) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n';
});
//event.wait();
std::cout << "set\n";
zk.set("/test", "new", -1,
[&](const ZooKeeper::SetResponse & response)
{
if (response.error)
std::cerr << "Error (set) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
std::cerr << "Set\n";
//event.set();
});
//event.wait();
std::cout << "list\n";
zk.list("/",
[&](const ZooKeeper::ListResponse & response)
{
if (response.error)
std::cerr << "Error (list) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
{
std::cerr << "Children:\n";
for (const auto & name : response.names)
std::cerr << name << "\n";
}
//event.set();
},
[](const ZooKeeper::WatchResponse & response)
{
if (response.error)
std::cerr << "Watch (list) on /, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n';
});
//event.wait();
std::cout << "exists\n";
zk.exists("/test",
[&](const ZooKeeper::ExistsResponse & response)
{
if (response.error)
std::cerr << "Error (exists) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
std::cerr << "Exists\n";
//event.set();
},
[](const ZooKeeper::WatchResponse & response)
{
if (response.error)
std::cerr << "Watch (exists) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n';
});
//event.wait();
std::cout << "remove\n";
zk.remove("/test", -1, [&](const ZooKeeper::RemoveResponse & response)
{
if (response.error)
std::cerr << "Error (remove) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
std::cerr << "Removed\n";
//event.set();
});
//event.wait();
std::cout << "multi\n";
ZooKeeper::Requests ops;
{
ZooKeeper::CreateRequest create_request;
create_request.path = "/test";
create_request.data = "multi1";
ops.emplace_back(std::make_shared<ZooKeeper::CreateRequest>(std::move(create_request)));
}
{
ZooKeeper::SetRequest set_request;
set_request.path = "/test";
set_request.data = "multi2";
ops.emplace_back(std::make_shared<ZooKeeper::SetRequest>(std::move(set_request)));
}
{
ZooKeeper::RemoveRequest remove_request;
remove_request.path = "/test";
ops.emplace_back(std::make_shared<ZooKeeper::RemoveRequest>(std::move(remove_request)));
}
zk.multi(ops, [&](const ZooKeeper::MultiResponse & response)
{
if (response.error)
std::cerr << "Error (multi) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
else
{
for (const auto & elem : response.responses)
if (elem->error)
std::cerr << "Error (elem) " << elem->error << ": " << ZooKeeper::errorMessage(elem->error) << '\n';
std::cerr << "Created path: " << typeid_cast<const ZooKeeper::CreateResponse &>(*response.responses[0]).path_created << '\n';
}
event.set();
});
event.wait();
return 0;
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(__PRETTY_FUNCTION__) << '\n';
return 1;
}

View File

@ -0,0 +1,26 @@
#include <Common/ZooKeeper/ZooKeeperImpl.h>
#include <iostream>
int main()
try
{
ZooKeeperImpl::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}, {0, 50000});
zookeeper.create("/test", "hello", false, false, {}, [](const ZooKeeperImpl::ZooKeeper::CreateResponse & response)
{
if (response.error)
std::cerr << "Error " << response.error << ": " << ZooKeeperImpl::ZooKeeper::errorMessage(response.error) << "\n";
else
std::cerr << "Path created: " << response.path_created << "\n";
});
sleep(100);
return 0;
}
catch (...)
{
DB::tryLogCurrentException(__PRETTY_FUNCTION__);
return 1;
}

View File

@ -179,7 +179,7 @@ struct IndexIdentity
/// For countEqual.
struct IndexCount
{
using ResultType = UInt32;
using ResultType = UInt64;
static bool apply(size_t, ResultType & current) { ++current; return true; }
};

View File

@ -549,16 +549,16 @@ void DDLWorker::processTask(DDLTask & task)
String active_node_path = task.entry_path + "/active/" + task.host_id_str;
String finished_node_path = task.entry_path + "/finished/" + task.host_id_str;
auto code = zookeeper->tryCreateWithRetries(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy);
if (code == ZOK || code == ZNODEEXISTS)
auto code = zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy);
if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
{
// Ok
}
else if (code == ZNONODE)
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
{
/// There is no parent
createStatusDirs(task.entry_path);
if (ZOK != zookeeper->tryCreateWithRetries(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy))
if (ZooKeeperImpl::ZooKeeper::ZOK != zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy))
throw zkutil::KeeperException(code, active_node_path);
}
else
@ -599,10 +599,9 @@ void DDLWorker::processTask(DDLTask & task)
/// FIXME: if server fails right here, the task will be executed twice. We need WAL here.
/// Delete active flag and create finish flag
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(active_node_path, -1));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(finished_node_path, task.execution_status.serializeText(),
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
zkutil::Requests ops;
ops.emplace_back(zkutil::makeRemoveRequest(active_node_path, -1));
ops.emplace_back(zkutil::makeCreateRequest(finished_node_path, task.execution_status.serializeText(), zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
}
@ -779,9 +778,9 @@ void DDLWorker::cleanupQueue()
}
/// Remove the lock node and its parent atomically
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(lock_path, -1));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(node_path, -1));
zkutil::Requests ops;
ops.emplace_back(zkutil::makeRemoveRequest(lock_path, -1));
ops.emplace_back(zkutil::makeRemoveRequest(node_path, -1));
zookeeper->multi(ops);
lock->unlockAssumeLockNodeRemovedManually();
@ -798,13 +797,20 @@ void DDLWorker::cleanupQueue()
/// Try to create nonexisting "status" dirs for a node
void DDLWorker::createStatusDirs(const std::string & node_path)
{
zkutil::Ops ops;
auto acl = zookeeper->getDefaultACL();
ops.emplace_back(std::make_shared<zkutil::Op::Create>(node_path + "/active", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(node_path + "/finished", "", acl, zkutil::CreateMode::Persistent));
int code = zookeeper->tryMulti(ops);
if (code != ZOK && code != ZNODEEXISTS)
zkutil::Requests ops;
{
zkutil::CreateRequest request;
request.path = node_path + "/active";
ops.emplace_back(std::make_shared<zkutil::CreateRequest>(std::move(request)));
}
{
zkutil::CreateRequest request;
request.path = node_path + "/finished";
ops.emplace_back(std::make_shared<zkutil::CreateRequest>(std::move(request)));
}
zkutil::Responses responses;
int code = zookeeper->tryMulti(ops, responses);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
throw zkutil::KeeperException(code);
}
@ -817,19 +823,7 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry)
String query_path_prefix = queue_dir + "/query-";
zookeeper->createAncestors(query_path_prefix);
String node_path;
try
{
node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential);
}
catch (const zkutil::KeeperException & e)
{
/// TODO: This condition could be relaxed with additional post-checks
if (e.isTemporaryError())
throw Exception("Unknown status of distributed DDL task", ErrorCodes::UNKNOWN_STATUS_OF_DISTRIBUTED_DDL_TASK);
throw;
}
String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential);
/// Optional step
try
@ -894,29 +888,22 @@ void DDLWorker::run()
{
if (e.isHardwareError())
{
if (!e.isTemporaryError())
{
LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false));
LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false));
while (!stop_flag)
while (!stop_flag)
{
try
{
try
{
zookeeper = context.getZooKeeper();
break;
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
using namespace std::chrono_literals;
std::this_thread::sleep_for(5s);
}
zookeeper = context.getZooKeeper();
break;
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
using namespace std::chrono_literals;
std::this_thread::sleep_for(5s);
}
}
else
{
LOG_DEBUG(log, "Retry task processing after: " << getCurrentExceptionMessage(false));
}
}
else
@ -1060,7 +1047,7 @@ private:
{
Strings res;
int code = zookeeper->tryGetChildren(node_path, res);
if (code != ZOK && code != ZNONODE)
if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE)
throw zkutil::KeeperException(code, node_path);
return res;
}

View File

@ -192,6 +192,9 @@ else ()
add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_BUNDLE})
endif ()
set (GLIBC_MAX_REQUIRED 2.4)
add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-server | grep '@GLIBC' | grep -oP 'GLIBC_[\\d\\.]+' | sort | uniq | sort -r | perl -lnE 'exit 1 if $_ gt q{GLIBC_${GLIBC_MAX_REQUIRED}}'")
install (
FILES config.xml users.xml
DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-server

View File

@ -354,24 +354,26 @@ struct TaskCluster
};
struct MultiTransactionInfo
{
int32_t code;
zkutil::Requests requests;
zkutil::Responses responses;
};
/// Atomically checks that is_dirty node is not exists, and made the remaining op
/// Returns relative number of failed operation in the second field (the passed op has 0 index)
static zkutil::MultiTransactionInfo checkNoNodeAndCommit(
static MultiTransactionInfo checkNoNodeAndCommit(
const zkutil::ZooKeeperPtr & zookeeper,
const String & checking_node_path,
zkutil::OpPtr && op)
zkutil::RequestPtr && op)
{
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Create>(checking_node_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(checking_node_path, -1));
ops.emplace_back(std::move(op));
zkutil::MultiTransactionInfo info;
zookeeper->tryMultiNoThrow(ops, nullptr, &info);
if (info.code != ZOK && !zkutil::isUserError(info.code))
throw zkutil::KeeperException(info.code);
MultiTransactionInfo info;
info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent));
info.requests.emplace_back(zkutil::makeRemoveRequest(checking_node_path, -1));
info.requests.emplace_back(std::move(op));
info.code = zookeeper->tryMulti(info.requests, info.responses);
return info;
}
@ -732,7 +734,7 @@ public:
{
auto zookeeper = getZooKeeper();
task_description_watch_callback = [this] (zkutil::ZooKeeper &, int, int, const char *)
task_description_watch_callback = [this] (const ZooKeeperImpl::ZooKeeper::WatchResponse &)
{
UInt64 version = ++task_descprtion_version;
LOG_DEBUG(log, "Task description should be updated, local version " << version);
@ -896,7 +898,7 @@ public:
int code;
zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code);
if (code != ZOK)
if (code)
throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS);
LOG_DEBUG(log, "Loading description, zxid=" << task_descprtion_current_stat.czxid);
@ -1050,15 +1052,16 @@ protected:
}
else
{
zkutil::Ops ops;
ops.emplace_back(new zkutil::Op::SetData(workers_version_path, description, version));
ops.emplace_back(new zkutil::Op::Create(current_worker_path, description, zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral));
auto code = zookeeper->tryMulti(ops);
zkutil::Requests ops;
ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version));
ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral));
zkutil::Responses responses;
auto code = zookeeper->tryMulti(ops, responses);
if (code == ZOK || code == ZNODEEXISTS)
if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
return std::make_shared<zkutil::EphemeralNodeHolder>(current_worker_path, *zookeeper, false, false, description);
if (code == ZBADVERSION)
if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
{
LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again");
}
@ -1212,7 +1215,7 @@ protected:
}
catch (zkutil::KeeperException & e)
{
if (e.code == ZNODEEXISTS)
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
{
LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep");
std::this_thread::sleep_for(default_sleep_time);
@ -1259,10 +1262,10 @@ protected:
}
/// Remove the locking node
zkutil::Ops ops;
ops.emplace_back(new zkutil::Op::Remove(dirt_cleaner_path, -1));
ops.emplace_back(new zkutil::Op::Remove(is_dirty_flag_path, -1));
zookeeper->multi(ops);
zkutil::Requests requests;
requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1));
requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1));
zookeeper->multi(requests);
LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name);
return true;
@ -1457,7 +1460,6 @@ protected:
ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name);
auto zookeeper = getZooKeeper();
auto acl = zookeeper->getDefaultACL();
String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath();
String current_task_is_active_path = task_partition.getActiveWorkerPath();
@ -1469,7 +1471,7 @@ protected:
auto create_is_dirty_node = [&] ()
{
auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent);
if (code != ZOK && code != ZNODEEXISTS)
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
throw zkutil::KeeperException(code, is_dirty_flag_path);
};
@ -1520,7 +1522,7 @@ protected:
}
catch (const zkutil::KeeperException & e)
{
if (e.code == ZNODEEXISTS)
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
{
LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path);
return PartitionTaskStatus::Active;
@ -1589,18 +1591,20 @@ protected:
/// Try start processing, create node about it
{
String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id);
auto op_create = std::make_shared<zkutil::Op::Create>(current_task_status_path, start_state, acl, zkutil::CreateMode::Persistent);
zkutil::MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_create));
auto op_create = zkutil::makeCreateRequest(current_task_status_path, start_state, zkutil::CreateMode::Persistent);
MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_create));
if (info.code != ZOK)
if (info.code)
{
if (info.getFailedOp().getPath() == is_dirty_flag_path)
zkutil::KeeperMultiException exception(info.code, info.requests, info.responses);
if (exception.getPathForFirstFailedOp() == is_dirty_flag_path)
{
LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled");
return PartitionTaskStatus::Error;
}
throw zkutil::KeeperException(info.code, current_task_status_path);
throw exception;
}
}
@ -1663,7 +1667,7 @@ protected:
output = io_insert.out;
}
using ExistsFuture = zkutil::ZooKeeper::ExistsFuture;
using ExistsFuture = std::future<zkutil::ExistsResponse>;
std::unique_ptr<ExistsFuture> future_is_dirty_checker;
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
@ -1682,7 +1686,7 @@ protected:
/// Otherwise, the insertion will slow a little bit
if (watch.elapsedMilliseconds() >= check_period_milliseconds)
{
zkutil::ZooKeeper::StatAndExists status;
zkutil::ExistsResponse status;
try
{
@ -1692,14 +1696,10 @@ protected:
catch (zkutil::KeeperException & e)
{
future_is_dirty_checker.reset();
if (e.isTemporaryError())
LOG_INFO(log, "ZooKeeper is lagging: " << e.displayText());
else
throw;
throw;
}
if (status.exists)
if (status.error != ZooKeeperImpl::ZooKeeper::ZNONODE)
throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
}
@ -1735,12 +1735,14 @@ protected:
/// Finalize the processing, change state of current partition task (and also check is_dirty flag)
{
String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id);
auto op_set = std::make_shared<zkutil::Op::SetData>(current_task_status_path, state_finished, 0);
zkutil::MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_set));
auto op_set = zkutil::makeSetRequest(current_task_status_path, state_finished, 0);
MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_set));
if (info.code != ZOK)
if (info.code)
{
if (info.getFailedOp().getPath() == is_dirty_flag_path)
zkutil::KeeperMultiException exception(info.code, info.requests, info.responses);
if (exception.getPathForFirstFailedOp() == is_dirty_flag_path)
LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled");
else
LOG_INFO(log, "Someone made the node abandoned. Will refill partition. " << zkutil::ZooKeeper::error2string(info.code));

View File

@ -369,10 +369,14 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (secure)
/// Bug in old poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl
/// https://github.com/pocoproject/poco/pull/2257
socket.bind(address, true);
socket.bind(address, /* reuseAddress = */ true);
else
#endif
#if POCO_VERSION < 0x01080000
socket.bind(address, /* reuseAddress = */ true);
#else
socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false));
#endif
socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64));

View File

@ -10,6 +10,7 @@
#include <chrono>
#include <optional>
#include <Interpreters/Cluster.h>
#include <Interpreters/Context.h>
namespace Poco
{

View File

@ -30,7 +30,7 @@ public:
};
AbandonableLockInZooKeeper(
const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, zkutil::Ops * precheck_ops = nullptr)
const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, zkutil::Requests * precheck_ops = nullptr)
: zookeeper(&zookeeper_), path_prefix(path_prefix_)
{
String abandonable_path = temp_path + "/abandonable_lock-";
@ -42,12 +42,9 @@ public:
}
else
{
precheck_ops->emplace_back(std::make_shared<zkutil::Op::Create>(
abandonable_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::EphemeralSequential));
zkutil::OpResultsPtr op_results = zookeeper->multi(*precheck_ops);
holder_path = op_results->back().value;
precheck_ops->emplace_back(zkutil::makeCreateRequest(abandonable_path, "", zkutil::CreateMode::EphemeralSequential));
zkutil::Responses op_results = zookeeper->multi(*precheck_ops);
holder_path = dynamic_cast<const zkutil::CreateResponse &>(*op_results.back()).path_created;
}
/// Write the path to the secondary node in the main node.
@ -101,11 +98,11 @@ public:
}
/// Adds actions equivalent to `unlock()` to the list.
void getUnlockOps(zkutil::Ops & ops)
void getUnlockOps(zkutil::Requests & ops)
{
checkCreated();
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(path, -1));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(holder_path, -1));
ops.emplace_back(zkutil::makeRemoveRequest(path, -1));
ops.emplace_back(zkutil::makeRemoveRequest(holder_path, -1));
}
/// Do not delete nodes in destructor. You may call this method after 'getUnlockOps' and successful execution of these ops,
@ -128,7 +125,7 @@ public:
try
{
zookeeper->tryRemoveEphemeralNodeWithRetries(holder_path);
zookeeper->tryRemove(holder_path);
zookeeper->trySet(path, ""); /// It's not necessary.
}
catch (...)
@ -156,7 +153,7 @@ public:
/// If there is no secondary node, you need to test again the existence of the main node,
/// because during this time you might have time to call unlock().
/// At the same time, we will remove the path to the secondary node from there.
if (zookeeper.trySet(path, "") == ZOK)
if (zookeeper.trySet(path, "") == ZooKeeperImpl::ZooKeeper::ZOK)
return ABANDONED;
return UNLOCKED;

View File

@ -1,9 +1,9 @@
#include <Storages/MergeTree/MergeTreeBaseBlockInputStream.h>
#include <Storages/MergeTree/MergeTreeRangeReader.h>
#include <Storages/MergeTree/MergeTreeReader.h>
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
#include <Columns/FilterDescription.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnsCommon.h>
#include <Common/typeid_cast.h>
#include <ext/range.h>
@ -71,8 +71,6 @@ Block MergeTreeBaseBlockInputStream::readImpl()
Block MergeTreeBaseBlockInputStream::readFromPart()
{
Block res;
if (task->size_predictor)
task->size_predictor->startBlock();
@ -102,7 +100,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
size_t rows_to_read_for_max_size_column
= task.size_predictor->estimateNumRowsForMaxSizeColumn(preferred_max_column_in_block_size_bytes);
double filtration_ratio = std::max(min_filtration_ratio, 1.0 - task.size_predictor->filtered_rows_ratio);
size_t rows_to_read_for_max_size_column_with_filtration
auto rows_to_read_for_max_size_column_with_filtration
= static_cast<size_t>(rows_to_read_for_max_size_column / filtration_ratio);
/// If preferred_max_column_in_block_size_bytes is used, number of rows to read can be less than index_granularity.
@ -117,357 +115,69 @@ Block MergeTreeBaseBlockInputStream::readFromPart()
return index_granularity * granule_to_read - reader.numReadRowsInCurrentGranule();
};
// read rows from reader and clear columns
auto skipRows = [&estimateNumRows](
Block & block, MergeTreeRangeReader & reader, MergeTreeReadTask & task, size_t rows)
if (!task->range_reader.isInitialized())
{
size_t recommended_rows = rows;
if (task.size_predictor)
recommended_rows = std::max<size_t>(1, estimateNumRows(task, reader));
while (rows)
if (prewhere_actions)
{
size_t rows_to_skip = std::min(rows, recommended_rows);
rows -= rows_to_skip;
reader.read(block, rows_to_skip);
for (const auto i : ext::range(0, block.columns()))
if (reader->getColumns().empty())
{
auto & col = block.getByPosition(i);
if (task.column_name_set.count(col.name))
{
if (const ColumnArray * column_array = typeid_cast<const ColumnArray *>(col.column.get()))
{
/// ColumnArray columns in block could have common offset column, which is used while reading.
/// This is in case of nested data structures.
/// TODO Very dangerous and unclear. Get rid of this after implemented full-featured Nested data type.
/// Have to call resize(0) instead of cloneEmpty to save structure.
/// (To keep offsets possibly shared between different arrays.)
static_cast<ColumnArray &>(column_array->assumeMutableRef()).getOffsets().resize(0);
/// It's ok until multidimensional arrays are not stored in MergeTree.
static_cast<ColumnArray &>(column_array->assumeMutableRef()).getDataPtr() = column_array->getDataPtr()->cloneEmpty();
}
else
col.column = col.column->cloneEmpty();
}
}
}
};
if (prewhere_actions)
{
do
{
/// Let's read the full block of columns needed to calculate the expression in PREWHERE.
MarkRanges ranges_to_read;
/// Last range may be partl read. The same number of rows we need to read after prewhere
size_t rows_was_read_in_last_range = 0;
std::optional<MergeTreeRangeReader> pre_range_reader;
auto processNextRange = [& ranges_to_read, & rows_was_read_in_last_range, & pre_range_reader](
MergeTreeReadTask & task, MergeTreeReader & pre_reader)
{
auto & range = task.mark_ranges.back();
pre_range_reader = pre_reader.readRange(range.begin, range.end);
ranges_to_read.push_back(range);
rows_was_read_in_last_range = 0;
task.mark_ranges.pop_back();
};
if (task->current_range_reader)
{
/// Havn't finihsed reading at last step. Copy state for prewhere columns
pre_range_reader = task->current_range_reader->copyForReader(*pre_reader);
if (task->number_of_rows_to_skip)
{
/// number_of_rows_to_skip already was read for prewhere columns. skip them.
pre_range_reader = pre_range_reader->getFutureState(task->number_of_rows_to_skip);
pre_range_reader->disableNextSeek();
}
}
else
processNextRange(*task, *pre_reader);
/// FIXME: size prediction model is updated by filtered rows, but it predicts size of unfiltered rows also
size_t recommended_rows = estimateNumRows(*task, *pre_range_reader);
if (res && recommended_rows < 1)
break;
size_t space_left = std::max(static_cast<decltype(max_block_size_rows)>(1), std::min(max_block_size_rows, recommended_rows));
while ((pre_range_reader || !task->mark_ranges.empty()) && space_left && !isCancelled())
{
if (!pre_range_reader)
processNextRange(*task, *pre_reader);
size_t rows_to_read = std::min(pre_range_reader->numPendingRows(), space_left);
size_t read_rows = pre_range_reader->read(res, rows_to_read);
rows_was_read_in_last_range += read_rows;
if (pre_range_reader->isReadingFinished())
pre_range_reader.reset();
space_left -= read_rows;
}
/// In case of isCancelled.
if (!res)
{
task->current_range_reader.reset();
return res;
}
progressImpl({ res.rows(), res.bytes() });
pre_reader->fillMissingColumns(res, task->ordered_names, task->should_reorder, res.rows());
/// Compute the expression in PREWHERE.
prewhere_actions->execute(res);
ColumnPtr prewhere_column = res.getByName(prewhere_column_name).column;
if (task->remove_prewhere_column)
res.erase(prewhere_column_name);
const auto pre_bytes = res.bytes();
ConstantFilterDescription constant_filter_description(*prewhere_column);
/** If the filter is a constant (for example, it says PREWHERE 0),
* then either return an empty block, or return the block unchanged.
*/
if (constant_filter_description.always_false)
{
/*
If this filter is PREWHERE 0, MergeTree Stream can be marked as done,
and this task can be clear.
If we don't mark this task finished here, readImpl could
jump into endless loop.
Error scenario:
select * from table where isNull(NOT_NULLABLE_COLUMN) AND OTHER PRED;
and isNull pred is promoted to PREWHERE.
(though it is difficult to reproduce)
*/
task->current_range_reader.reset();
task->mark_ranges.clear();
res.clear();
return res;
}
else if (constant_filter_description.always_true)
{
if (task->current_range_reader)
{
if (task->number_of_rows_to_skip)
skipRows(res, *task->current_range_reader, *task, task->number_of_rows_to_skip);
size_t rows_to_read = ranges_to_read.empty()
? rows_was_read_in_last_range : task->current_range_reader->numPendingRows();
task->current_range_reader->read(res, rows_to_read);
}
for (auto range_idx : ext::range(0, ranges_to_read.size()))
{
const auto & range = ranges_to_read[range_idx];
task->current_range_reader = reader->readRange(range.begin, range.end);
size_t rows_to_read = range_idx + 1 == ranges_to_read.size()
? rows_was_read_in_last_range : task->current_range_reader->numPendingRows();
task->current_range_reader->read(res, rows_to_read);
}
if (!pre_range_reader)
task->current_range_reader.reset();
task->number_of_rows_to_skip = 0;
progressImpl({ 0, res.bytes() - pre_bytes });
task->range_reader = MergeTreeRangeReader(
pre_reader.get(), index_granularity, nullptr, prewhere_actions,
&prewhere_column_name, &task->ordered_names,
task->should_reorder, task->remove_prewhere_column, true);
}
else
{
FilterDescription filter_and_holder(*prewhere_column);
task->pre_range_reader = MergeTreeRangeReader(
pre_reader.get(), index_granularity, nullptr, prewhere_actions,
&prewhere_column_name, &task->ordered_names,
task->should_reorder, task->remove_prewhere_column, false);
const auto & pre_filter = *filter_and_holder.data;
auto & number_of_rows_to_skip = task->number_of_rows_to_skip;
if (!task->current_range_reader)
number_of_rows_to_skip = 0;
IColumn::Filter post_filter(pre_filter.size());
/// Let's read the rest of the columns in the required segments and compose our own filter for them.
size_t pre_filter_pos = 0;
size_t post_filter_pos = 0;
size_t next_range_idx = 0;
while (pre_filter_pos < pre_filter.size())
{
if (!task->current_range_reader)
{
if (next_range_idx == ranges_to_read.size())
throw Exception("Not enough ranges to read after prewhere.", ErrorCodes::LOGICAL_ERROR);
const auto & range = ranges_to_read[next_range_idx++];
task->current_range_reader = reader->readRange(range.begin, range.end);
}
MergeTreeRangeReader & range_reader = *task->current_range_reader;
size_t current_range_rows_read = 0;
auto pre_filter_begin_pos = pre_filter_pos;
/// Now we need to read the same number of rows as in prewhere.
size_t rows_to_read = next_range_idx == ranges_to_read.size()
? rows_was_read_in_last_range : (task->current_range_reader->numPendingRows() - number_of_rows_to_skip);
auto readRows = [&]()
{
if (pre_filter_pos != pre_filter_begin_pos)
{
/// Fulfilling the promise to read (pre_filter_pos - pre_filter_begin_pos) rows
auto rows = pre_filter_pos - pre_filter_begin_pos;
memcpy(&post_filter[post_filter_pos], &pre_filter[pre_filter_begin_pos], rows);
post_filter_pos += rows;
current_range_rows_read += rows;
if (number_of_rows_to_skip)
{
/** Wasn't able to skip 'number_of_rows_to_skip' with false prewhere conditon
* Just read them and throw away. */
skipRows(res, range_reader, *task, number_of_rows_to_skip);
number_of_rows_to_skip = 0;
}
range_reader.read(res, rows);
}
};
/** (pre_filter_pos - pre_filter_begin_pos) here is the number of rows we promies to read, but
haven't read yet to merge consecutive nonempy granulas. */
while (current_range_rows_read + (pre_filter_pos - pre_filter_begin_pos) < rows_to_read)
{
auto rows_should_be_copied = pre_filter_pos - pre_filter_begin_pos;
auto range_reader_with_skipped_rows = range_reader.getFutureState(number_of_rows_to_skip + rows_should_be_copied);
auto unread_rows_in_current_granule = range_reader_with_skipped_rows.numPendingRowsInCurrentGranule();
const size_t limit = std::min(pre_filter.size(), pre_filter_pos + unread_rows_in_current_granule);
bool will_read_until_mark = unread_rows_in_current_granule == limit - pre_filter_pos;
if (memoryIsZero(&pre_filter[pre_filter_pos], (limit - pre_filter_pos) * sizeof(pre_filter[0])))
{
/// Zero! Prewhere condition is false for all (limit - pre_filter_pos) rows.
readRows();
if (will_read_until_mark)
{
/// Can skip the rest of granule with false prewhere conditon right now.
do
{
size_t rows_was_skipped = range_reader.skipToNextMark();
if (number_of_rows_to_skip < rows_was_skipped)
{
current_range_rows_read += rows_was_skipped - number_of_rows_to_skip;
number_of_rows_to_skip = 0;
}
else
number_of_rows_to_skip -= rows_was_skipped;
}
while (number_of_rows_to_skip);
}
else
{
/// Here reading seems to be done. It's still possible to skip rows during next reading.
number_of_rows_to_skip += limit - pre_filter_pos;
current_range_rows_read += limit - pre_filter_pos;
}
pre_filter_begin_pos = limit;
}
pre_filter_pos = limit;
}
readRows();
if (next_range_idx != ranges_to_read.size())
task->current_range_reader.reset();
}
if (!pre_range_reader)
task->current_range_reader.reset();
if (!post_filter_pos)
{
if (task->size_predictor)
task->size_predictor->updateFilteredRowsRation(pre_filter.size(), pre_filter.size());
res.clear();
continue;
}
progressImpl({ 0, res.bytes() - pre_bytes });
post_filter.resize(post_filter_pos);
/// At this point we may have arrays with non-zero offsets but with empty data,
/// as a result of reading components of Nested data structures with no data in filesystem.
/// We must fill these arrays to filter them correctly.
reader->fillMissingColumns(res, task->ordered_names, task->should_reorder, post_filter.size(), true);
/// Filter the columns related to PREWHERE using pre_filter,
/// other columns - using post_filter.
size_t rows = 0;
for (const auto i : ext::range(0, res.columns()))
{
auto & col = res.getByPosition(i);
if (col.name == prewhere_column_name && res.columns() > 1)
continue;
col.column = col.column->filter(task->column_name_set.count(col.name) ? post_filter : pre_filter, -1);
rows = col.column->size();
}
if (task->size_predictor)
task->size_predictor->updateFilteredRowsRation(pre_filter.size(), pre_filter.size() - rows);
/// Replace column with condition value from PREWHERE to a constant.
if (!task->remove_prewhere_column)
res.getByName(prewhere_column_name).column = DataTypeUInt8().createColumnConst(rows, UInt64(1))->convertToFullColumnIfConst();
}
if (res)
{
if (task->size_predictor)
task->size_predictor->update(res);
reader->fillMissingColumns(res, task->ordered_names, true, res.rows());
res.checkNumberOfRows();
task->range_reader = MergeTreeRangeReader(
reader.get(), index_granularity, &task->pre_range_reader, nullptr,
nullptr, &task->ordered_names, true, false, true);
}
}
while (!task->isFinished() && !res && !isCancelled());
}
else
{
size_t space_left = std::max(static_cast<decltype(max_block_size_rows)>(1), max_block_size_rows);
while (!task->isFinished() && space_left && !isCancelled())
else
{
if (!task->current_range_reader)
{
auto & range = task->mark_ranges.back();
task->current_range_reader = reader->readRange(range.begin, range.end);
task->mark_ranges.pop_back();
}
size_t rows_to_read = space_left;
size_t recommended_rows = estimateNumRows(*task, *task->current_range_reader);
if (res && recommended_rows < 1)
break;
rows_to_read = std::min(rows_to_read, std::max(static_cast<decltype(recommended_rows)>(1), recommended_rows));
size_t rows_was_read = task->current_range_reader->read(res, rows_to_read);
if (task->current_range_reader->isReadingFinished())
task->current_range_reader.reset();
if (res && task->size_predictor)
{
task->size_predictor->update(res);
}
space_left -= rows_was_read;
task->range_reader = MergeTreeRangeReader(
reader.get(), index_granularity, nullptr, prewhere_actions,
nullptr, &task->ordered_names, task->should_reorder, false, true);
}
/// In the case of isCancelled.
if (!res)
return res;
progressImpl({ res.rows(), res.bytes() });
reader->fillMissingColumns(res, task->ordered_names, task->should_reorder, res.rows());
}
return res;
size_t recommended_rows = estimateNumRows(*task, task->range_reader);
size_t rows_to_read = std::max(static_cast<decltype(max_block_size_rows)>(1),
std::min(max_block_size_rows, recommended_rows));
auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges);
/// All rows were filtered. Repeat.
if (read_result.block.rows() == 0)
read_result.block.clear();
size_t num_filtered_rows = read_result.numReadRows() - read_result.block.rows();
progressImpl({ read_result.numReadRows(), read_result.numBytesRead() });
if (task->size_predictor)
{
task->size_predictor->updateFilteredRowsRation(read_result.numReadRows(), num_filtered_rows);
if (read_result.block)
task->size_predictor->update(read_result.block);
}
if (read_result.block && prewhere_actions && !task->remove_prewhere_column)
{
/// Convert const column to full here because it's cheaper to filter const column than full.
auto & column = read_result.block.getByName(prewhere_column_name);
column.column = column.column->convertToFullColumnIfConst();
}
read_result.block.checkNumberOfRows();
return read_result.block;
}

View File

@ -49,13 +49,11 @@ struct MergeTreeReadTask
const bool should_reorder;
/// Used to satistfy preferred_block_size_bytes limitation
MergeTreeBlockSizePredictorPtr size_predictor;
/// used to save current range processing status
std::optional<MergeTreeRangeReader> current_range_reader;
/// the number of rows wasn't read by range_reader if condition in prewhere was false
/// helps to skip graunule if all conditions will be aslo false
size_t number_of_rows_to_skip;
/// Used to save current range processing status
MergeTreeRangeReader range_reader;
MergeTreeRangeReader pre_range_reader;
bool isFinished() const { return mark_ranges.empty() && !current_range_reader; }
bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); }
MergeTreeReadTask(
const MergeTreeData::DataPartPtr & data_part, const MarkRanges & mark_ranges, const size_t part_index_in_query,

View File

@ -1,70 +1,633 @@
#include <Storages/MergeTree/MergeTreeReader.h>
#include <Columns/FilterDescription.h>
#include <ext/range.h>
#include <Columns/ColumnsCommon.h>
#if __SSE2__
#include <emmintrin.h>
#endif
namespace DB
{
MergeTreeRangeReader::MergeTreeRangeReader(
MergeTreeReader & merge_tree_reader, size_t from_mark, size_t to_mark, size_t index_granularity)
: merge_tree_reader(merge_tree_reader), current_mark(from_mark), last_mark(to_mark)
, index_granularity(index_granularity)
MergeTreeRangeReader::DelayedStream::DelayedStream(
size_t from_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader)
: current_mark(from_mark), current_offset(0), num_delayed_rows(0)
, index_granularity(index_granularity), merge_tree_reader(merge_tree_reader)
, continue_reading(false), is_finished(false)
{
}
size_t MergeTreeRangeReader::skipToNextMark()
size_t MergeTreeRangeReader::DelayedStream::position() const
{
auto unread_rows_in_current_part = numPendingRowsInCurrentGranule();
continue_reading = false;
++current_mark;
if (current_mark == last_mark)
is_reading_finished = true;
read_rows_after_current_mark = 0;
return unread_rows_in_current_part;
return current_mark * index_granularity + current_offset + num_delayed_rows;
}
MergeTreeRangeReader MergeTreeRangeReader::getFutureState(size_t rows_to_read) const
size_t MergeTreeRangeReader::DelayedStream::readRows(Block & block, size_t num_rows)
{
MergeTreeRangeReader copy = *this;
copy.read_rows_after_current_mark += rows_to_read;
size_t read_parts = copy.read_rows_after_current_mark / index_granularity;
copy.current_mark += read_parts;
copy.read_rows_after_current_mark -= index_granularity * read_parts;
return copy;
if (num_rows)
{
size_t rows_read = merge_tree_reader->readRows(current_mark, continue_reading, num_rows, block);
continue_reading = true;
/// Zero rows_read my be either because reading has finished
/// or because there is no columns we can read in current part (for example, all columns are default).
/// In the last case we can't finish reading, but it's also ok for the first case
/// because we can finish reading by calculation the number of pending rows.
if (0 < rows_read && rows_read < num_rows)
is_finished = true;
return rows_read;
}
return 0;
}
size_t MergeTreeRangeReader::read(Block & res, size_t max_rows_to_read)
size_t MergeTreeRangeReader::DelayedStream::read(Block & block, size_t from_mark, size_t offset, size_t num_rows)
{
size_t rows_to_read = numPendingRows();
rows_to_read = std::min(rows_to_read, max_rows_to_read);
if (rows_to_read == 0)
throw Exception("Logical error: 0 rows to read.", ErrorCodes::LOGICAL_ERROR);
if (position() == from_mark * index_granularity + offset)
{
num_delayed_rows += num_rows;
return 0;
}
else
{
size_t read_rows = finalize(block);
auto read_rows = merge_tree_reader.get().readRows(current_mark, continue_reading, rows_to_read, res);
continue_reading = false;
current_mark = from_mark;
current_offset = offset;
num_delayed_rows = num_rows;
if (read_rows && read_rows < rows_to_read)
is_reading_finished = true;
return read_rows;
}
}
if (!read_rows)
read_rows = rows_to_read;
size_t MergeTreeRangeReader::DelayedStream::finalize(Block & block)
{
if (current_offset && !continue_reading)
{
size_t granules_to_skip = current_offset / index_granularity;
current_mark += granules_to_skip;
current_offset -= granules_to_skip * index_granularity;
continue_reading = true;
if (current_offset)
{
Block temp_block;
readRows(temp_block, current_offset);
}
}
read_rows_after_current_mark += read_rows;
size_t read_parts = read_rows_after_current_mark / index_granularity;
current_mark += read_parts;
read_rows_after_current_mark -= index_granularity * read_parts;
size_t rows_to_read = num_delayed_rows;
current_offset += num_delayed_rows;
num_delayed_rows = 0;
if (current_mark == last_mark)
is_reading_finished = true;
return readRows(block, rows_to_read);
}
MergeTreeRangeReader::Stream::Stream(
size_t from_mark, size_t to_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader)
: current_mark(from_mark), offset_after_current_mark(0)
, index_granularity(index_granularity), last_mark(to_mark)
, stream(from_mark, index_granularity, merge_tree_reader)
{
}
void MergeTreeRangeReader::Stream::checkNotFinished() const
{
if (isFinished())
throw Exception("Cannot read out of marks range.", ErrorCodes::LOGICAL_ERROR);
}
void MergeTreeRangeReader::Stream::checkEnoughSpaceInCurrentGranule(size_t num_rows) const
{
if (num_rows + offset_after_current_mark > index_granularity)
throw Exception("Cannot read from granule more than index_granularity.", ErrorCodes::LOGICAL_ERROR);
}
size_t MergeTreeRangeReader::Stream::readRows(Block & block, size_t num_rows)
{
size_t rows_read = stream.read(block, current_mark, offset_after_current_mark, num_rows);
if (stream.isFinished())
finish();
return rows_read;
}
size_t MergeTreeRangeReader::Stream::read(Block & block, size_t num_rows, bool skip_remaining_rows_in_current_granule)
{
checkEnoughSpaceInCurrentGranule(num_rows);
if (num_rows)
{
checkNotFinished();
size_t read_rows = readRows(block, num_rows);
offset_after_current_mark += num_rows;
if (offset_after_current_mark == index_granularity || skip_remaining_rows_in_current_granule)
{
/// Start new granule; skipped_rows_after_offset is already zero.
++current_mark;
offset_after_current_mark = 0;
}
return read_rows;
}
else
{
/// Nothing to read.
if (skip_remaining_rows_in_current_granule)
{
/// Skip the rest of the rows in granule and start new one.
checkNotFinished();
++current_mark;
offset_after_current_mark = 0;
}
return 0;
}
}
void MergeTreeRangeReader::Stream::skip(size_t num_rows)
{
if (num_rows)
{
checkNotFinished();
checkEnoughSpaceInCurrentGranule(num_rows);
offset_after_current_mark += num_rows;
if (offset_after_current_mark == index_granularity)
{
/// Start new granule; skipped_rows_after_offset is already zero.
++current_mark;
offset_after_current_mark = 0;
}
}
}
size_t MergeTreeRangeReader::Stream::finalize(Block & block)
{
size_t read_rows = stream.finalize(block);
if (stream.isFinished())
finish();
return read_rows;
}
MergeTreeRangeReader MergeTreeRangeReader::copyForReader(MergeTreeReader & reader)
void MergeTreeRangeReader::ReadResult::addGranule(size_t num_rows)
{
MergeTreeRangeReader copy(reader, current_mark, last_mark, index_granularity);
copy.continue_reading = continue_reading;
copy.read_rows_after_current_mark = read_rows_after_current_mark;
return copy;
rows_per_granule.push_back(num_rows);
total_rows_per_granule += num_rows;
}
void MergeTreeRangeReader::ReadResult::adjustLastGranule()
{
size_t num_rows_to_subtract = total_rows_per_granule - num_read_rows;
if (rows_per_granule.empty())
throw Exception("Can't adjust last granule because no granules were added.", ErrorCodes::LOGICAL_ERROR);
if (num_rows_to_subtract > rows_per_granule.back())
throw Exception("Can't adjust last granule because it has " + toString(rows_per_granule.back())
+ "rows, but try to subtract " + toString(num_rows_to_subtract) + " rows.",
ErrorCodes::LOGICAL_ERROR);
rows_per_granule.back() -= num_rows_to_subtract;
total_rows_per_granule -= num_rows_to_subtract;
}
void MergeTreeRangeReader::ReadResult::clear()
{
/// Need to save information about the number of granules.
num_rows_to_skip_in_last_granule += rows_per_granule.back();
rows_per_granule.assign(rows_per_granule.size(), 0);
total_rows_per_granule = 0;
filter_holder = nullptr;
filter = nullptr;
}
void MergeTreeRangeReader::ReadResult::optimize()
{
if (total_rows_per_granule == 0 || filter == nullptr)
return;
NumRows zero_tails;
auto total_zero_rows_in_tails = countZeroTails(filter->getData(), zero_tails);
if (total_zero_rows_in_tails == filter->size())
{
clear();
return;
}
else if (total_zero_rows_in_tails == 0 && countBytesInFilter(filter->getData()) == filter->size())
{
filter_holder = nullptr;
filter = nullptr;
return;
}
/// Just a guess. If only a few rows may be skipped, it's better not to skip at all.
if (2 * total_zero_rows_in_tails > filter->size())
{
auto new_filter = ColumnUInt8::create(filter->size() - total_zero_rows_in_tails);
IColumn::Filter & new_data = new_filter->getData();
size_t rows_in_last_granule = rows_per_granule.back();
collapseZeroTails(filter->getData(), new_data, zero_tails);
total_rows_per_granule = new_filter->size();
num_rows_to_skip_in_last_granule += rows_in_last_granule - rows_per_granule.back();
filter = new_filter.get();
filter_holder = std::move(new_filter);
}
}
size_t MergeTreeRangeReader::ReadResult::countZeroTails(const IColumn::Filter & filter, NumRows & zero_tails) const
{
zero_tails.resize(0);
zero_tails.reserve(rows_per_granule.size());
auto filter_data = filter.data();
size_t total_zero_rows_in_tails = 0;
for (auto rows_to_read : rows_per_granule)
{
/// Count the number of zeros at the end of filter for rows were read from current granule.
zero_tails.push_back(numZerosInTail(filter_data, filter_data + rows_to_read));
total_zero_rows_in_tails += zero_tails.back();
filter_data += rows_to_read;
}
return total_zero_rows_in_tails;
}
void MergeTreeRangeReader::ReadResult::collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter,
const NumRows & zero_tails)
{
auto filter_data = filter.data();
auto new_filter_data = new_filter.data();
for (auto i : ext::range(0, rows_per_granule.size()))
{
auto & rows_to_read = rows_per_granule[i];
auto filtered_rows_num_at_granule_end = zero_tails[i];
rows_to_read -= filtered_rows_num_at_granule_end;
memcpySmallAllowReadWriteOverflow15(new_filter_data, filter_data, rows_to_read);
filter_data += rows_to_read;
new_filter_data += rows_to_read;
filter_data += filtered_rows_num_at_granule_end;
}
new_filter.resize(new_filter_data - new_filter.data());
}
size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, const UInt8 * end)
{
size_t count = 0;
#if __SSE2__ && __POPCNT__
const __m128i zero16 = _mm_setzero_si128();
while (end - begin >= 64)
{
end -= 64;
auto pos = end;
UInt64 val =
static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos)),
zero16)))
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 16)),
zero16))) << 16)
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 32)),
zero16))) << 32)
| (static_cast<UInt64>(_mm_movemask_epi8(_mm_cmpgt_epi8(
_mm_loadu_si128(reinterpret_cast<const __m128i *>(pos + 48)),
zero16))) << 48);
if (val == 0)
count += 64;
else
{
count += __builtin_clzll(val);
return count;
}
}
#endif
while (end > begin && *(--end) == 0)
{
++count;
}
return count;
}
void MergeTreeRangeReader::ReadResult::setFilter(const ColumnPtr & new_filter)
{
if (!new_filter && filter)
throw Exception("Can't replace existing filter with empty.", ErrorCodes::LOGICAL_ERROR);
if (filter)
{
size_t new_size = new_filter->size();
if (new_size != total_rows_per_granule)
throw Exception("Can't set filter because it's size is " + toString(new_size) + " but "
+ toString(total_rows_per_granule) + " rows was read.", ErrorCodes::LOGICAL_ERROR);
}
ConstantFilterDescription const_description(*new_filter);
if (const_description.always_false)
clear();
else if (!const_description.always_true)
{
FilterDescription filter_description(*new_filter);
filter_holder = filter_description.data_holder ? filter_description.data_holder : new_filter;
filter = typeid_cast<const ColumnUInt8 *>(filter_holder.get());
if (!filter)
throw Exception("setFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR);
}
}
MergeTreeRangeReader::MergeTreeRangeReader(
MergeTreeReader * merge_tree_reader, size_t index_granularity,
MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions,
const String * prewhere_column_name, const Names * ordered_names,
bool always_reorder, bool remove_prewhere_column, bool last_reader_in_chain)
: index_granularity(index_granularity), merge_tree_reader(merge_tree_reader)
, prev_reader(prev_reader), prewhere_column_name(prewhere_column_name)
, ordered_names(ordered_names), prewhere_actions(std::move(prewhere_actions))
, always_reorder(always_reorder), remove_prewhere_column(remove_prewhere_column)
, last_reader_in_chain(last_reader_in_chain), is_initialized(true)
{
}
bool MergeTreeRangeReader::isReadingFinished() const
{
return prev_reader ? prev_reader->isReadingFinished() : stream.isFinished();
}
size_t MergeTreeRangeReader::numReadRowsInCurrentGranule() const
{
return prev_reader ? prev_reader->numReadRowsInCurrentGranule() : stream.numReadRowsInCurrentGranule();
}
size_t MergeTreeRangeReader::numPendingRowsInCurrentGranule() const
{
if (prev_reader)
return prev_reader->numPendingRowsInCurrentGranule();
auto pending_rows = stream.numPendingRowsInCurrentGranule();
/// If pending_rows is zero, than stream is not initialized.
return pending_rows ? pending_rows : index_granularity;
}
bool MergeTreeRangeReader::isCurrentRangeFinished() const
{
return prev_reader ? prev_reader->isCurrentRangeFinished() : stream.isFinished();
}
MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, MarkRanges & ranges)
{
if (max_rows == 0)
throw Exception("Expected at least 1 row to read, got 0.", ErrorCodes::LOGICAL_ERROR);
ReadResult read_result;
size_t prev_bytes = 0;
if (prev_reader)
{
read_result = prev_reader->read(max_rows, ranges);
prev_bytes = read_result.block.bytes();
Block block = continueReadingChain(read_result);
bool should_reorder = false;
bool should_evaluate_missing_defaults = false;
if (block)
{
/// block.rows() <= read_result.block. We must filter block before adding columns to read_result.block
/// Fill missing columns before filtering because some arrays from Nested may have empty data.
merge_tree_reader->fillMissingColumns(block, should_reorder, should_evaluate_missing_defaults);
if (read_result.getFilter())
filterBlock(block, read_result.getFilter()->getData());
for (auto i : ext::range(0, block.columns()))
read_result.block.insert(std::move(block.getByPosition(i)));
}
if (read_result.block)
{
if (should_evaluate_missing_defaults)
merge_tree_reader->evaluateMissingDefaults(read_result.block);
if (should_reorder || always_reorder || block.columns())
merge_tree_reader->reorderColumns(read_result.block, *ordered_names);
}
}
else
{
read_result = startReadingChain(max_rows, ranges);
if (read_result.block)
{
bool should_reorder;
bool should_evaluate_missing_defaults;
merge_tree_reader->fillMissingColumns(read_result.block, should_reorder, should_evaluate_missing_defaults);
if (should_evaluate_missing_defaults)
merge_tree_reader->evaluateMissingDefaults(read_result.block);
if (should_reorder || always_reorder)
merge_tree_reader->reorderColumns(read_result.block, *ordered_names);
}
}
if (!read_result.block)
return read_result;
read_result.addNumBytesRead(read_result.block.bytes() - prev_bytes);
executePrewhereActionsAndFilterColumns(read_result);
return read_result;
}
void MergeTreeRangeReader::filterBlock(Block & block, const IColumn::Filter & filter) const
{
for (const auto i : ext::range(0, block.columns()))
{
auto & col = block.getByPosition(i);
if (col.column)
{
col.column = col.column->filter(filter, -1);
if (col.column->empty())
{
block.clear();
return;
}
}
}
}
MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t max_rows, MarkRanges & ranges)
{
ReadResult result;
/// Stream is lazy. result.num_added_rows is the number of rows added to block which is not equal to
/// result.num_rows_read until call to stream.finalize(). Also result.num_added_rows may be less than
/// result.num_rows_read if the last granule in range also the last in part (so we have to adjust last granule).
{
size_t space_left = max_rows;
while (space_left && (!stream.isFinished() || !ranges.empty()))
{
if (stream.isFinished())
{
result.addRows(stream.finalize(result.block));
stream = Stream(ranges.back().begin, ranges.back().end, index_granularity, merge_tree_reader);
result.addRange(ranges.back());
ranges.pop_back();
}
auto rows_to_read = std::min(space_left, stream.numPendingRowsInCurrentGranule());
bool last = rows_to_read == space_left;
result.addRows(stream.read(result.block, rows_to_read, !last));
result.addGranule(rows_to_read);
space_left -= rows_to_read;
}
}
result.addRows(stream.finalize(result.block));
/// Last granule may be incomplete.
result.adjustLastGranule();
return result;
}
Block MergeTreeRangeReader::continueReadingChain(ReadResult & result)
{
Block block;
if (result.rowsPerGranule().empty())
{
/// If zero rows were read on prev step, than there is no more rows to read.
/// Last granule may have less rows than index_granularity, so finish reading manually.
stream.finish();
return block;
}
auto & rows_per_granule = result.rowsPerGranule();
auto & started_ranges = result.startedRanges();
size_t added_rows = 0;
size_t next_range_to_start = 0;
auto size = rows_per_granule.size();
for (auto i : ext::range(0, size))
{
if (next_range_to_start < started_ranges.size()
&& i == started_ranges[next_range_to_start].num_granules_read_before_start)
{
added_rows += stream.finalize(block);
auto & range = started_ranges[next_range_to_start].range;
++next_range_to_start;
stream = Stream(range.begin, range.end, index_granularity, merge_tree_reader);
}
bool last = i + 1 == size;
added_rows += stream.read(block, rows_per_granule[i], !last);
}
stream.skip(result.numRowsToSkipInLastGranule());
added_rows += stream.finalize(block);
/// added_rows may be zero if all columns were read in prewhere and it's ok.
if (added_rows && added_rows != result.totalRowsPerGranule())
throw Exception("RangeReader read " + toString(added_rows) + " rows, but "
+ toString(result.totalRowsPerGranule()) + " expected.", ErrorCodes::LOGICAL_ERROR);
return block;
}
void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result)
{
if (!prewhere_actions)
return;
prewhere_actions->execute(result.block);
auto & prewhere_column = result.block.getByName(*prewhere_column_name);
size_t prev_rows = result.block.rows();
ColumnPtr filter = prewhere_column.column;
prewhere_column.column = nullptr;
if (result.getFilter())
{
/// TODO: implement for prewhere chain.
/// In order to do it we need combine filter and result.filter, where filter filters only '1' in result.filter.
throw Exception("MergeTreeRangeReader chain with several prewhere actions in not implemented.",
ErrorCodes::LOGICAL_ERROR);
}
result.setFilter(filter);
if (!last_reader_in_chain)
result.optimize();
bool filter_always_true = !result.getFilter() && result.totalRowsPerGranule() == filter->size();
if (result.totalRowsPerGranule() == 0)
result.block.clear();
else if (!filter_always_true)
{
FilterDescription filter_description(*filter);
if (last_reader_in_chain)
{
size_t num_bytes_in_filter = countBytesInFilter(*filter_description.data);
if (num_bytes_in_filter == 0)
result.block.clear();
else if (num_bytes_in_filter == filter->size())
filter_always_true = true;
}
if (!filter_always_true)
filterBlock(result.block, *filter_description.data);
}
if (!result.block)
return;
if (remove_prewhere_column)
result.block.erase(*prewhere_column_name);
else
{
/// Calculate the number of rows in block in order to create const column.
size_t rows = result.block.rows();
/// If block has single column, it's filter. We need to count bytes in it in order to get the number of rows.
if (result.block.columns() == 1)
{
if (result.getFilter())
rows = countBytesInFilter(result.getFilter()->getData());
else
rows = prev_rows;
}
prewhere_column.column = prewhere_column.type->createColumnConst(rows, UInt64(1));
}
}
}

View File

@ -1,10 +1,16 @@
#pragma once
#include <Core/Block.h>
#include <common/logger_useful.h>
#include <Interpreters/ExpressionActions.h>
#include <Storages/MergeTree/MarkRange.h>
namespace DB
{
template <typename T>
class ColumnVector;
using ColumnUInt8 = ColumnVector<UInt8>;
class MergeTreeReader;
/// MergeTreeReader iterator which allows sequential reading for arbitrary number of rows between pairs of marks in the same part.
@ -13,40 +19,170 @@ class MergeTreeReader;
class MergeTreeRangeReader
{
public:
size_t numPendingRows() const { return (last_mark - current_mark) * index_granularity - read_rows_after_current_mark; }
size_t numPendingRowsInCurrentGranule() const { return index_granularity - read_rows_after_current_mark; }
MergeTreeRangeReader(MergeTreeReader * merge_tree_reader, size_t index_granularity,
MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions,
const String * prewhere_column_name, const Names * ordered_names,
bool always_reorder, bool remove_prewhere_column, bool last_reader_in_chain);
size_t numReadRowsInCurrentGranule() const { return read_rows_after_current_mark; }
MergeTreeRangeReader() = default;
/// Seek to next mark before next reading.
size_t skipToNextMark();
/// Seturn state will be afrer reading rows_to_read, no reading happens.
MergeTreeRangeReader getFutureState(size_t rows_to_read) const;
bool isReadingFinished() const;
/// If columns are not present in the block, adds them. If they are present - appends the values that have been read.
/// Do not add columns, if the files are not present for them.
/// Block should contain either no columns from the columns field, or all columns for which files are present.
/// Returns the number of rows was read.
size_t read(Block & res, size_t max_rows_to_read);
size_t numReadRowsInCurrentGranule() const;
size_t numPendingRowsInCurrentGranule() const;
bool isReadingFinished() const { return is_reading_finished; }
bool isCurrentRangeFinished() const;
bool isInitialized() const { return is_initialized; }
void disableNextSeek() { continue_reading = true; }
/// Return the same state for other MergeTreeReader.
MergeTreeRangeReader copyForReader(MergeTreeReader & reader);
class DelayedStream
{
public:
DelayedStream() = default;
DelayedStream(size_t from_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader);
/// Returns the number of rows added to block.
/// NOTE: have to return number of rows because block has broken invariant:
/// some columns may have different size (for example, default columns may be zero size).
size_t read(Block & block, size_t from_mark, size_t offset, size_t num_rows);
size_t finalize(Block & block);
bool isFinished() const { return is_finished; }
private:
size_t current_mark = 0;
size_t current_offset = 0;
size_t num_delayed_rows = 0;
size_t index_granularity = 0;
MergeTreeReader * merge_tree_reader = nullptr;
bool continue_reading = false;
bool is_finished = true;
size_t position() const;
size_t readRows(Block & block, size_t num_rows);
};
class Stream
{
public:
Stream() = default;
Stream(size_t from_mark, size_t to_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader);
/// Returns the n
size_t read(Block & block, size_t num_rows, bool skip_remaining_rows_in_current_granule);
size_t finalize(Block & block);
void skip(size_t num_rows);
void finish() { current_mark = last_mark; }
bool isFinished() const { return current_mark >= last_mark; }
size_t numReadRowsInCurrentGranule() const { return offset_after_current_mark; }
size_t numPendingRowsInCurrentGranule() const { return index_granularity - numReadRowsInCurrentGranule(); }
size_t numRendingGranules() const { return last_mark - current_mark; }
size_t numPendingRows() const { return numRendingGranules() * index_granularity - offset_after_current_mark; }
private:
size_t current_mark = 0;
/// Invariant: offset_after_current_mark + skipped_rows_after_offset < index_granularity
size_t offset_after_current_mark = 0;
size_t index_granularity = 0;
size_t last_mark = 0;
DelayedStream stream;
void checkNotFinished() const;
void checkEnoughSpaceInCurrentGranule(size_t num_rows) const;
size_t readRows(Block & block, size_t num_rows);
};
/// Statistics after next reading step.
class ReadResult
{
public:
using NumRows = std::vector<size_t>;
struct RangeInfo
{
size_t num_granules_read_before_start;
MarkRange range;
};
using RangesInfo = std::vector<RangeInfo>;
const RangesInfo & startedRanges() const { return started_ranges; }
const NumRows & rowsPerGranule() const { return rows_per_granule; }
/// The number of rows were read at LAST iteration in chain. <= num_added_rows + num_filtered_rows.
size_t totalRowsPerGranule() const { return total_rows_per_granule; }
/// The number of rows were added to block as a result of reading chain.
size_t numReadRows() const { return num_read_rows; }
size_t numRowsToSkipInLastGranule() const { return num_rows_to_skip_in_last_granule; }
/// The number of bytes read from disk.
size_t numBytesRead() const { return num_bytes_read; }
/// Filter you need to apply to newly-read columns in order to add them to block.
const ColumnUInt8 * getFilter() const { return filter; }
void addGranule(size_t num_rows);
void adjustLastGranule();
void addRows(size_t rows) { num_read_rows += rows; }
void addRange(const MarkRange & range) { started_ranges.push_back({rows_per_granule.size(), range}); }
/// Set filter or replace old one. Filter must have more zeroes than previous.
void setFilter(const ColumnPtr & new_filter);
/// For each granule calculate the number of filtered rows at the end. Remove them and update filter.
void optimize();
/// Remove all rows from granules.
void clear();
void addNumBytesRead(size_t count) { num_bytes_read += count; }
Block block;
private:
RangesInfo started_ranges;
/// The number of rows read from each granule.
NumRows rows_per_granule;
/// Sum(rows_per_granule)
size_t total_rows_per_granule = 0;
/// The number of rows was read at first step. May be zero if no read columns present in part.
size_t num_read_rows = 0;
/// The number of rows was removed from last granule after clear or optimize.
size_t num_rows_to_skip_in_last_granule = 0;
/// Without any filtration.
size_t num_bytes_read = 0;
/// nullptr if prev reader hasn't prewhere_actions. Otherwise filter.size() >= total_rows_per_granule.
ColumnPtr filter_holder;
const ColumnUInt8 * filter = nullptr;
void collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter, const NumRows & zero_tails);
size_t countZeroTails(const IColumn::Filter & filter, NumRows & zero_tails) const;
static size_t numZerosInTail(const UInt8 * begin, const UInt8 * end);
};
ReadResult read(size_t max_rows, MarkRanges & ranges);
private:
MergeTreeRangeReader(MergeTreeReader & merge_tree_reader, size_t from_mark, size_t to_mark, size_t index_granularity);
std::reference_wrapper<MergeTreeReader> merge_tree_reader;
size_t current_mark;
size_t last_mark;
size_t read_rows_after_current_mark = 0;
size_t index_granularity;
bool continue_reading = false;
bool is_reading_finished = false;
ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges);
Block continueReadingChain(ReadResult & result);
void executePrewhereActionsAndFilterColumns(ReadResult & result);
void filterBlock(Block & block, const IColumn::Filter & filter) const;
friend class MergeTreeReader;
size_t index_granularity = 0;
MergeTreeReader * merge_tree_reader = nullptr;
MergeTreeRangeReader * prev_reader = nullptr; /// If not nullptr, read from prev_reader firstly.
const String * prewhere_column_name = nullptr;
const Names * ordered_names = nullptr;
ExpressionActionsPtr prewhere_actions = nullptr; /// If not nullptr, calculate filter.
Stream stream;
bool always_reorder = true;
bool remove_prewhere_column = false;
bool last_reader_in_chain = false;
bool is_initialized = false;
};
}

View File

@ -66,12 +66,6 @@ const MergeTreeReader::ValueSizeMap & MergeTreeReader::getAvgValueSizeHints() co
}
MergeTreeRangeReader MergeTreeReader::readRange(size_t from_mark, size_t to_mark)
{
return MergeTreeRangeReader(*this, from_mark, to_mark, storage.index_granularity);
}
size_t MergeTreeReader::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Block & res)
{
size_t read_rows = 0;
@ -414,8 +408,7 @@ static bool arrayHasNoElementsRead(const IColumn & column)
}
void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_names, bool always_reorder,
size_t rows, bool never_evaluate_defaults)
void MergeTreeReader::fillMissingColumns(Block & res, bool & should_reorder, bool & should_evaluate_missing_defaults)
{
if (!res)
throw Exception("Empty block passed to fillMissingColumns", ErrorCodes::LOGICAL_ERROR);
@ -443,8 +436,8 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name
}
}
bool should_evaluate_defaults = false;
bool should_sort = always_reorder;
should_evaluate_missing_defaults = false;
should_reorder = false;
/// insert default values only for columns without default expressions
for (const auto & requested_column : columns)
@ -462,10 +455,10 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name
if (!has_column)
{
should_sort = true;
should_reorder = true;
if (storage.getColumns().defaults.count(requested_column.name) != 0)
{
should_evaluate_defaults = true;
should_evaluate_missing_defaults = true;
continue;
}
@ -489,28 +482,46 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name
{
/// We must turn a constant column into a full column because the interpreter could infer that it is constant everywhere
/// but in some blocks (from other parts) it can be a full column.
column_to_add.column = column_to_add.type->createColumnConstWithDefaultValue(rows)->convertToFullColumnIfConst();
column_to_add.column = column_to_add.type->createColumnConstWithDefaultValue(res.rows())->convertToFullColumnIfConst();
}
res.insert(std::move(column_to_add));
}
}
/// evaluate defaulted columns if necessary
if (!never_evaluate_defaults && should_evaluate_defaults)
evaluateMissingDefaults(res, columns, storage.getColumns().defaults, storage.context);
/// sort columns to ensure consistent order among all blocks
if (!never_evaluate_defaults && should_sort)
{
Block ordered_block;
for (const auto & name : ordered_names)
if (res.has(name))
ordered_block.insert(res.getByName(name));
std::swap(res, ordered_block);
}
}
catch (Exception & e)
{
/// Better diagnostics.
e.addMessage("(while reading from part " + path + ")");
throw;
}
}
void MergeTreeReader::reorderColumns(Block & res, const Names & ordered_names)
{
try
{
Block ordered_block;
for (const auto & name : ordered_names)
if (res.has(name))
ordered_block.insert(res.getByName(name));
std::swap(res, ordered_block);
}
catch (Exception & e)
{
/// Better diagnostics.
e.addMessage("(while reading from part " + path + ")");
throw;
}
}
void MergeTreeReader::evaluateMissingDefaults(Block & res)
{
try
{
DB::evaluateMissingDefaults(res, columns, storage.getColumns().defaults, storage.context);
}
catch (Exception & e)
{

View File

@ -38,17 +38,16 @@ public:
const ValueSizeMap & getAvgValueSizeHints() const;
/// Create MergeTreeRangeReader iterator, which allows reading arbitrary number of rows from range.
MergeTreeRangeReader readRange(size_t from_mark, size_t to_mark);
/// Add columns from ordered_names that are not present in the block.
/// Missing columns are added in the order specified by ordered_names.
/// If at least one column was added, reorders all columns in the block according to ordered_names.
/// Rows is the number of rows will be used to create default columns. It's res.rows() when all columns in block has the same size.
/// This function may be used in order to create default columns in nested structures. In that case block may has
/// columns with different size, and default columns can't be evaluated, so never_evaluate_defaults = false is used.
void fillMissingColumns(Block & res, const Names & ordered_names, const bool always_reorder,
size_t rows, bool never_evaluate_defaults = false);
void fillMissingColumns(Block & res, bool & should_reorder, bool & should_evaluate_missing_defaults);
/// Sort columns to ensure consistent order among all blocks.
void reorderColumns(Block & res, const Names & ordered_names);
/// Evaluate defaulted columns if necessary.
void evaluateMissingDefaults(Block & res);
const NamesAndTypesList & getColumns() const { return columns; }
private:
class Stream
@ -121,7 +120,7 @@ private:
/// If continue_reading is true, continue reading from last state, otherwise seek to from_mark
size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Block & res);
friend class MergeTreeRangeReader;
friend class MergeTreeRangeReader::DelayedStream;
};
}

View File

@ -140,10 +140,10 @@ void ReplicatedMergeTreeAlterThread::run()
++changed_parts;
/// Update part metadata in ZooKeeper.
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(
zkutil::Requests ops;
ops.emplace_back(zkutil::makeSetRequest(
storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(
ops.emplace_back(zkutil::makeSetRequest(
storage.replica_path + "/parts/" + part->name + "/checksums",
storage.getChecksumsForZooKeeper(transaction->getNewChecksums()),
-1));
@ -155,7 +155,7 @@ void ReplicatedMergeTreeAlterThread::run()
catch (const zkutil::KeeperException & e)
{
/// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally.
if (e.code == ZNONODE)
if (e.code == ZooKeeperImpl::ZooKeeper::ZNONODE)
storage.enqueuePartForCheck(part->name);
throw;

View File

@ -60,9 +60,9 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe
{
quorum_info.status_path = storage.zookeeper_path + "/quorum/status";
zkutil::ZooKeeper::TryGetFuture quorum_status_future = zookeeper->asyncTryGet(quorum_info.status_path);
zkutil::ZooKeeper::TryGetFuture is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active");
zkutil::ZooKeeper::TryGetFuture host_future = zookeeper->asyncTryGet(storage.replica_path + "/host");
std::future<zkutil::GetResponse> quorum_status_future = zookeeper->asyncTryGet(quorum_info.status_path);
std::future<zkutil::GetResponse> is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active");
std::future<zkutil::GetResponse> host_future = zookeeper->asyncTryGet(storage.replica_path + "/host");
/// List of live replicas. All of them register an ephemeral node for leader_election.
@ -83,18 +83,18 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe
*/
auto quorum_status = quorum_status_future.get();
if (quorum_status.exists)
throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status.value, ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE);
if (quorum_status.error != ZooKeeperImpl::ZooKeeper::ZNONODE)
throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status.data, ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE);
/// Both checks are implicitly made also later (otherwise there would be a race condition).
auto is_active = is_active_future.get();
auto host = host_future.get();
if (!is_active.exists || !host.exists)
if (is_active.error == ZooKeeperImpl::ZooKeeper::ZNONODE || host.error == ZooKeeperImpl::ZooKeeper::ZNONODE)
throw Exception("Replica is not active right now", ErrorCodes::READONLY);
quorum_info.is_active_node_value = is_active.value;
quorum_info.is_active_node_value = is_active.data;
quorum_info.is_active_node_version = is_active.stat.version;
quorum_info.host_node_version = host.stat.version;
}
@ -205,21 +205,19 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned.
/// Also, make deduplication check. If a duplicate is detected, no nodes are created.
auto acl = zookeeper->getDefaultACL();
/// Deduplication stuff
bool deduplicate_block = !block_id.empty();
String block_id_path;
zkutil::Ops deduplication_check_ops;
zkutil::Ops * deduplication_check_ops_ptr = nullptr;
zkutil::Requests deduplication_check_ops;
zkutil::Requests * deduplication_check_ops_ptr = nullptr;
if (deduplicate_block)
{
block_id_path = storage.zookeeper_path + "/blocks/" + block_id;
/// Lets check for duplicates in advance, to avoid superflous block numbers allocation
deduplication_check_ops.emplace_back(std::make_shared<zkutil::Op::Create>(block_id_path, "", acl, zkutil::CreateMode::Persistent));
deduplication_check_ops.emplace_back(std::make_shared<zkutil::Op::Remove>(block_id_path, -1));
deduplication_check_ops.emplace_back(zkutil::makeCreateRequest(block_id_path, "", zkutil::CreateMode::Persistent));
deduplication_check_ops.emplace_back(zkutil::makeRemoveRequest(block_id_path, -1));
deduplication_check_ops_ptr = &deduplication_check_ops;
}
@ -229,11 +227,9 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// 2 RTT
block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, deduplication_check_ops_ptr);
}
catch (zkutil::KeeperMultiException & e)
catch (const zkutil::KeeperMultiException & e)
{
zkutil::MultiTransactionInfo & info = e.info;
if (deduplicate_block && info.code == ZNODEEXISTS && info.getFailedOp().getPath() == block_id_path)
if (deduplicate_block && e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && e.getPathForFirstFailedOp() == block_id_path)
{
LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (skip the insertion)");
part->is_duplicate = true;
@ -244,7 +240,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
}
catch (zkutil::KeeperException & e)
catch (const zkutil::KeeperException & e)
{
throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION);
}
@ -276,45 +272,40 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// Simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock.
/// Information about the part.
zkutil::Ops ops;
zkutil::Requests ops;
if (deduplicate_block)
{
/// Make final duplicate check and commit block_id
ops.emplace_back(
std::make_shared<zkutil::Op::Create>(
zkutil::makeCreateRequest(
block_id_path,
toString(block_number), /// We will able to know original part number for duplicate blocks, if we want.
acl,
zkutil::CreateMode::Persistent));
}
/// Information about the part, in the replica data.
ops.emplace_back(std::make_shared<zkutil::Op::Check>(
ops.emplace_back(zkutil::makeCheckRequest(
storage.zookeeper_path + "/columns",
storage.columns_version));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
ops.emplace_back(zkutil::makeCreateRequest(
storage.replica_path + "/parts/" + part->name,
"",
acl,
zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
ops.emplace_back(zkutil::makeCreateRequest(
storage.replica_path + "/parts/" + part->name + "/columns",
part->columns.toString(),
acl,
zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
ops.emplace_back(zkutil::makeCreateRequest(
storage.replica_path + "/parts/" + part->name + "/checksums",
storage.getChecksumsForZooKeeper(part->checksums),
acl,
zkutil::CreateMode::Persistent));
/// Replication log.
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
ops.emplace_back(zkutil::makeCreateRequest(
storage.zookeeper_path + "/log/log-",
log_entry.toString(),
acl,
zkutil::CreateMode::PersistentSequential));
/// Deletes the information that the block number is used for writing.
@ -339,15 +330,14 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
*/
ops.emplace_back(
std::make_shared<zkutil::Op::Create>(
zkutil::makeCreateRequest(
quorum_info.status_path,
quorum_entry.toString(),
acl,
zkutil::CreateMode::Persistent));
/// Make sure that during the insertion time, the replica was not reinitialized or disabled (when the server is finished).
ops.emplace_back(
std::make_shared<zkutil::Op::Check>(
zkutil::makeCheckRequest(
storage.replica_path + "/is_active",
quorum_info.is_active_node_version));
@ -355,7 +345,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// But then the `host` value will change. We will check this.
/// It's great that these two nodes change in the same transaction (see MergeTreeRestartingThread).
ops.emplace_back(
std::make_shared<zkutil::Op::Check>(
zkutil::makeCheckRequest(
storage.replica_path + "/host",
quorum_info.host_node_version));
}
@ -363,10 +353,10 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
MergeTreeData::Transaction transaction; /// If you can not add a part to ZK, we'll remove it back from the working set.
storage.data.renameTempPartAndAdd(part, nullptr, &transaction);
zkutil::MultiTransactionInfo info;
zookeeper->tryMultiNoThrow(ops, nullptr, &info); /// 1 RTT
zkutil::Responses responses;
int32_t multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT
if (info.code == ZOK)
if (multi_code == ZooKeeperImpl::ZooKeeper::ZOK)
{
transaction.commit();
storage.merge_selecting_event.set();
@ -374,11 +364,11 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// Lock nodes have been already deleted, do not delete them in destructor
block_number_lock.assumeUnlocked();
}
else if (zkutil::isUserError(info.code))
else if (zkutil::isUserError(multi_code))
{
String failed_op_path = info.getFailedOp().getPath();
String failed_op_path = zkutil::KeeperMultiException(multi_code, ops, responses).getPathForFirstFailedOp();
if (info.code == ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path)
if (multi_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path)
{
/// Block with the same id have just appeared in table (or other replica), rollback thee insertion.
LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")");
@ -388,7 +378,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
last_block_is_duplicate = true;
ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks);
}
else if (info.code == ZNODEEXISTS && failed_op_path == quorum_info.status_path)
else if (multi_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && failed_op_path == quorum_info.status_path)
{
transaction.rollback();
@ -399,32 +389,21 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo
/// NOTE: We could be here if the node with the quorum existed, but was quickly removed.
transaction.rollback();
throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': "
+ zkutil::ZooKeeper::error2string(info.code) + ", path " + failed_op_path,
+ zkutil::ZooKeeper::error2string(multi_code) + ", path " + failed_op_path,
ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR);
}
}
else if (zkutil::isTemporaryErrorCode(info.code))
{
/** If the connection is lost, and we do not know if the changes were applied, you can not delete the local part
* if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again.
*/
transaction.commit();
storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER);
/// We do not know whether or not data has been inserted.
throw Exception("Unknown status, client must retry. Reason: " + zkutil::ZooKeeper::error2string(info.code), ErrorCodes::UNKNOWN_STATUS_OF_INSERT);
}
else if (zkutil::isUnrecoverableErrorCode(info.code))
else if (zkutil::isHardwareError(multi_code))
{
transaction.rollback();
throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': "
+ zkutil::ZooKeeper::error2string(info.code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR);
+ zkutil::ZooKeeper::error2string(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR);
}
else
{
transaction.rollback();
throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': "
+ zkutil::ZooKeeper::error2string(info.code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR);
+ zkutil::ZooKeeper::error2string(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR);
}
if (quorum)

View File

@ -95,15 +95,15 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs()
if (entries.empty())
return;
zkutil::Ops ops;
zkutil::Requests ops;
for (size_t i = 0; i < entries.size(); ++i)
{
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(storage.zookeeper_path + "/log/" + entries[i], -1));
ops.emplace_back(zkutil::makeRemoveRequest(storage.zookeeper_path + "/log/" + entries[i], -1));
if (ops.size() > 4 * zkutil::MULTI_BATCH_SIZE || i + 1 == entries.size())
{
/// Simultaneously with clearing the log, we check to see if replica was added since we received replicas list.
ops.emplace_back(std::make_shared<zkutil::Op::Check>(storage.zookeeper_path + "/replicas", stat.version));
ops.emplace_back(zkutil::makeCheckRequest(storage.zookeeper_path + "/replicas", stat.version));
zookeeper->multi(ops);
ops.clear();
}
@ -148,7 +148,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime);
auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold);
std::vector<std::pair<String, zkutil::ZooKeeper::TryRemoveFuture>> try_remove_futures;
std::vector<std::pair<String, std::future<zkutil::RemoveResponse>>> try_remove_futures;
for (auto it = first_outdated_block; it != timed_blocks.end(); ++it)
{
String path = storage.zookeeper_path + "/blocks/" + it->node;
@ -158,13 +158,13 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks()
for (auto & pair : try_remove_futures)
{
const String & path = pair.first;
int32_t rc = pair.second.get();
if (rc == ZNOTEMPTY)
int32_t rc = pair.second.get().error;
if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY)
{
/// Can happen if there are leftover block nodes with children created by previous server versions.
zookeeper->removeRecursive(path);
}
else if (rc != ZOK)
else if (rc)
LOG_WARNING(log,
"Error while deleting ZooKeeper path `" << path << "`: " + zkutil::ZooKeeper::error2string(rc) << ", ignoring.");
}
@ -181,7 +181,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
Strings blocks;
zkutil::Stat stat;
if (ZOK != zookeeper.tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat))
if (zookeeper.tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat))
throw Exception(storage.zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);
/// Clear already deleted blocks from the cache, cached_block_ctime should be subset of blocks
@ -203,7 +203,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
<< " to clear old ones from ZooKeeper.");
}
std::vector<std::pair<String, zkutil::ZooKeeper::ExistsFuture>> exists_futures;
std::vector<std::pair<String, std::future<zkutil::ExistsResponse>>> exists_futures;
for (const String & block : blocks)
{
auto it = cached_block_stats.find(block);
@ -222,8 +222,8 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper &
/// Put fetched stats into the cache
for (auto & elem : exists_futures)
{
zkutil::ZooKeeper::StatAndExists status = elem.second.get();
if (status.exists)
auto status = elem.second.get();
if (status.error != ZooKeeperImpl::ZooKeeper::ZNONODE)
{
cached_block_stats.emplace(elem.first, status.stat.ctime);
timed_blocks.emplace_back(elem.first, status.stat.ctime);

View File

@ -2,6 +2,7 @@
#include <Core/Types.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h>
#include <common/logger_useful.h>
#include <thread>
#include <map>

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/Exception.h>
#include <Common/ZooKeeper/Types.h>
#include <Core/Types.h>
#include <IO/WriteHelpers.h>
@ -8,9 +9,6 @@
#include <condition_variable>
struct Stat;
namespace DB
{
@ -100,7 +98,7 @@ struct ReplicatedMergeTreeLogEntry : ReplicatedMergeTreeLogEntryData
std::condition_variable execution_complete; /// Awake when currently_executing becomes false.
static Ptr parse(const String & s, const Stat & stat);
static Ptr parse(const String & s, const zkutil::Stat & stat);
};

View File

@ -53,7 +53,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
std::sort(children.begin(), children.end());
std::vector<std::pair<String, zkutil::ZooKeeper::GetFuture>> futures;
std::vector<std::pair<String, std::future<zkutil::GetResponse>>> futures;
futures.reserve(children.size());
for (const String & child : children)
@ -61,8 +61,9 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper)
for (auto & future : futures)
{
zkutil::ZooKeeper::ValueAndStat res = future.second.get();
LogEntryPtr entry = LogEntry::parse(res.value, res.stat);
zkutil::GetResponse res = future.second.get();
LogEntryPtr entry = LogEntry::parse(res.data, res.stat);
entry->znode_name = future.first;
insertUnlocked(entry, min_unprocessed_insert_time_changed, lock);
@ -166,21 +167,22 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper(
/// because we update times in ZooKeeper with unlocked mutex, while these times may change.
/// Consider it unimportant (for a short time, ZK will have a slightly different time value).
zkutil::Ops ops;
zkutil::Requests ops;
if (min_unprocessed_insert_time_changed)
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(
ops.emplace_back(zkutil::makeSetRequest(
replica_path + "/min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1));
if (max_processed_insert_time_changed)
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(
ops.emplace_back(zkutil::makeSetRequest(
replica_path + "/max_processed_insert_time", toString(*max_processed_insert_time_changed), -1));
if (!ops.empty())
{
auto code = zookeeper->tryMulti(ops);
zkutil::Responses responses;
auto code = zookeeper->tryMulti(ops, responses);
if (code != ZOK)
if (code)
LOG_ERROR(log, "Couldn't set value of nodes for insert times ("
<< replica_path << "/min_unprocessed_insert_time, max_processed_insert_time)" << ": "
<< zkutil::ZooKeeper::error2string(code) + ". This shouldn't happen often.");
@ -192,7 +194,7 @@ void ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, LogEntryPt
{
auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name);
if (code != ZOK)
if (code)
LOG_ERROR(log, "Couldn't remove " << replica_path << "/queue/" << entry->znode_name << ": "
<< zkutil::ZooKeeper::error2string(code) << ". This shouldn't happen often.");
@ -261,13 +263,6 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
{
std::lock_guard<std::mutex> lock(pull_logs_to_queue_mutex);
bool dirty_entries_loaded = false;
if (is_dirty)
{
dirty_entries_loaded = load(zookeeper);
is_dirty = false;
}
String index_str = zookeeper->get(replica_path + "/log_pointer");
UInt64 index;
@ -320,7 +315,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
LOG_DEBUG(log, "Pulling " << (end - begin) << " entries to queue: " << *begin << " - " << *last);
std::vector<std::pair<String, zkutil::ZooKeeper::GetFuture>> futures;
std::vector<std::pair<String, std::future<zkutil::GetResponse>>> futures;
futures.reserve(end - begin);
for (auto it = begin; it != end; ++it)
@ -328,7 +323,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
/// Simultaneously add all new entries to the queue and move the pointer to the log.
zkutil::Ops ops;
zkutil::Requests ops;
std::vector<LogEntryPtr> copied_entries;
copied_entries.reserve(end - begin);
@ -336,11 +331,12 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
for (auto & future : futures)
{
zkutil::ZooKeeper::ValueAndStat res = future.second.get();
copied_entries.emplace_back(LogEntry::parse(res.value, res.stat));
zkutil::GetResponse res = future.second.get();
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
replica_path + "/queue/queue-", res.value, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
copied_entries.emplace_back(LogEntry::parse(res.data, res.stat));
ops.emplace_back(zkutil::makeCreateRequest(
replica_path + "/queue/queue-", res.data, zkutil::CreateMode::PersistentSequential));
const auto & entry = *copied_entries.back();
if (entry.type == LogEntry::GET_PART)
@ -354,27 +350,14 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
}
}
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(
ops.emplace_back(zkutil::makeSetRequest(
replica_path + "/log_pointer", toString(last_entry_index + 1), -1));
if (min_unprocessed_insert_time_changed)
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(
ops.emplace_back(zkutil::makeSetRequest(
replica_path + "/min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1));
try
{
zookeeper->multi(ops);
}
catch (const zkutil::KeeperException & ex)
{
if (ex.isTemporaryError())
{
LOG_WARNING(log, "Unknown status of queue update, marking queue dirty (will reload on next iteration).");
is_dirty = true;
}
throw;
}
auto responses = zookeeper->multi(ops);
/// Now we have successfully updated the queue in ZooKeeper. Update it in RAM.
@ -384,7 +367,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
for (size_t i = 0, size = copied_entries.size(); i < size; ++i)
{
String path_created = dynamic_cast<zkutil::Op::Create &>(*ops[i]).getPathCreated();
String path_created = dynamic_cast<const zkutil::CreateResponse &>(*responses[i]).path_created;
copied_entries[i]->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
std::optional<time_t> unused = false;
@ -411,7 +394,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z
next_update_event->set();
}
return dirty_entries_loaded || !log_entries.empty();
return !log_entries.empty();
}
@ -476,7 +459,7 @@ void ReplicatedMergeTreeQueue::removeGetsAndMergesInRange(zkutil::ZooKeeperPtr z
if ((*it)->currently_executing)
to_wait.push_back(*it);
auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name);
if (code != ZOK)
if (code)
LOG_INFO(log, "Couldn't remove " << replica_path + "/queue/" + (*it)->znode_name << ": "
<< zkutil::ZooKeeper::error2string(code));

View File

@ -52,10 +52,6 @@ private:
*/
Queue queue;
/// If true, the queue in RAM is possibly out of sync with ZK and we need to reload it.
/// Protected by pull_logs_to_queue_mutex.
bool is_dirty = false;
InsertsByTime inserts_by_time;
time_t min_unprocessed_insert_time = 0;
time_t max_processed_insert_time = 0;

View File

@ -260,10 +260,11 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts()
{
LOG_DEBUG(log, "Found part " << part_name << " with failed quorum. Moving to detached. This shouldn't happen often.");
zkutil::Ops ops;
zkutil::Requests ops;
zkutil::Responses responses;
storage.removePartFromZooKeeper(part_name, ops);
auto code = zookeeper->tryMulti(ops);
if (code == ZNONODE)
auto code = zookeeper->tryMulti(ops, responses);
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
LOG_WARNING(log, "Part " << part_name << " with failed quorum is not in ZooKeeper. This shouldn't happen often.");
storage.data.renameAndDetachPart(part, "noquorum");
@ -312,26 +313,25 @@ void ReplicatedMergeTreeRestartingThread::activateReplica()
* This is possible only when session in ZooKeeper expires.
*/
String data;
Stat stat;
zkutil::Stat stat;
bool has_is_active = zookeeper->tryGet(is_active_path, data, &stat);
if (has_is_active && data == active_node_identifier)
{
auto code = zookeeper->tryRemove(is_active_path, stat.version);
if (code == ZBADVERSION)
if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
throw Exception("Another instance of replica " + storage.replica_path + " was created just now."
" You shouldn't run multiple instances of same replica. You need to check configuration files.",
ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);
if (code != ZOK && code != ZNONODE)
if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE)
throw zkutil::KeeperException(code, is_active_path);
}
/// Simultaneously declare that this replica is active, and update the host.
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Create>(is_active_path,
active_node_identifier, zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral));
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(storage.replica_path + "/host", address.toString(), -1));
zkutil::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral));
ops.emplace_back(zkutil::makeSetRequest(storage.replica_path + "/host", address.toString(), -1));
try
{
@ -339,7 +339,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica()
}
catch (const zkutil::KeeperException & e)
{
if (e.code == ZNODEEXISTS)
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
throw Exception("Replica " + storage.replica_path + " appears to be already active. If you're sure it's not, "
"try again in a minute or remove znode " + storage.replica_path + "/is_active manually", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE);

View File

@ -192,9 +192,12 @@ BlockInputStreams StorageDistributed::read(
size_t num_remote_shards = cluster->getRemoteShardCount();
size_t result_size = (num_remote_shards * settings.max_parallel_replicas) + num_local_shards;
processed_stage = result_size == 1 || settings.distributed_group_by_no_merge
? QueryProcessingStage::Complete
: QueryProcessingStage::WithMergeableState;
if (settings.distributed_group_by_no_merge)
processed_stage = QueryProcessingStage::Complete;
else /// Normal mode.
processed_stage = result_size == 1
? QueryProcessingStage::Complete
: QueryProcessingStage::WithMergeableState;
const auto & modified_query_ast = rewriteSelectQuery(
query_info.query, remote_database, remote_table);

View File

@ -232,7 +232,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
catch (const zkutil::KeeperException & e)
{
/// Failed to connect to ZK (this became known when trying to perform the first operation).
if (e.code == ZCONNECTIONLOSS)
if (e.code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
current_zookeeper = nullptr;
@ -477,32 +477,31 @@ void StorageReplicatedMergeTree::createTableIfNotExists()
/// We write metadata of table so that the replicas can check table parameters with them.
String metadata = TableMetadata(data).toString();
auto acl = zookeeper->getDefaultACL();
zkutil::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata,
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", getColumns().toString(),
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/blocks", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/block_numbers", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/nonincrement_block_numbers", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/leader_election", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/temp", "",
zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "",
zkutil::CreateMode::Persistent));
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path, "",
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path + "/metadata", metadata,
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_unique<zkutil::Op::Create>(zookeeper_path + "/columns", getColumns().toString(),
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path + "/log", "",
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path + "/blocks", "",
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path + "/block_numbers", "",
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path + "/nonincrement_block_numbers", "",
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path + "/leader_election", "",
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path + "/temp", "",
acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(zookeeper_path + "/replicas", "",
acl, zkutil::CreateMode::Persistent));
auto code = zookeeper->tryMulti(ops);
if (code != ZOK && code != ZNODEEXISTS)
zkutil::Responses responses;
auto code = zookeeper->tryMulti(ops, responses);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
throw zkutil::KeeperException(code);
}
@ -571,14 +570,13 @@ void StorageReplicatedMergeTree::createReplica()
LOG_DEBUG(log, "Creating replica " << replica_path);
/// Create an empty replica. We'll create `columns` node at the end - we'll use it as a sign that replica creation is complete.
auto acl = zookeeper->getDefaultACL();
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Create>(replica_path, "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(replica_path + "/host", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(replica_path + "/log_pointer", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(replica_path + "/queue", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(replica_path + "/parts", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(replica_path + "/flags", "", acl, zkutil::CreateMode::Persistent));
zkutil::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/host", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_pointer", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/queue", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/parts", "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/flags", "", zkutil::CreateMode::Persistent));
try
{
@ -586,7 +584,7 @@ void StorageReplicatedMergeTree::createReplica()
}
catch (const zkutil::KeeperException & e)
{
if (e.code == ZNODEEXISTS)
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
throw Exception("Replica " + replica_path + " already exists.", ErrorCodes::REPLICA_IS_ALREADY_EXIST);
throw;
@ -604,7 +602,7 @@ void StorageReplicatedMergeTree::createReplica()
*/
String source_replica;
Stat stat;
zkutil::Stat stat;
zookeeper->exists(replica_path, &stat);
auto my_create_time = stat.czxid;
@ -837,7 +835,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
{
LOG_ERROR(log, "Adding unexpected local part to ZooKeeper: " << part->name);
zkutil::Ops ops;
zkutil::Requests ops;
checkPartChecksumsAndAddCommitOps(zookeeper, part, ops);
zookeeper->multi(ops);
}
@ -862,10 +860,10 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name);
/// We assume that this occurs before the queue is loaded (queue.initialize).
zkutil::Ops ops;
zkutil::Requests ops;
removePartFromZooKeeper(name, ops);
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential));
ops.emplace_back(zkutil::makeCreateRequest(
replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential));
zookeeper->multi(ops);
}
@ -879,7 +877,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks)
void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper,
const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String part_name, NameSet * absent_replicas_paths)
const MergeTreeData::DataPartPtr & part, zkutil::Requests & ops, String part_name, NameSet * absent_replicas_paths)
{
if (part_name.empty())
part_name = part->name;
@ -934,17 +932,16 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil:
if (!has_been_alredy_added)
{
auto acl = zookeeper->getDefaultACL();
String part_path = replica_path + "/parts/" + part_name;
ops.emplace_back(std::make_shared<zkutil::Op::Check>(
ops.emplace_back(zkutil::makeCheckRequest(
zookeeper_path + "/columns", expected_columns_version));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
part_path, "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
part_path + "/columns", part->columns.toString(), acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
part_path + "/checksums", getChecksumsForZooKeeper(part->checksums), acl, zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(
part_path, "", zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(
part_path + "/columns", part->columns.toString(), zkutil::CreateMode::Persistent));
ops.emplace_back(zkutil::makeCreateRequest(
part_path + "/checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent));
}
else
{
@ -960,7 +957,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
while (true)
{
zkutil::Ops ops;
zkutil::Requests ops;
NameSet absent_part_paths_on_replicas;
/// Checksums are checked here and `ops` is filled. In fact, the part is added to ZK just below, when executing `multi`.
@ -973,11 +970,11 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
/// Will check that the part did not suddenly appear on skipped replicas
if (!absent_part_paths_on_replicas.empty())
{
zkutil::Ops new_ops;
zkutil::Requests new_ops;
for (const String & part_path : absent_part_paths_on_replicas)
{
new_ops.emplace_back(std::make_shared<zkutil::Op::Create>(part_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
new_ops.emplace_back(std::make_shared<zkutil::Op::Remove>(part_path, -1));
new_ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent));
new_ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1));
}
/// Add check ops at the beginning
@ -990,14 +987,14 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd
zookeeper->multi(ops);
return transaction.commit();
}
catch (zkutil::KeeperMultiException & e)
catch (const zkutil::KeeperMultiException & e)
{
size_t num_check_ops = 2 * absent_part_paths_on_replicas.size();
size_t failed_op_index = zkutil::getFailedOpIndex(e.info.op_results, e.info.code);
size_t failed_op_index = e.failed_op_index;
if (failed_op_index < num_check_ops && e.info.code == ZNODEEXISTS)
if (failed_op_index < num_check_ops && e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
{
LOG_INFO(log, "The part " << e.info.getFailedOp().describe() << " on a replica suddenly appeared, will recheck checksums");
LOG_INFO(log, "The part " << e.getPathForFirstFailedOp() << " on a replica suddenly appeared, will recheck checksums");
}
else
throw;
@ -1347,14 +1344,14 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree::
Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas");
zkutil::Ops ops;
zkutil::Requests ops;
for (size_t i = 0, size = replicas.size(); i < size; ++i)
{
Stat stat;
zkutil::Stat stat;
String path = zookeeper_path + "/replicas/" + replicas[i] + "/host";
zookeeper->get(path, &stat);
ops.emplace_back(std::make_shared<zkutil::Op::Check>(path, stat.version));
ops.emplace_back(zkutil::makeCheckRequest(path, stat.version));
}
/// We verify that while we were collecting versions, the replica with the necessary part did not come alive.
@ -1365,7 +1362,7 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree::
if (replica.empty())
{
Stat quorum_stat;
zkutil::Stat quorum_stat;
String quorum_path = zookeeper_path + "/quorum/status";
String quorum_str = zookeeper->get(quorum_path, &quorum_stat);
ReplicatedMergeTreeQuorumEntry quorum_entry;
@ -1373,7 +1370,7 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree::
if (quorum_entry.part_name == entry.new_part_name)
{
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(quorum_path, quorum_stat.version));
ops.emplace_back(zkutil::makeRemoveRequest(quorum_path, quorum_stat.version));
auto part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version);
@ -1383,35 +1380,32 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree::
zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers/" + part_info.partition_id, "");
auto acl = zookeeper->getDefaultACL();
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
ops.emplace_back(zkutil::makeCreateRequest(
zookeeper_path + "/nonincrement_block_numbers/" + part_info.partition_id + "/block-" + padIndex(part_info.min_block),
"",
acl,
zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
ops.emplace_back(zkutil::makeCreateRequest(
zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name,
"",
acl,
zkutil::CreateMode::Persistent));
/// Deleting from `blocks`.
if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id))
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(zookeeper_path + "/blocks/" + entry.block_id, -1));
ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1));
auto code = zookeeper->tryMulti(ops);
zkutil::Responses responses;
auto code = zookeeper->tryMulti(ops, responses);
if (code == ZOK)
if (code == ZooKeeperImpl::ZooKeeper::ZOK)
{
LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed.");
return true; /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merges.
}
else if (code == ZBADVERSION || code == ZNONODE || code == ZNODEEXISTS)
else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || code == ZooKeeperImpl::ZooKeeper::ZNONODE || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
{
LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part "
<< entry.new_part_name << " as failed. Code: " << zerror(code));
<< entry.new_part_name << " as failed. Code: " << zkutil::ZooKeeper::error2string(code));
}
else
throw zkutil::KeeperException(code);
@ -1514,13 +1508,14 @@ void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTr
if (entry.detach)
data.renameAndDetachPart(part);
zkutil::Ops ops;
zkutil::Requests ops;
zkutil::Responses responses;
removePartFromZooKeeper(part->name, ops);
auto code = getZooKeeper()->tryMulti(ops);
auto code = getZooKeeper()->tryMulti(ops, responses);
/// If the part is already removed (for example, because it was never added to ZK due to crash,
/// see ReplicatedMergeTreeBlockOutputStream), then Ok.
if (code != ZOK && code != ZNONODE)
if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE)
throw zkutil::KeeperException(code);
/// If the part needs to be removed, it is more reliable to delete the directory after the changes in ZooKeeper.
@ -1571,10 +1566,10 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry &
continue;
/// Update part metadata in ZooKeeper.
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(
zkutil::Requests ops;
ops.emplace_back(zkutil::makeSetRequest(
replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1));
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(
ops.emplace_back(zkutil::makeSetRequest(
replica_path + "/parts/" + part->name + "/checksums", getChecksumsForZooKeeper(transaction->getNewChecksums()), -1));
zookeeper->multi(ops);
@ -1609,14 +1604,6 @@ void StorageReplicatedMergeTree::queueUpdatingThread()
update_in_progress = false;
queue_updating_event->wait();
}
catch (const zkutil::KeeperException & e)
{
if (e.code == ZINVALIDSTATE)
restarting_thread->wakeup();
tryLogCurrentException(__PRETTY_FUNCTION__);
queue_updating_event->tryWait(QUEUE_UPDATE_ERROR_SLEEP_MS);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
@ -2013,13 +2000,13 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts(
}
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Ops & ops)
void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Requests & ops)
{
String part_path = replica_path + "/parts/" + part_name;
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(part_path + "/checksums", -1));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(part_path + "/columns", -1));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(part_path, -1));
ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/checksums", -1));
ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/columns", -1));
ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1));
}
@ -2035,16 +2022,16 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n
log_entry->source_replica = "";
log_entry->new_part_name = part_name;
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Create>(
replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(),
zkutil::Requests ops;
ops.emplace_back(zkutil::makeCreateRequest(
replica_path + "/queue/queue-", log_entry->toString(),
zkutil::CreateMode::PersistentSequential));
removePartFromZooKeeper(part_name, ops);
auto results = zookeeper->multi(ops);
String path_created = dynamic_cast<zkutil::Op::Create &>(*ops[0]).getPathCreated();
String path_created = dynamic_cast<const zkutil::CreateResponse &>(*results[0]).path_created;
log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1);
queue.insert(zookeeper, log_entry);
}
@ -2185,21 +2172,22 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
{
/// The quorum is reached. Delete the node, and update information about the last part that was successfully written with quorum.
zkutil::Ops ops;
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(quorum_status_path, stat.version));
ops.emplace_back(std::make_shared<zkutil::Op::SetData>(quorum_last_part_path, part_name, -1));
auto code = zookeeper->tryMulti(ops);
zkutil::Requests ops;
zkutil::Responses responses;
ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version));
ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1));
auto code = zookeeper->tryMulti(ops, responses);
if (code == ZOK)
if (code == ZooKeeperImpl::ZooKeeper::ZOK)
{
break;
}
else if (code == ZNONODE)
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
{
/// The quorum has already been achieved.
break;
}
else if (code == ZBADVERSION)
else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
{
/// Node was updated meanwhile. We must re-read it and repeat all the actions.
continue;
@ -2212,16 +2200,16 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name)
/// We update the node, registering there one more replica.
auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version);
if (code == ZOK)
if (code == ZooKeeperImpl::ZooKeeper::ZOK)
{
break;
}
else if (code == ZNONODE)
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
{
/// The quorum has already been achieved.
break;
}
else if (code == ZBADVERSION)
else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
{
/// Node was updated meanwhile. We must re-read it and repeat all the actions.
continue;
@ -2975,7 +2963,7 @@ void StorageReplicatedMergeTree::drop()
/// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line.
Strings replicas;
if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty())
if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZooKeeperImpl::ZooKeeper::ZOK && replicas.empty())
{
LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)");
zookeeper->tryRemoveRecursive(zookeeper_path);
@ -3021,13 +3009,13 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path)
AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper,
zkutil::Ops * precheck_ops)
zkutil::Requests * precheck_ops)
{
String partition_path = zookeeper_path + "/block_numbers/" + partition_id;
if (!existsNodeCached(partition_path))
{
int code = zookeeper->tryCreate(partition_path, "", zkutil::CreateMode::Persistent);
if (code != ZOK && code != ZNODEEXISTS)
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
throw zkutil::KeeperException(code, partition_path);
}
@ -3648,26 +3636,10 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK()
}
static int32_t tryMultiWithRetries(zkutil::ZooKeeperPtr & zookeeper, zkutil::Ops & ops) noexcept
{
int32_t code;
try
{
code = zookeeper->tryMultiWithRetries(ops);
}
catch (const zkutil::KeeperException & e)
{
code = e.code;
}
return code;
}
void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
NameSet * parts_should_be_retied)
{
zkutil::Ops ops;
zkutil::Requests ops;
auto it_first_node_in_batch = part_names.cbegin();
for (auto it = part_names.cbegin(); it != part_names.cend(); ++it)
@ -3677,44 +3649,44 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr &
auto it_next = std::next(it);
if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend())
{
/// It is Ok to use multi with retries to delete nodes, because new nodes with the same names cannot appear here
auto code = tryMultiWithRetries(zookeeper, ops);
zkutil::Responses unused_responses;
auto code = zookeeper->tryMultiNoThrow(ops, unused_responses);
ops.clear();
if (code == ZNONODE)
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
{
/// Fallback
LOG_DEBUG(log, "There are no some part nodes in ZooKeeper, will remove part nodes sequentially");
for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
{
zkutil::Ops cur_ops;
zkutil::Requests cur_ops;
removePartFromZooKeeper(*it_in_batch, cur_ops);
auto cur_code = tryMultiWithRetries(zookeeper, cur_ops);
auto cur_code = zookeeper->tryMultiNoThrow(cur_ops, unused_responses);
if (cur_code == ZNONODE)
if (cur_code == ZooKeeperImpl::ZooKeeper::ZNONODE)
{
LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem");
}
else if (parts_should_be_retied && zkutil::isHardwareErrorCode(cur_code))
else if (parts_should_be_retied && zkutil::isHardwareError(cur_code))
{
parts_should_be_retied->emplace(*it_in_batch);
}
else if (cur_code != ZOK)
else if (cur_code)
{
LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << ::zerror(cur_code));
LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << zkutil::ZooKeeper::error2string(cur_code));
}
}
}
else if (parts_should_be_retied && zkutil::isHardwareErrorCode(code))
else if (parts_should_be_retied && zkutil::isHardwareError(code))
{
for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch)
parts_should_be_retied->emplace(*it_in_batch);
}
else if (code != ZOK)
else if (code)
{
LOG_WARNING(log, "There was a problem with deleting " << (it_next - it_first_node_in_batch)
<< " nodes from ZooKeeper: " << ::zerror(code));
<< " nodes from ZooKeeper: " << ::zkutil::ZooKeeper::error2string(code));
}
it_first_node_in_batch = it_next;
@ -3727,11 +3699,11 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num)
{
Strings blocks;
if (ZOK != zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks))
if (zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks))
throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE);
String partition_prefix = partition_id + "_";
std::vector<std::pair<String, zkutil::ZooKeeper::TryGetFuture>> get_futures;
std::vector<std::pair<String, std::future<zkutil::GetResponse>>> get_futures;
for (const String & block_id : blocks)
{
if (startsWith(block_id, partition_prefix))
@ -3741,16 +3713,16 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
}
}
std::vector<std::pair<String, zkutil::ZooKeeper::TryRemoveFuture>> to_delete_futures;
std::vector<std::pair<String, std::future<zkutil::RemoveResponse>>> to_delete_futures;
for (auto & pair : get_futures)
{
const String & path = pair.first;
zkutil::ZooKeeper::ValueAndStatAndExists result = pair.second.get();
auto result = pair.second.get();
if (!result.exists)
if (result.error == ZooKeeperImpl::ZooKeeper::ZNONODE)
continue;
ReadBufferFromString buf(result.value);
ReadBufferFromString buf(result.data);
Int64 block_num = 0;
bool parsed = tryReadIntText(block_num, buf) && buf.eof();
if (!parsed || (min_block_num <= block_num && block_num <= max_block_num))
@ -3760,13 +3732,13 @@ void StorageReplicatedMergeTree::clearBlocksInPartition(
for (auto & pair : to_delete_futures)
{
const String & path = pair.first;
int32_t rc = pair.second.get();
if (rc == ZNOTEMPTY)
int32_t rc = pair.second.get().error;
if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY)
{
/// Can happen if there are leftover block nodes with children created by previous server versions.
zookeeper.removeRecursive(path);
}
else if (rc != ZOK)
else if (rc)
LOG_WARNING(log,
"Error while deleting ZooKeeper path `" << path << "`: " + zkutil::ZooKeeper::error2string(rc) << ", ignoring.");
}

View File

@ -321,7 +321,7 @@ private:
* Call under TableStructureLock.
*/
void checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const MergeTreeData::DataPartPtr & part,
zkutil::Ops & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr);
zkutil::Requests & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr);
String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums);
@ -330,7 +330,7 @@ private:
const MergeTreeData::DataPartPtr & part);
/// Adds actions to `ops` that remove a part from ZooKeeper.
void removePartFromZooKeeper(const String & part_name, zkutil::Ops & ops);
void removePartFromZooKeeper(const String & part_name, zkutil::Requests & ops);
/// Quickly removes big set of parts from ZooKeeper (using async multi queries)
void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names,
@ -415,7 +415,7 @@ private:
/// Creates new block number and additionally perform precheck_ops while creates 'abandoned node'
AbandonableLockInZooKeeper allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper,
zkutil::Ops * precheck_ops = nullptr);
zkutil::Requests * precheck_ops = nullptr);
/** Wait until all replicas, including this, execute the specified action from the log.
* If replicas are added at the same time, it can not wait the added replica .

View File

@ -131,7 +131,7 @@ BlockInputStreams StorageSystemZooKeeper::read(
if (path_part == "/")
path_part.clear();
std::vector<zkutil::ZooKeeper::TryGetFuture> futures;
std::vector<std::future<zkutil::GetResponse>> futures;
futures.reserve(nodes.size());
for (const String & node : nodes)
futures.push_back(zookeeper->asyncTryGet(path_part + '/' + node));
@ -141,14 +141,14 @@ BlockInputStreams StorageSystemZooKeeper::read(
for (size_t i = 0, size = nodes.size(); i < size; ++i)
{
auto res = futures[i].get();
if (!res.exists)
if (res.error == ZooKeeperImpl::ZooKeeper::ZNONODE)
continue; /// Node was deleted meanwhile.
const zkutil::Stat & stat = res.stat;
size_t col_num = 0;
res_columns[col_num++]->insert(nodes[i]);
res_columns[col_num++]->insert(res.value);
res_columns[col_num++]->insert(res.data);
res_columns[col_num++]->insert(Int64(stat.czxid));
res_columns[col_num++]->insert(Int64(stat.mzxid));
res_columns[col_num++]->insert(UInt64(stat.ctime / 1000));

View File

@ -50,7 +50,7 @@ scp ./dbms/src/Server/clickhouse yourserver:~/clickhouse-tsan
## Start ClickHouse and run tests
```
sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1 suppressions=tsan_suppressions' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml
sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml
```

View File

@ -1,2 +0,0 @@
# ZooKeeper C library is a trash:
race:contrib/zookeeper

View File

@ -1,6 +1,6 @@
/usr/bin/clickhouse-client
/usr/bin/clickhouse-local
/usr/bin/clickhouse-compressor
/usr/bin/clickhouse-benchmark
/etc/clickhouse-client/config.xml
/usr/bin/clickhouse-extract-from-config
usr/bin/clickhouse-client
usr/bin/clickhouse-local
usr/bin/clickhouse-compressor
usr/bin/clickhouse-benchmark
etc/clickhouse-client/config.xml
usr/bin/clickhouse-extract-from-config

View File

@ -0,0 +1,3 @@
usr/bin/clickhouse
etc/security/limits.d/clickhouse.conf
usr/share/clickhouse/*

View File

@ -1,11 +0,0 @@
/usr/bin/clickhouse
/usr/bin/clickhouse-server
/usr/bin/clickhouse-clang
/usr/bin/clickhouse-lld
/usr/bin/clickhouse-copier
/usr/bin/clickhouse-report
/etc/systemd/system/clickhouse-server.service
/etc/init.d/clickhouse-server
/etc/cron.d/clickhouse-server
/usr/share/clickhouse/*
/etc/security/limits.d/clickhouse.conf

View File

@ -1,2 +0,0 @@
etc/clickhouse-server/config.xml etc/clickhouse-server
etc/clickhouse-server/users.xml etc/clickhouse-server

View File

@ -0,0 +1 @@
usr/bin/clickhouse usr/bin/clickhouse-server

8
debian/clickhouse-server.install vendored Normal file
View File

@ -0,0 +1,8 @@
usr/bin/clickhouse-server
usr/bin/clickhouse-clang
usr/bin/clickhouse-lld
usr/bin/clickhouse-copier
usr/bin/clickhouse-report
etc/clickhouse-server/config.xml
etc/clickhouse-server/users.xml
etc/systemd/system/clickhouse-server.service

35
debian/control vendored
View File

@ -13,11 +13,11 @@ Build-Depends: debhelper (>= 9),
libreadline-dev,
libssl-dev,
unixodbc-dev
Standards-Version: 3.8.0
Standards-Version: 3.9.8
Package: clickhouse-client
Architecture: any
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-server-base (= ${binary:Version})
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version})
Replaces: clickhouse-compressor
Conflicts: clickhouse-compressor
Description: Client binary for clickhouse
@ -26,28 +26,37 @@ Description: Client binary for clickhouse
.
This package provides clickhouse-client , clickhouse-local and clickhouse-benchmark
Package: clickhouse-server-base
Package: clickhouse-common-static
Architecture: any
Depends: ${shlibs:Depends}, ${misc:Depends}, adduser, tzdata
Depends: ${shlibs:Depends}, ${misc:Depends}
Replaces: clickhouse-server-base
Provides: clickhouse-server-base
Conflicts: clickhouse-server-base
Description: Common files for clickhouse
Yandex ClickHouse is a column-oriented database management system
that allows generating analytical data reports in real time.
.
This package provides common files for both clickhouse server and client
Package: clickhouse-server
Architecture: any
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}), adduser, tzdata
Replaces: clickhouse-server-common
Provides: clickhouse-server-common
Conflicts: clickhouse-server-common
Description: Server binary for clickhouse
Yandex ClickHouse is a column-oriented database management system
that allows generating analytical data reports in real time.
.
This package provides clickhouse common configuration files
Package: clickhouse-server-common
Architecture: any
Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-server-base (= ${binary:Version})
Description: clickhouse-server-common
Common configuration files for clickhouse-server-base package
Package: clickhouse-common-dbg
Architecture: any
Section: debug
Priority: extra
Depends: ${misc:Depends}, clickhouse-server-base (= ${binary:Version})
Description: debugging symbols for clickhouse-server-base
This package contains the debugging symbols for clickhouse-server-base.
Depends: ${misc:Depends}, clickhouse-common-static (= ${binary:Version})
Description: debugging symbols for clickhouse-common-static
This package contains the debugging symbols for clickhouse-common.
Package: clickhouse-test
Priority: extra

14
debian/rules vendored
View File

@ -66,32 +66,26 @@ override_dh_auto_test:
override_dh_clean:
rm -rf $(BUILDDIR)
rm -rf $(DESTDIR)
rm -rf debian/copyright debian/clickhouse-server-common.docs debian/clickhouse-client.docs
rm -rf debian/copyright debian/clickhouse-client.docs debian/clickhouse-common-static.docs
dh_clean
override_dh_strip:
dh_strip -pclickhouse-server-base --dbg-package=clickhouse-common-dbg
dh_strip -pclickhouse-common-static --dbg-package=clickhouse-common-dbg
override_dh_install:
# Making docs
cp LICENSE debian/copyright
ln -sf clickhouse-server-base.docs debian/clickhouse-client.docs
ln -sf clickhouse-server-base.docs debian/clickhouse-server-common.docs
ln -sf clickhouse-server.docs debian/clickhouse-client.docs
ln -sf clickhouse-server.docs debian/clickhouse-common-static.docs
mkdir -p $(DESTDIR)/etc/security/limits.d
cp debian/clickhouse.limits $(DESTDIR)/etc/security/limits.d/clickhouse.conf
# todo: remove after renaming package:
mkdir -p $(DESTDIR)/etc/init.d
cp debian/clickhouse-server.init $(DESTDIR)/etc/init.d/clickhouse-server
# systemd compatibility
mkdir -p $(DESTDIR)/etc/systemd/system/
cp debian/clickhouse-server.service $(DESTDIR)/etc/systemd/system/
mkdir -p $(DESTDIR)/etc/cron.d
cp debian/clickhouse-server.cron.d $(DESTDIR)/etc/cron.d/clickhouse-server
# In case building clickhouse-server, adding to package binary of clang, ld and header files - for dynamic compilation.
mkdir -p $(DESTDIR)/usr/share/clickhouse/headers

View File

@ -1,6 +1,6 @@
tar-ignore
tar-ignore="build"
tar-ignore="build_*"
tar-ignore="build/*"
tar-ignore="build_*/*"
tar-ignore="contrib/poco/openssl/*"
tar-ignore="contrib/poco/gradle/*"
tar-ignore="contrib/poco/Data/SQLite/*"

View File

@ -8,7 +8,7 @@ RUN apt-get update && \
mkdir -p /etc/apt/sources.list.d && \
echo $repository | tee /etc/apt/sources.list.d/clickhouse.list && \
apt-get update && \
apt-get install --allow-unauthenticated -y clickhouse-server-common=$version clickhouse-server-base=$version && \
apt-get install --allow-unauthenticated -y clickhouse-server=$version && \
rm -rf /var/lib/apt/lists/* /var/cache/debconf && \
apt-get clean

0
docs/en/agg_functions/combinators.md Normal file → Executable file
View File

1
docs/en/agg_functions/index.md Normal file → Executable file
View File

@ -8,3 +8,4 @@ ClickHouse also supports:
- [Parametric aggregate functions](parametric_functions.md#aggregate_functions_parametric), which accept other parameters in addition to columns.
- [Combinators](combinators.md#aggregate_functions_combinators), which change the behavior of aggregate functions.

0
docs/en/agg_functions/parametric_functions.md Normal file → Executable file
View File

34
docs/en/agg_functions/reference.md Normal file → Executable file
View File

@ -19,7 +19,7 @@ In some cases, you can rely on the order of execution. This applies to cases whe
When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. To get behavior like in MySQL, you can put the other columns in the `any` aggregate function.
## anyHeavy(x)
## anyHeavy
Selects a frequently occurring value using the [heavy hitters](http://www.cs.umd.edu/~samir/498/karp.pdf) algorithm. If there is a value that occurs more than in half the cases in each of the query's execution threads, this value is returned. Normally, the result is nondeterministic.
@ -39,7 +39,6 @@ Take the [OnTime](../getting_started/example_datasets/ontime.md#example_datasets
SELECT anyHeavy(AirlineID) AS res
FROM ontime
```
```
┌───res─┐
│ 19690 │
@ -125,11 +124,11 @@ The result is always Float64.
Calculates the approximate number of different values of the argument. Works for numbers, strings, dates, date-with-time, and for multiple arguments and tuple arguments.
Uses an adaptive sampling algorithm: for the calculation state, it uses a sample of element hash values with a size up to 65536.
This algorithm is also very accurate for data sets with small cardinality (up to 65536) and very efficient on CPU (when computing not too many of these functions, using `uniq` is almost as fast as using other aggregate functions).
This algorithm is also very accurate for data sets with low cardinality (up to 65536) and very efficient on CPU (when computing not too many of these functions, using `uniq` is almost as fast as using other aggregate functions).
The result is determinate (it doesn't depend on the order of query processing).
This function provides excellent accuracy even for data sets with huge cardinality (10B+ elements) and is recommended for use by default.
This function provides excellent accuracy even for data sets with extremely high cardinality (over 10 billion elements). It is recommended for default use.
## uniqCombined(x)
@ -139,16 +138,16 @@ A combination of three algorithms is used: array, hash table and [HyperLogLog](h
The result is determinate (it doesn't depend on the order of query processing).
The `uniqCombined` function is a good default choice for calculating the number of different values, but the following should be considered: for data sets with large cardinality (200M+) error of estimate will only grow and for data sets with huge cardinality(1B+ elements) it returns result with high inaccuracy.
The `uniqCombined` function is a good default choice for calculating the number of different values, but keep in mind that the estimation error will increase for high-cardinality data sets (200M+ elements), and the function will return very inaccurate results for data sets with extremely high cardinality (1B+ elements).
## uniqHLL12(x)
Uses the [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) algorithm to approximate the number of different values of the argument.
212 5-bit cells are used. The size of the state is slightly more than 2.5 KB. Result is not very accurate (error up to ~10%) for data sets of small cardinality(<10K elements), but for data sets with large cardinality (10K - 100M) result is quite accurate (error up to ~1.6%) and after that error of estimate will only grow and for data sets with huge cardinality (1B+ elements) it returns result with high inaccuracy.
212 5-bit cells are used. The size of the state is slightly more than 2.5 KB. The result is not very accurate (up to ~10% error) for small data sets (<10K elements). However, the result is fairly accurate for high-cardinality data sets (10K-100M), with a maximum error of ~1.6%. Starting from 100M, the estimation error increases, and the function will return very inaccurate results for data sets with extremely high cardinality (1B+ elements).
The result is determinate (it doesn't depend on the order of query processing).
This function is not recommended for use, and in most cases, use the `uniq` or `uniqCombined` function.
We don't recommend using this function. In most cases, use the `uniq` or `uniqCombined` function.
## uniqExact(x)
@ -170,7 +169,7 @@ In some cases, you can still rely on the order of execution. This applies to cas
<a name="agg_functions_groupArrayInsertAt"></a>
## groupArrayInsertAt(x)
## groupArrayInsertAt
Inserts a value into the array in the specified position.
@ -236,8 +235,8 @@ For its purpose (calculating quantiles of page loading times), using this functi
## quantileTimingWeighted(level)(x, weight)
Differs from the 'quantileTiming' function in that it has a second argument, "weights". Weight is a non-negative integer.
The result is calculated as if the 'x' value were passed 'weight' number of times to the 'quantileTiming' function.
Differs from the `quantileTiming` function in that it has a second argument, "weights". Weight is a non-negative integer.
The result is calculated as if the `x` value were passed `weight` number of times to the `quantileTiming` function.
## quantileExact(level)(x)
@ -257,7 +256,7 @@ The performance of the function is lower than for ` quantile`, ` quantileTiming`
The result depends on the order of running the query, and is nondeterministic.
## median(x)
## median
All the quantile functions have corresponding median functions: `median`, `medianDeterministic`, `medianTiming`, `medianTimingWeighted`, `medianExact`, `medianExactWeighted`, `medianTDigest`. They are synonyms and their behavior is identical.
@ -275,7 +274,7 @@ Returns `Float64`. When `n <= 1`, returns `+∞`.
## varPop(x)
Calculates the amount `Σ((x - x̅)^2) / n`, where `n` is the sample size and `x̅`is the average value of `x`.
Calculates the amount `Σ((x - x̅)^2) / (n - 1)`, where `n` is the sample size and `x̅`is the average value of `x`.
In other words, dispersion for a set of values. Returns `Float64`.
@ -287,30 +286,33 @@ The result is equal to the square root of `varSamp(x)`.
The result is equal to the square root of `varPop(x)`.
## topK(N)(column)
## topK
Returns an array of the most frequent values in the specified column. The resulting array is sorted in descending order of frequency of values (not by the values themselves).
Implements the [ Filtered Space-Saving](http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf) algorithm for analyzing TopK, based on the reduce-and-combine algorithm from [Parallel Space Saving](https://arxiv.org/pdf/1401.0702.pdf).
```
topK(N)(column)
```
This function doesn't provide a guaranteed result. In certain situations, errors might occur and it might return frequent values that aren't the most frequent values.
We recommend using the `N < 10 ` value; performance is reduced with large `N` values. Maximum value of ` N = 65536`.
**Arguments**
- 'N' The number of values.
- 'N' is the number of values.
- ' x ' The column.
**Example**
Take the [OnTime](../getting_started/example_datasets/ontime.md#example_datasets-ontime)data set and select the three most frequently occurring values in the `AirlineID` column.
Take the [OnTime](../getting_started/example_datasets/ontime.md#example_datasets-ontime) data set and select the three most frequently occurring values in the `AirlineID` column.
```sql
SELECT topK(3)(AirlineID) AS res
FROM ontime
```
```
┌─res─────────────────┐
│ [19393,19790,19805] │

0
docs/en/data_types/array.md Normal file → Executable file
View File

0
docs/en/data_types/boolean.md Normal file → Executable file
View File

2
docs/en/data_types/date.md Normal file → Executable file
View File

@ -1,6 +1,6 @@
# Date
Date. Stored in two bytes as the number of days since 1970-01-01 (unsigned). Allows storing values from just after the beginning of the Unix Epoch to the upper threshold defined by a constant at the compilation stage (currently, this is until the year 2038, but it may be expanded to 2106).
A date. Stored in two bytes as the number of days since 1970-01-01 (unsigned). Allows storing values from just after the beginning of the Unix Epoch to the upper threshold defined by a constant at the compilation stage (currently, this is until the year 2106, but the final fully-supported year is 2105).
The minimum value is output as 0000-00-00.
The date is stored without the time zone.

0
docs/en/data_types/datetime.md Normal file → Executable file
View File

0
docs/en/data_types/enum.md Normal file → Executable file
View File

0
docs/en/data_types/fixedstring.md Normal file → Executable file
View File

10
docs/en/data_types/float.md Normal file → Executable file
View File

@ -4,8 +4,8 @@
Types are equivalent to types of C:
- `Float32` - `float`;
- `Float64` - ` double`.
- `Float32` - `float`
- `Float64` - ` double`
We recommend that you store data in integer form whenever possible. For example, convert fixed precision numbers to integer values, such as monetary amounts or page load times in milliseconds.
@ -24,9 +24,7 @@ SELECT 1 - 0.9
```
- The result of the calculation depends on the calculation method (the processor type and architecture of the computer system).
- Floating-point calculations might result in numbers such as infinity (`Inf`) and "not-a-number" (`NaN`). This should be taken into account when processing the results of calculations.
- When reading floating point numbers from rows, the result might not be the nearest machine-representable number.
## NaN and Inf
@ -44,6 +42,7 @@ SELECT 0.5 / 0
│ inf │
└────────────────┘
```
- `-Inf` Negative infinity.
```sql
@ -55,6 +54,7 @@ SELECT -0.5 / 0
│ -inf │
└─────────────────┘
```
- `NaN` Not a number.
```
@ -67,5 +67,5 @@ SELECT 0 / 0
└──────────────┘
```
See the rules for ` NaN` sorting in the section [ORDER BY clause](../query_language/queries.md#query_language-queries-order_by).
See the rules for ` NaN` sorting in the section [ORDER BY clause](../query_language/queries.md#query_language-queries-order_by).

5
docs/en/data_types/index.md Normal file → Executable file
View File

@ -2,6 +2,7 @@
# Data types
ClickHouse table fields can contain data of different types.
ClickHouse can store various types of data in table cells.
This section describes the supported data types and special considerations when using and/or implementing them, if any.
The topic contains descriptions of data types supported and specificity of their usage of implementation if exists.

0
docs/en/data_types/int_uint.md Normal file → Executable file
View File

View File

0
docs/en/data_types/nested_data_structures/index.md Normal file → Executable file
View File

0
docs/en/data_types/nested_data_structures/nested.md Normal file → Executable file
View File

0
docs/en/data_types/special_data_types/expression.md Normal file → Executable file
View File

0
docs/en/data_types/special_data_types/index.md Normal file → Executable file
View File

0
docs/en/data_types/special_data_types/set.md Normal file → Executable file
View File

0
docs/en/data_types/string.md Normal file → Executable file
View File

0
docs/en/data_types/tuple.md Normal file → Executable file
View File

65
docs/en/development/style.md Normal file → Executable file
View File

@ -4,7 +4,7 @@
1. The following are recommendations, not requirements.
2. If you are editing code, it makes sense to follow the formatting of the existing code.
3. Code style is needed for consistency. Consistency makes it easier to read the code. and it also makes it easier to search the code.
3. Code style is needed for consistency. Consistency makes it easier to read the code, and it also makes it easier to search the code.
4. Many of the rules do not have logical reasons; they are dictated by established practices.
## Formatting
@ -93,25 +93,25 @@
14. In classes and structures, public, private, and protected are written on the same level as the class/struct, but all other internal elements should be deeper.
```cpp
template <typename T, typename Ptr = std::shared_ptr<T>>
class MultiVersion
{
public:
/// The specific version of the object to use.
using Version = Ptr;
...
}
template <typename T>
class MultiVersion
{
public:
/// Version of object for usage. shared_ptr manage lifetime of version.
using Version = std::shared_ptr<const T>;
...
}
```
15. If the same namespace is used for the entire file, and there isn't anything else significant, an offset is not necessary inside namespace.
16. If the block for if, for, while... expressions consists of a single statement, you don't need to use curly brackets. Place the statement on a separate line, instead. The same is true for a nested if, for, while... statement. But if the inner statement contains curly brackets or else, the external block should be written in curly brackets.
```cpp
/// Finish write.
for (auto & stream : streams)
stream.second->finalize();
```
```cpp
/// Finish write.
for (auto & stream : streams)
stream.second->finalize();
```
17. There should be any spaces at the ends of lines.
@ -218,11 +218,11 @@
*/
void executeQuery(
ReadBuffer & istr, /// Where to read the query from (and data for INSERT, if applicable)
WriteBuffer & ostr, /// Where to write the result
Context & context, /// DB, tables, data types, engines, functions, aggregate functions...
BlockInputStreamPtr & query_plan, /// A description of query processing can be included here
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// The last stage to process the SELECT query to
)
WriteBuffer & ostr, /// Where to write the result
Context & context, /// DB, tables, data types, engines, functions, aggregate functions...
BlockInputStreamPtr & query_plan, /// A description of query processing can be included here
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// The last stage to process the SELECT query to
)
```
4. Comments should be written in English only.
@ -252,7 +252,7 @@
*/
```
(Example taken from: [http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/)](http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/)
(the example is borrowed from the resource [http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/](http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/)
7. Do not write garbage comments (author, creation date ..) at the beginning of each file.
@ -497,7 +497,15 @@ This is not recommended, but it is allowed.
You can create a separate code block inside a single function in order to make certain variables local, so that the destructors are called when exiting the block.
```cpp
Block block = data.in->read();{ std::lock_guard<std::mutex> lock(mutex); data.ready = true; data.block = block;}ready_any.set();
Block block = data.in->read();
{
std::lock_guard<std::mutex> lock(mutex);
data.ready = true;
data.block = block;
}
ready_any.set();
```
7. Multithreading.
@ -560,13 +568,12 @@ This is not recommended, but it is allowed.
```cpp
using AggregateFunctionPtr = std::shared_ptr<IAggregateFunction>;
/** Creates an aggregate function by name.
*/
/** Creates an aggregate function by name. */
class AggregateFunctionFactory
{
public:
AggregateFunctionFactory();
AggregateFunctionFactory();
AggregateFunctionPtr get(const String & name, const DataTypes & argument_types) const;
```
@ -591,10 +598,10 @@ This is not recommended, but it is allowed.
If later youll need to delay initialization, you can add a default constructor that will create an invalid object. Or, for a small number of objects, you can use shared_ptr/unique_ptr.
```cpp
Loader(DB::Connection * connection_, const std::string & query, size_t max_block_size_);
/// For delayed initialization
Loader() {}
Loader(DB::Connection * connection_, const std::string & query, size_t max_block_size_);
/// For delayed initialization
Loader() {}
```
17. Virtual functions.

5
docs/en/dicts/external_dicts.md Normal file → Executable file
View File

@ -21,12 +21,11 @@ The dictionary config file has the following format:
<!--Optional element. File name with substitutions-->
<include_from>/etc/metrika.xml</include_from>
<dictionary>
<!-- Dictionary configuration -->
<!-- Dictionary configuration -->
</dictionary>
...
<dictionary>

8
docs/en/dicts/external_dicts_dict.md Normal file → Executable file
View File

@ -27,7 +27,7 @@ The dictionary configuration has the following structure:
```
- name The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`.
- [source](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources) Source of the dictionary.
- [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) Location of the dictionary in memory.
- [structure](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure) Structure of the dictionary. A key and attributes that can be retrieved by this key.
- [lifetime](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime) How frequently to update dictionaries.
- [source](external_dicts_dict_sources.md/#dicts-external_dicts_dict_sources) — Source of the dictionary .
- [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) — Dictionary layout in memory.
- [structure](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure) — Structure of the dictionary . A key and attributes that can be retrieved by this key.
- [lifetime](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime) — Frequency of dictionary updates.

0
docs/en/dicts/external_dicts_dict_layout.md Normal file → Executable file
View File

6
docs/en/dicts/external_dicts_dict_lifetime.md Normal file → Executable file
View File

@ -36,13 +36,13 @@ Example of settings:
When upgrading the dictionaries, the ClickHouse server applies different logic depending on the type of [ source](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources):
> - For a text file, it checks the time of modification. If the time differs from the previously recorded time, the dictionary is updated.
> - For MyISAM tables, the time of modification is checked using a `SHOW TABLE STATUS` query.
> - Dictionaries from other sources are updated every time by default.
- For MyISAM tables, the time of modification is checked using a `SHOW TABLE STATUS` query.
- Dictionaries from other sources are updated every time by default.
For MySQL (InnoDB) and ODBC sources, you can set up a query that will update the dictionaries only if they really changed, rather than each time. To do this, follow these steps:
> - The dictionary table must have a field that always changes when the source data is updated.
> - The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. The query must be specified in the `<invalidate_query>` field in the [ source](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources) settings.
- The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. Specify the query in the `<invalidate_query>` field in the settings for the [source](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources).
Example of settings:

6
docs/en/dicts/external_dicts_dict_sources.md Normal file → Executable file
View File

@ -80,7 +80,7 @@ Setting fields:
## HTTP(s)
Working with executable files depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method.
Working with an HTTP(s) server depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method.
Example of settings:
@ -135,9 +135,9 @@ Installing unixODBC and the ODBC driver for PostgreSQL:
Configuring `/etc/odbc.ini` (or `~/.odbc.ini`):
```
[DEFAULT]
[DEFAULT]
Driver = myconnection
[myconnection]
Description = PostgreSQL connection to my_db
Driver = PostgreSQL Unicode

14
docs/en/dicts/external_dicts_dict_structure.md Normal file → Executable file
View File

@ -25,8 +25,8 @@ Overall structure:
Columns are described in the structure:
- `<id>` [Key column](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-key).
- `<attribute>` [Data column](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-attributes). There can be a large number of columns.
- `<id>` - [key column](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-key).
- `<attribute>` - [data column](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-attributes). There can be a large number of columns.
<a name="dicts-external_dicts_dict_structure-key"></a>
@ -63,12 +63,10 @@ Configuration fields:
### Composite key
The key can be a `tuple` from any types of fields. The [ layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) in this case must be `complex_key_hashed` or `complex_key_cache`.
The key can be a `tuple` from any types of fields. The [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) in this case must be `complex_key_hashed` or `complex_key_cache`.
<div class="admonition tip">
A composite key can also consist of a single element, which makes it possible to use a string as the key, for instance.
A composite key can consist of a single element. This makes it possible to use a string as the key, for instance.
</div>
The key structure is set in the element `<key>`. Key fields are specified in the same format as the dictionary [attributes](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-attributes). Example:
@ -119,6 +117,6 @@ Configuration fields:
- `null_value` The default value for a non-existing element. In the example, it is an empty string.
- `expression` The attribute can be an expression. The tag is not required.
- `hierarchical` Hierarchical support. Mirrored to the parent identifier. By default, ` false`.
- `injective` Whether the `id -> attribute` image is injective. If ` true`, then you can optimize the ` GROUP BY` clause. By default, `false`.
- `is_object_id` - Used for query mongo documents by ObjectId
- `injective` Whether the `id -> attribute` image is injective. If ` true`, then you can optimize the ` GROUP BY` clause. By default, `false`.
- `is_object_id` Whether the query is executed for a MongoDB document by `ObjectID`.

0
docs/en/dicts/index.md Normal file → Executable file
View File

Some files were not shown because too many files have changed in this diff Show More