Merge branch 'master' into CLICKHOUSE-3847

This commit is contained in:
Vadim 2018-08-27 15:26:41 +03:00 committed by GitHub
commit be2d821669
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
242 changed files with 4493 additions and 2916 deletions

View File

@ -78,7 +78,7 @@ if (USE_STATIC_LIBRARIES)
list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES)
endif ()
if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "amd64.*|x86_64.*|AMD64.*")
if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64")
option (USE_INTERNAL_MEMCPY "Use internal implementation of 'memcpy' function instead of provided by libc. Only for x86_64." ON)
if (OS_LINUX)

View File

@ -26,9 +26,9 @@ endif ()
if (NOT Boost_SYSTEM_LIBRARY)
set (USE_INTERNAL_BOOST_LIBRARY 1)
set (Boost_PROGRAM_OPTIONS_LIBRARY boost_program_options_internal)
set (Boost_SYSTEM_LIBRARY boost_system_internal)
set (Boost_FILESYSTEM_LIBRARY boost_filesystem_internal)
set (Boost_PROGRAM_OPTIONS_LIBRARY boost_program_options_internal)
set (Boost_FILESYSTEM_LIBRARY boost_filesystem_internal ${Boost_SYSTEM_LIBRARY})
set (Boost_INCLUDE_DIRS)

View File

@ -1,6 +1,4 @@
if (NOT OS_FREEBSD AND NOT APPLE)
option (USE_INTERNAL_ZLIB_LIBRARY "Set to FALSE to use system zlib library instead of bundled" ${NOT_UNBUNDLED})
endif ()
if (NOT USE_INTERNAL_ZLIB_LIBRARY)
find_package (ZLIB)

View File

@ -104,14 +104,14 @@ endif ()
if (ENABLE_MYSQL AND USE_INTERNAL_MYSQL_LIBRARY)
add_subdirectory (mariadb-connector-c-cmake)
target_include_directories(mysqlclient PRIVATE BEFORE ${ZLIB_INCLUDE_DIR})
target_include_directories(mysqlclient PRIVATE BEFORE ${OPENSSL_INCLUDE_DIR})
target_include_directories(mysqlclient BEFORE PRIVATE ${ZLIB_INCLUDE_DIR})
target_include_directories(mysqlclient BEFORE PRIVATE ${OPENSSL_INCLUDE_DIR})
endif ()
if (USE_INTERNAL_RDKAFKA_LIBRARY)
add_subdirectory (librdkafka-cmake)
target_include_directories(rdkafka PRIVATE BEFORE ${ZLIB_INCLUDE_DIR})
target_include_directories(rdkafka PRIVATE BEFORE ${OPENSSL_INCLUDE_DIR})
target_include_directories(rdkafka BEFORE PRIVATE ${ZLIB_INCLUDE_DIR})
target_include_directories(rdkafka BEFORE PRIVATE ${OPENSSL_INCLUDE_DIR})
endif ()
if (ENABLE_ODBC AND USE_INTERNAL_ODBC_LIBRARY)

View File

@ -16,7 +16,7 @@ if (NOT MSVC)
add_definitions(-Wno-unused-variable -Wno-deprecated-declarations)
endif ()
add_library(boost_program_options_internal
add_library(boost_program_options_internal ${SPLIT_SHARED}
${LIBRARY_DIR}/libs/program_options/src/cmdline.cpp
${LIBRARY_DIR}/libs/program_options/src/config_file.cpp
${LIBRARY_DIR}/libs/program_options/src/convert.cpp
@ -29,7 +29,7 @@ ${LIBRARY_DIR}/libs/program_options/src/value_semantic.cpp
${LIBRARY_DIR}/libs/program_options/src/variables_map.cpp
${LIBRARY_DIR}/libs/program_options/src/winmain.cpp)
add_library(boost_filesystem_internal
add_library(boost_filesystem_internal ${SPLIT_SHARED}
${LIBRARY_DIR}/libs/filesystem/src/codecvt_error_category.cpp
${LIBRARY_DIR}/libs/filesystem/src/operations.cpp
${LIBRARY_DIR}/libs/filesystem/src/path.cpp
@ -39,9 +39,11 @@ ${LIBRARY_DIR}/libs/filesystem/src/unique_path.cpp
${LIBRARY_DIR}/libs/filesystem/src/utf8_codecvt_facet.cpp
${LIBRARY_DIR}/libs/filesystem/src/windows_file_codecvt.cpp)
add_library(boost_system_internal
add_library(boost_system_internal ${SPLIT_SHARED}
${LIBRARY_DIR}/libs/system/src/error_code.cpp)
target_link_libraries (boost_filesystem_internal PUBLIC boost_system_internal)
target_include_directories (boost_program_options_internal SYSTEM BEFORE PUBLIC ${Boost_INCLUDE_DIRS})
target_include_directories (boost_filesystem_internal SYSTEM BEFORE PUBLIC ${Boost_INCLUDE_DIRS})
target_include_directories (boost_system_internal SYSTEM BEFORE PUBLIC ${Boost_INCLUDE_DIRS})

View File

@ -1,6 +1,6 @@
SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/cctz)
add_library(cctz
add_library(cctz ${SPLIT_SHARED}
${LIBRARY_DIR}/src/civil_time_detail.cc
${LIBRARY_DIR}/src/time_zone_fixed.cc
${LIBRARY_DIR}/src/time_zone_format.cc

View File

@ -54,7 +54,7 @@ ${RDKAFKA_SOURCE_DIR}/lz4hc.c
${RDKAFKA_SOURCE_DIR}/rdgz.c
)
add_library(rdkafka STATIC ${SRCS})
add_library(rdkafka ${SPLIT_SHARED} ${SRCS})
target_include_directories(rdkafka PRIVATE include)
target_include_directories(rdkafka SYSTEM PUBLIC ${RDKAFKA_SOURCE_DIR})
target_link_libraries(rdkafka PUBLIC ${ZLIB_LIBRARIES} ${OPENSSL_SSL_LIBRARY} ${OPENSSL_CRYPTO_LIBRARY})

View File

@ -23,7 +23,7 @@ ${ODBC_SOURCE_DIR}/libltdl/loaders/preopen.c
${CMAKE_CURRENT_SOURCE_DIR}/linux_x86_64/libltdl/libltdlcS.c
)
add_library(ltdl STATIC ${SRCS})
add_library(ltdl ${SPLIT_SHARED} ${SRCS})
target_include_directories(ltdl PRIVATE ${CMAKE_CURRENT_SOURCE_DIR}/linux_x86_64/libltdl)
target_include_directories(ltdl PUBLIC ${ODBC_SOURCE_DIR}/libltdl)
@ -273,7 +273,7 @@ ${ODBC_SOURCE_DIR}/lst/lstSetFreeFunc.c
${ODBC_SOURCE_DIR}/lst/_lstVisible.c
)
add_library(unixodbc STATIC ${SRCS})
add_library(unixodbc ${SPLIT_SHARED} ${SRCS})
target_link_libraries(unixodbc ltdl)

View File

@ -125,6 +125,6 @@ IF (ZSTD_LEGACY_SUPPORT)
${LIBRARY_LEGACY_DIR}/zstd_v07.h)
ENDIF (ZSTD_LEGACY_SUPPORT)
ADD_LIBRARY(zstd ${Sources} ${Headers})
ADD_LIBRARY(zstd ${SPLIT_SHARED} ${Sources} ${Headers})
target_include_directories (zstd PUBLIC ${LIBRARY_DIR})

View File

@ -1,4 +1,5 @@
#include "TestHint.h"
#include "ConnectionParameters.h"
#include <port/unistd.h>
#include <stdlib.h>
@ -13,11 +14,11 @@
#include <optional>
#include <boost/program_options.hpp>
#include <boost/algorithm/string/replace.hpp>
#include <Poco/String.h>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <common/readline_use.h>
#include <common/find_first_symbols.h>
#include <common/SetTerminalEcho.h>
#include <Common/ClickHouseRevision.h>
#include <Common/Stopwatch.h>
#include <Common/Exception.h>
@ -57,7 +58,10 @@
#include <Common/InterruptListener.h>
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>
#include <ext/scope_guard.h>
#if USE_READLINE
#include "Suggest.h"
#endif
/// http://en.wikipedia.org/wiki/ANSI_escape_code
@ -72,7 +76,6 @@
#define DISABLE_LINE_WRAPPING "\033[?7l"
#define ENABLE_LINE_WRAPPING "\033[?7h"
namespace DB
{
@ -173,6 +176,8 @@ private:
int expected_client_error = 0;
int actual_server_error = 0;
int actual_client_error = 0;
UInt64 server_revision = 0;
String server_version;
String server_display_name;
@ -188,65 +193,6 @@ private:
/// External tables info.
std::list<ExternalTable> external_tables;
struct ConnectionParameters
{
String host;
UInt16 port;
String default_database;
String user;
String password;
Protocol::Secure security;
Protocol::Compression compression;
ConnectionTimeouts timeouts;
ConnectionParameters() {}
ConnectionParameters(const Poco::Util::AbstractConfiguration & config)
{
bool is_secure = config.getBool("secure", false);
security = is_secure
? Protocol::Secure::Enable
: Protocol::Secure::Disable;
host = config.getString("host", "localhost");
port = config.getInt("port",
config.getInt(is_secure ? "tcp_port_secure" : "tcp_port",
is_secure ? DBMS_DEFAULT_SECURE_PORT : DBMS_DEFAULT_PORT));
default_database = config.getString("database", "");
user = config.getString("user", "");
if (config.getBool("ask-password", false))
{
if (config.has("password"))
throw Exception("Specified both --password and --ask-password. Remove one of them", ErrorCodes::BAD_ARGUMENTS);
std::cout << "Password for user " << user << ": ";
SetTerminalEcho(false);
SCOPE_EXIT({
SetTerminalEcho(true);
});
std::getline(std::cin, password);
std::cout << std::endl;
}
else
{
password = config.getString("password", "");
}
compression = config.getBool("compression", true)
? Protocol::Compression::Enable
: Protocol::Compression::Disable;
timeouts = ConnectionTimeouts(
Poco::Timespan(config.getInt("connect_timeout", DBMS_DEFAULT_CONNECT_TIMEOUT_SEC), 0),
Poco::Timespan(config.getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0),
Poco::Timespan(config.getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0));
}
};
ConnectionParameters connection_parameters;
@ -343,7 +289,6 @@ private:
|| (now.month() == 1 && now.day() <= 5);
}
int mainImpl()
{
registerFunctions();
@ -459,9 +404,26 @@ private:
if (print_time_to_stderr)
throw Exception("time option could be specified only in non-interactive mode", ErrorCodes::BAD_ARGUMENTS);
#if USE_READLINE
if (server_revision >= Suggest::MIN_SERVER_REVISION
&& !config().getBool("disable_suggestion", false))
{
/// Load suggestion data from the server.
Suggest::instance().load(connection_parameters, config().getInt("suggestion_limit"));
/// Added '.' to the default list. Because it is used to separate database and table.
rl_basic_word_break_characters = " \t\n\r\"\\'`@$><=;|&{(.";
/// Not append whitespace after single suggestion. Because whitespace after function name is meaningless.
rl_completion_append_character = '\0';
rl_completion_entry_function = Suggest::generator;
}
else
#else
/// Turn tab completion off.
rl_bind_key('\t', rl_insert);
#endif
/// Load command history if present.
if (config().has("history_file"))
history_file = config().getString("history_file");
@ -482,10 +444,40 @@ private:
Poco::File(history_file).createFile();
}
#if USE_READLINE
/// Install Ctrl+C signal handler that will be used in interactive mode.
if (rl_initialize())
throw Exception("Cannot initialize readline", ErrorCodes::CANNOT_READLINE);
auto clear_prompt_or_exit = [](int)
{
/// This is signal safe.
ssize_t res = write(STDOUT_FILENO, "\n", 1);
/// Allow to quit client while query is in progress by pressing Ctrl+C twice.
/// (First press to Ctrl+C will try to cancel query by InterruptListener).
if (res == 1 && rl_line_buffer[0] && !RL_ISSTATE(RL_STATE_DONE))
{
rl_replace_line("", 0);
if (rl_forced_update_display())
_exit(0);
}
else
{
/// A little dirty, but we struggle to find better way to correctly
/// force readline to exit after returning from the signal handler.
_exit(0);
}
};
if (signal(SIGINT, clear_prompt_or_exit) == SIG_ERR)
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
#endif
loop();
std::cout << (isNewYearMode() ? "Happy new year." : "Bye.") << std::endl;
return 0;
}
else
@ -528,7 +520,6 @@ private:
UInt64 server_version_major = 0;
UInt64 server_version_minor = 0;
UInt64 server_version_patch = 0;
UInt64 server_revision = 0;
if (max_client_network_bandwidth)
{
@ -1518,33 +1509,6 @@ public:
}
}
#if USE_READLINE
if (rl_initialize())
throw Exception("Cannot initialize readline", ErrorCodes::CANNOT_READLINE);
auto clear_prompt_or_exit = [](int)
{
/// This is signal safe.
ssize_t res = write(STDOUT_FILENO, "\n", 1);
if (res == 1 && rl_line_buffer[0])
{
rl_replace_line("", 0);
if (rl_forced_update_display())
_exit(0);
}
else
{
/// A little dirty, but we struggle to find better way to correctly
/// force readline to exit after returning from the signal handler.
_exit(0);
}
};
if (signal(SIGINT, clear_prompt_or_exit) == SIG_ERR)
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
#endif
ioctl(0, TIOCGWINSZ, &terminal_size);
namespace po = boost::program_options;
@ -1566,7 +1530,7 @@ public:
("config-file,c", po::value<std::string>(), "config-file path")
("host,h", po::value<std::string>()->default_value("localhost"), "server host")
("port", po::value<int>()->default_value(9000), "server port")
("secure,s", "secure")
("secure,s", "Use TLS connection")
("user,u", po::value<std::string>()->default_value("default"), "user")
("password", po::value<std::string>(), "password")
("ask-password", "ask-password")
@ -1574,6 +1538,9 @@ public:
("query,q", po::value<std::string>(), "query")
("database,d", po::value<std::string>(), "database")
("pager", po::value<std::string>(), "pager")
("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.")
("suggestion_limit", po::value<int>()->default_value(10000),
"Suggestion limit for how many databases, tables and columns to fetch.")
("multiline,m", "multiline")
("multiquery,n", "multiquery")
("format,f", po::value<std::string>(), "default output format")
@ -1685,7 +1652,7 @@ public:
config().setInt("port", options["port"].as<int>());
if (options.count("secure"))
config().setBool("secure", true);
if (options.count("user"))
if (options.count("user") && !options["user"].defaulted())
config().setString("user", options["user"].as<std::string>());
if (options.count("password"))
config().setString("password", options["password"].as<std::string>());
@ -1717,12 +1684,15 @@ public:
config().setBool("compression", options["compression"].as<bool>());
if (options.count("server_logs_file"))
server_logs_file = options["server_logs_file"].as<std::string>();
if (options.count("disable_suggestion"))
config().setBool("disable_suggestion", true);
if (options.count("suggestion_limit"))
config().setInt("suggestion_limit", options["suggestion_limit"].as<int>());
}
};
}
int mainEntryClickHouseClient(int argc, char ** argv)
{
DB::Client client;

View File

@ -0,0 +1,83 @@
#pragma once
#include <iostream>
#include <Core/Types.h>
#include <Core/Protocol.h>
#include <Core/Defines.h>
#include <Common/Exception.h>
#include <IO/ConnectionTimeouts.h>
#include <common/SetTerminalEcho.h>
#include <ext/scope_guard.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
struct ConnectionParameters
{
String host;
UInt16 port;
String default_database;
String user;
String password;
Protocol::Secure security;
Protocol::Compression compression;
ConnectionTimeouts timeouts;
ConnectionParameters() {}
ConnectionParameters(const Poco::Util::AbstractConfiguration & config)
{
bool is_secure = config.getBool("secure", false);
security = is_secure
? Protocol::Secure::Enable
: Protocol::Secure::Disable;
host = config.getString("host", "localhost");
port = config.getInt("port",
config.getInt(is_secure ? "tcp_port_secure" : "tcp_port",
is_secure ? DBMS_DEFAULT_SECURE_PORT : DBMS_DEFAULT_PORT));
default_database = config.getString("database", "");
user = config.getString("user", "");
if (config.getBool("ask-password", false))
{
if (config.has("password"))
throw Exception("Specified both --password and --ask-password. Remove one of them", ErrorCodes::BAD_ARGUMENTS);
std::cout << "Password for user " << user << ": ";
SetTerminalEcho(false);
SCOPE_EXIT({
SetTerminalEcho(true);
});
std::getline(std::cin, password);
std::cout << std::endl;
}
else
{
password = config.getString("password", "");
}
compression = config.getBool("compression", true)
? Protocol::Compression::Enable
: Protocol::Compression::Disable;
timeouts = ConnectionTimeouts(
Poco::Timespan(config.getInt("connect_timeout", DBMS_DEFAULT_CONNECT_TIMEOUT_SEC), 0),
Poco::Timespan(config.getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0),
Poco::Timespan(config.getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0));
}
};
}

View File

@ -0,0 +1,217 @@
#pragma once
#include "ConnectionParameters.h"
#include <string>
#include <sstream>
#include <string.h>
#include <vector>
#include <algorithm>
#include <ext/singleton.h>
#include <common/readline_use.h>
#include <Common/typeid_cast.h>
#include <Columns/ColumnString.h>
#include <Client/Connection.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_PACKET_FROM_SERVER;
}
class Suggest : public ext::singleton<Suggest>
{
private:
/// The vector will be filled with completion words from the server and sorted.
using Words = std::vector<std::string>;
/// Keywords may be not up to date with ClickHouse parser.
Words words
{
"CREATE", "DATABASE", "IF", "NOT", "EXISTS", "TEMPORARY", "TABLE", "ON", "CLUSTER", "DEFAULT", "MATERIALIZED", "ALIAS", "ENGINE",
"AS", "VIEW", "POPULATE", "SETTINGS", "ATTACH", "DETACH", "DROP", "RENAME", "TO", "ALTER", "ADD", "MODIFY", "CLEAR", "COLUMN", "AFTER",
"COPY", "PROJECT", "PRIMARY", "KEY", "CHECK", "PARTITION", "PART", "FREEZE", "FETCH", "FROM", "SHOW", "INTO", "OUTFILE", "FORMAT", "TABLES",
"DATABASES", "LIKE", "PROCESSLIST", "CASE", "WHEN", "THEN", "ELSE", "END", "DESCRIBE", "DESC", "USE", "SET", "OPTIMIZE", "FINAL", "DEDUPLICATE",
"INSERT", "VALUES", "SELECT", "DISTINCT", "SAMPLE", "ARRAY", "JOIN", "GLOBAL", "LOCAL", "ANY", "ALL", "INNER", "LEFT", "RIGHT", "FULL", "OUTER",
"CROSS", "USING", "PREWHERE", "WHERE", "GROUP", "BY", "WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC", "IN",
"KILL", "QUERY", "SYNC", "ASYNC", "TEST"
};
/// Words are fetched asynchonously.
std::thread loading_thread;
std::atomic<bool> ready{false};
/// Points to current word to suggest.
Words::const_iterator pos;
/// Points after the last possible match.
Words::const_iterator end;
/// Set iterators to the matched range of words if any.
void findRange(const char * prefix, size_t prefix_length)
{
std::string prefix_str(prefix);
std::tie(pos, end) = std::equal_range(words.begin(), words.end(), prefix_str,
[prefix_length](const std::string & s, const std::string & prefix) { return strncmp(s.c_str(), prefix.c_str(), prefix_length) < 0; });
}
/// Iterates through matched range.
char * nextMatch()
{
if (pos >= end)
return nullptr;
/// readline will free memory by itself.
char * word = strdup(pos->c_str());
++pos;
return word;
}
void loadImpl(Connection & connection, size_t suggestion_limit)
{
std::stringstream query;
query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM ("
"SELECT name FROM system.functions"
" UNION ALL "
"SELECT name FROM system.table_engines"
" UNION ALL "
"SELECT name FROM system.formats"
" UNION ALL "
"SELECT name FROM system.table_functions"
" UNION ALL "
"SELECT name FROM system.data_type_families"
" UNION ALL "
"SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate";
/// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero.
if (suggestion_limit > 0)
{
String limit_str = toString(suggestion_limit);
query <<
" UNION ALL "
"SELECT name FROM system.databases LIMIT " << limit_str
<< " UNION ALL "
"SELECT DISTINCT name FROM system.tables LIMIT " << limit_str
<< " UNION ALL "
"SELECT DISTINCT name FROM system.columns LIMIT " << limit_str;
}
query << ") WHERE notEmpty(res)";
fetch(connection, query.str());
}
void fetch(Connection & connection, const std::string & query)
{
connection.sendQuery(query);
while (true)
{
Connection::Packet packet = connection.receivePacket();
switch (packet.type)
{
case Protocol::Server::Data:
fillWordsFromBlock(packet.block);
continue;
case Protocol::Server::Progress:
continue;
case Protocol::Server::ProfileInfo:
continue;
case Protocol::Server::Totals:
continue;
case Protocol::Server::Extremes:
continue;
case Protocol::Server::Log:
continue;
case Protocol::Server::Exception:
packet.exception->rethrow();
return;
case Protocol::Server::EndOfStream:
return;
default:
throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER);
}
}
}
void fillWordsFromBlock(const Block & block)
{
if (!block)
return;
if (block.columns() != 1)
throw Exception("Wrong number of columns received for query to read words for suggestion", ErrorCodes::LOGICAL_ERROR);
const ColumnString & column = typeid_cast<const ColumnString &>(*block.getByPosition(0).column);
size_t rows = block.rows();
for (size_t i = 0; i < rows; ++i)
words.emplace_back(column.getDataAt(i).toString());
}
public:
/// More old server versions cannot execute the query above.
static constexpr int MIN_SERVER_REVISION = 54406;
void load(const ConnectionParameters & connection_parameters, size_t suggestion_limit)
{
loading_thread = std::thread([connection_parameters, suggestion_limit, this]
{
try
{
Connection connection(
connection_parameters.host,
connection_parameters.port,
connection_parameters.default_database,
connection_parameters.user,
connection_parameters.password,
connection_parameters.timeouts,
"client",
connection_parameters.compression,
connection_parameters.security);
loadImpl(connection, suggestion_limit);
}
catch (...)
{
std::cerr << "Cannot load data for command line suggestions: " << getCurrentExceptionMessage(false) << "\n";
}
/// Note that keyword suggestions are available even if we cannot load data from server.
std::sort(words.begin(), words.end());
ready = true;
});
}
/// A function for readline.
static char * generator(const char * text, int state)
{
Suggest & suggest = Suggest::instance();
if (!suggest.ready)
return nullptr;
if (state == 0)
suggest.findRange(text, strlen(text));
/// Do not append whitespace after word. For unknown reason, rl_completion_append_character = '\0' does not work.
rl_completion_suppress_append = 1;
return suggest.nextMatch();
}
~Suggest()
{
if (loading_thread.joinable())
loading_thread.join();
}
};
}

View File

@ -363,8 +363,8 @@ struct TaskCluster
struct MultiTransactionInfo
{
int32_t code;
zkutil::Requests requests;
zkutil::Responses responses;
Coordination::Requests requests;
Coordination::Responses responses;
};
@ -373,7 +373,7 @@ struct MultiTransactionInfo
static MultiTransactionInfo checkNoNodeAndCommit(
const zkutil::ZooKeeperPtr & zookeeper,
const String & checking_node_path,
zkutil::RequestPtr && op)
Coordination::RequestPtr && op)
{
MultiTransactionInfo info;
info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent));
@ -742,7 +742,7 @@ public:
{
auto zookeeper = context.getZooKeeper();
task_description_watch_callback = [this] (const ZooKeeperImpl::ZooKeeper::WatchResponse &)
task_description_watch_callback = [this] (const Coordination::WatchResponse &)
{
UInt64 version = ++task_descprtion_version;
LOG_DEBUG(log, "Task description should be updated, local version " << version);
@ -902,7 +902,7 @@ public:
task_description_watch_zookeeper = zookeeper;
String task_config_str;
zkutil::Stat stat;
Coordination::Stat stat;
int code;
zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code);
@ -1052,7 +1052,7 @@ protected:
{
updateConfigIfNeeded();
zkutil::Stat stat;
Coordination::Stat stat;
zookeeper->get(workers_version_path, &stat);
auto version = stat.version;
zookeeper->get(workers_path, &stat);
@ -1070,16 +1070,16 @@ protected:
}
else
{
zkutil::Requests ops;
Coordination::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;
Coordination::Responses responses;
auto code = zookeeper->tryMulti(ops, responses);
if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS)
return std::make_shared<zkutil::EphemeralNodeHolder>(current_worker_path, *zookeeper, false, false, description);
if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)
if (code == Coordination::ZBADVERSION)
{
++num_bad_version_errors;
@ -1093,7 +1093,7 @@ protected:
}
}
else
throw zkutil::KeeperException(code);
throw Coordination::Exception(code);
}
}
}
@ -1157,7 +1157,7 @@ protected:
zxid2.push_back(res.stat.pzxid);
}
}
catch (const zkutil::KeeperException & e)
catch (const Coordination::Exception & e)
{
LOG_INFO(log, "A ZooKeeper error occurred while checking partition " << partition_name
<< ". Will recheck the partition. Error: " << e.displayText());
@ -1242,9 +1242,9 @@ protected:
{
cleaner_holder = zkutil::EphemeralNodeHolder::create(dirt_cleaner_path, *zookeeper, host_id);
}
catch (const zkutil::KeeperException & e)
catch (const Coordination::Exception & e)
{
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
if (e.code == Coordination::ZNODEEXISTS)
{
LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep");
std::this_thread::sleep_for(default_sleep_time);
@ -1254,7 +1254,7 @@ protected:
throw;
}
zkutil::Stat stat;
Coordination::Stat stat;
if (zookeeper->exists(current_partition_active_workers_dir, &stat))
{
if (stat.numChildren != 0)
@ -1291,7 +1291,7 @@ protected:
}
/// Remove the locking node
zkutil::Requests requests;
Coordination::Requests requests;
requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1));
requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1));
zookeeper->multi(requests);
@ -1503,8 +1503,8 @@ protected:
auto create_is_dirty_node = [&] ()
{
auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
throw zkutil::KeeperException(code, is_dirty_flag_path);
if (code && code != Coordination::ZNODEEXISTS)
throw Coordination::Exception(code, is_dirty_flag_path);
};
/// Returns SELECT query filtering current partition and applying user filter
@ -1552,9 +1552,9 @@ protected:
{
partition_task_node_holder = zkutil::EphemeralNodeHolder::create(current_task_is_active_path, *zookeeper, host_id);
}
catch (const zkutil::KeeperException & e)
catch (const Coordination::Exception & e)
{
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
if (e.code == Coordination::ZNODEEXISTS)
{
LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path);
return PartitionTaskStatus::Active;
@ -1605,7 +1605,7 @@ protected:
if (count != 0)
{
zkutil::Stat stat_shards;
Coordination::Stat stat_shards;
zookeeper->get(task_partition.getPartitionShardsPath(), &stat_shards);
if (stat_shards.numChildren == 0)
@ -1698,7 +1698,7 @@ protected:
output = io_insert.out;
}
std::future<zkutil::ExistsResponse> future_is_dirty_checker;
std::future<Coordination::ExistsResponse> future_is_dirty_checker;
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
constexpr size_t check_period_milliseconds = 500;
@ -1716,9 +1716,9 @@ protected:
/// Otherwise, the insertion will slow a little bit
if (watch.elapsedMilliseconds() >= check_period_milliseconds)
{
zkutil::ExistsResponse status = future_is_dirty_checker.get();
Coordination::ExistsResponse status = future_is_dirty_checker.get();
if (status.error != ZooKeeperImpl::ZooKeeper::ZNONODE)
if (status.error != Coordination::ZNONODE)
throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED);
}
@ -2023,13 +2023,13 @@ private:
/// Auto update config stuff
UInt64 task_descprtion_current_version = 1;
std::atomic<UInt64> task_descprtion_version{1};
zkutil::WatchCallback task_description_watch_callback;
Coordination::WatchCallback task_description_watch_callback;
/// ZooKeeper session used to set the callback
zkutil::ZooKeeperPtr task_description_watch_zookeeper;
ConfigurationPtr task_cluster_initial_config;
ConfigurationPtr task_cluster_current_config;
zkutil::Stat task_descprtion_current_stat;
Coordination::Stat task_descprtion_current_stat;
std::unique_ptr<TaskCluster> task_cluster;

View File

@ -104,8 +104,8 @@ try
if (!config().has("query") && !config().has("table-structure")) /// Nothing to process
{
if (!config().hasOption("silent"))
std::cerr << "There are no queries to process." << std::endl;
if (config().hasOption("verbose"))
std::cerr << "There are no queries to process." << '\n';
return Application::EXIT_OK;
}
@ -200,8 +200,7 @@ try
}
catch (const Exception & e)
{
if (!config().hasOption("silent"))
std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace"));
std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n';
/// If exception code isn't zero, we should return non-zero return code anyway.
return e.code() ? e.code() : -1;
@ -274,7 +273,7 @@ void LocalServer::processQueries()
/// Use the same query_id (and thread group) for all queries
CurrentThread::QueryScope query_scope_holder(*context);
bool echo_query = config().hasOption("echo") || config().hasOption("verbose");
bool echo_queries = config().hasOption("echo") || config().hasOption("verbose");
std::exception_ptr exception;
for (const auto & query : queries)
@ -282,8 +281,12 @@ void LocalServer::processQueries()
ReadBufferFromString read_buf(query);
WriteBufferFromFileDescriptor write_buf(STDOUT_FILENO);
if (echo_query)
std::cerr << query << "\n";
if (echo_queries)
{
writeString(query, write_buf);
writeChar('\n', write_buf);
write_buf.next();
}
try
{
@ -297,8 +300,7 @@ void LocalServer::processQueries()
if (!exception)
exception = std::current_exception();
if (!config().has("silent"))
std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace"));
std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n';
}
}
@ -360,7 +362,7 @@ void LocalServer::setupUsers()
static void showClientVersion()
{
std::cout << DBMS_NAME << " client version " << VERSION_STRING << "." << std::endl;
std::cout << DBMS_NAME << " client version " << VERSION_STRING << "." << '\n';
}
std::string LocalServer::getHelpHeader() const
@ -421,7 +423,6 @@ void LocalServer::init(int argc, char ** argv)
("format,f", po::value<std::string>(), "default output format (clickhouse-client compatibility)")
("output-format", po::value<std::string>(), "default output format")
("silent,s", "quiet mode, do not print errors")
("stacktrace", "print stack traces of exceptions")
("echo", "print query before execution")
("verbose", "print query and other debugging info")
@ -477,8 +478,6 @@ void LocalServer::init(int argc, char ** argv)
if (options.count("output-format"))
config().setString("output-format", options["output-format"].as<std::string>());
if (options.count("silent"))
config().setBool("silent", true);
if (options.count("stacktrace"))
config().setBool("stacktrace", true);
if (options.count("echo"))
@ -507,7 +506,7 @@ int mainEntryClickHouseLocal(int argc, char ** argv)
}
catch (...)
{
std::cerr << DB::getCurrentExceptionMessage(true) << "\n";
std::cerr << DB::getCurrentExceptionMessage(true) << '\n';
auto code = DB::getCurrentExceptionCode();
return code ? code : 1;
}

View File

@ -10,6 +10,9 @@
#include <IO/WriteBufferFromHTTPServerResponse.h>
#include <IO/WriteHelpers.h>
#include <Common/HTMLForm.h>
#include <Parsers/ParserQueryWithOutput.h>
#include <Parsers/parseQuery.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include "validateODBCConnectionString.h"
@ -51,7 +54,8 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
Poco::Net::HTMLForm params(request, request.stream());
LOG_TRACE(log, "Request URI: " + request.getURI());
auto process_error = [&response, this](const std::string & message) {
auto process_error = [&response, this](const std::string & message)
{
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
if (!response.sent())
response.send() << message << std::endl;
@ -68,8 +72,14 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
process_error("No 'connection_string' in request URL");
return;
}
std::string schema_name = "";
std::string table_name = params.get("table");
std::string connection_string = params.get("connection_string");
if (params.has("schema"))
{
schema_name = params.get("schema");
LOG_TRACE(log, "Will fetch info for table '" << schema_name + "." + table_name << "'");
} else
LOG_TRACE(log, "Will fetch info for table '" << table_name << "'");
LOG_TRACE(log, "Got connection str '" << connection_string << "'");
@ -86,7 +96,18 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques
SCOPE_EXIT(SQLFreeStmt(hstmt, SQL_DROP));
/// TODO Why not do SQLColumns instead?
std::string query = "SELECT * FROM " + table_name + " WHERE 1 = 0";
std::string name = schema_name.empty() ? table_name : schema_name + "." + table_name;
std::stringstream ss;
std::string input = "SELECT * FROM " + name + " WHERE 1 = 0";
ParserQueryWithOutput parser;
ASTPtr select = parseQuery(parser, input.data(), input.data() + input.size(), "", 0);
IAST::FormatSettings settings(ss, true);
settings.always_quote_identifiers = true;
settings.identifier_quoting_style = IdentifierQuotingStyle::DoubleQuotes;
select->format(settings);
std::string query = ss.str();
if (Poco::Data::ODBC::Utility::isError(Poco::Data::ODBC::SQLPrepare(hstmt, reinterpret_cast<SQLCHAR *>(&query[0]), query.size())))
throw Poco::Data::ODBC::DescriptorException(session.dbc());

View File

@ -49,7 +49,8 @@ void ODBCHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
Poco::Net::HTMLForm params(request, request.stream());
LOG_TRACE(log, "Request URI: " + request.getURI());
auto process_error = [&response, this](const std::string & message) {
auto process_error = [&response, this](const std::string & message)
{
response.setStatusAndReason(Poco::Net::HTTPResponse::HTTP_INTERNAL_SERVER_ERROR);
if (!response.sent())
response.send() << message << std::endl;

View File

@ -366,12 +366,16 @@ int Server::main(const std::vector<std::string> & /*args*/)
dns_cache_updater = std::make_unique<DNSCacheUpdater>(*global_context);
}
if (!TaskStatsInfoGetter::checkProcessHasRequiredPermissions())
#if defined(__linux__)
if (!TaskStatsInfoGetter::checkPermissions())
{
LOG_INFO(log, "It looks like the process has not CAP_NET_ADMIN capability, some performance statistics will be disabled."
" It could happen due to incorrect clickhouse package installation."
" You could resolve the problem manually calling 'sudo setcap cap_net_admin=+ep /usr/bin/clickhouse'");
LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, some performance statistics will be disabled."
" It could happen due to incorrect ClickHouse package installation."
" You could resolve the problem manually with 'sudo setcap cap_net_admin=+ep /usr/bin/clickhouse'");
}
#else
LOG_INFO(log, "TaskStats is not implemented for this OS. IO accounting will be disabled.");
#endif
{
Poco::Timespan keep_alive_timeout(config().getUInt("keep_alive_timeout", 10), 0);

View File

@ -88,7 +88,7 @@ void TCPHandler::runImpl()
try
{
/// We try to send error information to the client.
sendException(e);
sendException(e, connection_context.getSettingsRef().calculate_text_stack_trace);
}
catch (...) {}
@ -103,7 +103,7 @@ void TCPHandler::runImpl()
Exception e("Database " + default_database + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
LOG_ERROR(log, "Code: " << e.code() << ", e.displayText() = " << e.displayText()
<< ", Stack trace:\n\n" << e.getStackTrace().toString());
sendException(e);
sendException(e, connection_context.getSettingsRef().calculate_text_stack_trace);
return;
}
@ -133,6 +133,8 @@ void TCPHandler::runImpl()
std::unique_ptr<Exception> exception;
bool network_error = false;
bool send_exception_with_stack_trace = connection_context.getSettingsRef().calculate_text_stack_trace;
try
{
/// Restore context of request.
@ -149,6 +151,8 @@ void TCPHandler::runImpl()
CurrentThread::initializeQuery();
send_exception_with_stack_trace = query_context.getSettingsRef().calculate_text_stack_trace;
/// Should we send internal logs to client?
if (client_revision >= DBMS_MIN_REVISION_WITH_SERVER_LOGS
&& query_context.getSettingsRef().send_logs_level.value != "none")
@ -158,7 +162,8 @@ void TCPHandler::runImpl()
CurrentThread::attachInternalTextLogsQueue(state.logs_queue);
}
query_context.setExternalTablesInitializer([&global_settings, this] (Context & context) {
query_context.setExternalTablesInitializer([&global_settings, this] (Context & context)
{
if (&context != &query_context)
throw Exception("Unexpected context in external tables initializer", ErrorCodes::LOGICAL_ERROR);
@ -245,7 +250,7 @@ void TCPHandler::runImpl()
tryLogCurrentException(log, "Can't send logs to client");
}
sendException(*exception);
sendException(*exception, send_exception_with_stack_trace);
}
}
catch (...)
@ -829,10 +834,10 @@ void TCPHandler::sendLogData(const Block & block)
}
void TCPHandler::sendException(const Exception & e)
void TCPHandler::sendException(const Exception & e, bool with_stack_trace)
{
writeVarUInt(Protocol::Server::Exception, *out);
writeException(e, *out);
writeException(e, *out, with_stack_trace);
out->next();
}

View File

@ -146,7 +146,7 @@ private:
void sendHello();
void sendData(const Block & block); /// Write a block to the network.
void sendLogData(const Block & block);
void sendException(const Exception & e);
void sendException(const Exception & e, bool with_stack_trace);
void sendProgress();
void sendLogs();
void sendEndOfStream();

View File

@ -14,6 +14,9 @@ AggregateFunctionPtr createAggregateFunctionRetention(const std::string & name,
{
assertNoParameters(name, params);
if (arguments.size() < 2)
throw Exception("Not enough event arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (arguments.size() > AggregateFunctionRetentionData::max_events)
throw Exception("Too many event arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);

View File

@ -126,19 +126,23 @@ public:
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
auto & data_to = static_cast<ColumnArray &>(to).getData();
auto & data_to = static_cast<ColumnUInt8 &>(static_cast<ColumnArray &>(to).getData()).getData();
auto & offsets_to = static_cast<ColumnArray &>(to).getOffsets();
ColumnArray::Offset current_offset = data_to.size();
data_to.resize(current_offset + events_size);
const bool first_flag = this->data(place).events.test(0);
data_to.insert(first_flag ? Field(static_cast<UInt64>(1)) : Field(static_cast<UInt64>(0)));
for (const auto i : ext::range(1, events_size))
data_to[current_offset] = first_flag;
++current_offset;
for (size_t i = 1; i < events_size; ++i)
{
if (first_flag && this->data(place).events.test(i))
data_to.insert(Field(static_cast<UInt64>(1)));
else
data_to.insert(Field(static_cast<UInt64>(0)));
data_to[current_offset] = (first_flag && this->data(place).events.test(i));
++current_offset;
}
offsets_to.push_back(offsets_to.size() == 0 ? events_size : offsets_to.back() + events_size);
offsets_to.push_back(current_offset);
}
const char * getHeaderFilePath() const override

View File

@ -113,7 +113,9 @@ public:
std::vector<MutableColumnPtr> scatter(ColumnIndex num_columns, const Selector & selector) const override;
void gather(ColumnGathererStream & gatherer_stream) override;
void getExtremes(Field & min, Field & max) const override {
void getExtremes(Field & min, Field & max) const override
{
return getDictionary().index(getIndexes(), 0)->getExtremes(min, max); /// TODO: optimize
}

View File

@ -540,7 +540,7 @@ ConfigProcessor::LoadedConfig ConfigProcessor::loadConfigWithZooKeeperIncludes(
if (!fallback_to_preprocessed)
throw;
const auto * zk_exception = dynamic_cast<const zkutil::KeeperException *>(ex.nested());
const auto * zk_exception = dynamic_cast<const Coordination::Exception *>(ex.nested());
if (!zk_exception)
throw;

View File

@ -3,6 +3,7 @@
#include <sstream>
#include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <Poco/File.h>
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Path.h>
#include <Poco/Util/AbstractConfiguration.h>
@ -13,10 +14,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING;
}
ODBCBridgeHelper::ODBCBridgeHelper(
const Configuration & config_, const Poco::Timespan & http_timeout_, const std::string & connection_string_)
: config(config_), http_timeout(http_timeout_), connection_string(connection_string_)
@ -29,16 +32,17 @@ ODBCBridgeHelper::ODBCBridgeHelper(
ping_url.setScheme("http");
ping_url.setPath(PING_HANDLER);
}
void ODBCBridgeHelper::startODBCBridge() const
{
Poco::Path path{config.getString("application.dir", "")};
path.setFileName("clickhouse-odbc-bridge");
path.setFileName("clickhouse");
if (!path.isFile())
throw Exception("clickhouse-odbc-bridge is not found", ErrorCodes::EXTERNAL_EXECUTABLE_NOT_FOUND);
if (!Poco::File(path).exists())
throw Exception("clickhouse binary is not found", ErrorCodes::EXTERNAL_EXECUTABLE_NOT_FOUND);
std::stringstream command;
command << path.toString() << ' ';
command << path.toString() << " odbc-bridge ";
command << "--http-port " << config.getUInt("odbc_bridge.port", DEFAULT_PORT) << ' ';
command << "--listen-host " << config.getString("odbc_bridge.listen_host", DEFAULT_HOST) << ' ';
command << "--http-timeout " << http_timeout.totalMicroseconds() << ' ';

View File

@ -7,8 +7,11 @@
#include <sstream>
#include <Common/StackTrace.h>
#include <Common/SimpleCache.h>
#include <common/demangle.h>
/// Arcadia compatibility DEVTOOLS-3976
#if defined(BACKTRACE_INCLUDE)
#include BACKTRACE_INCLUDE
@ -19,12 +22,16 @@
StackTrace::StackTrace()
{
frames_size = BACKTRACE_FUNC(frames, STACK_TRACE_MAX_DEPTH);
frames_size = BACKTRACE_FUNC(frames.data(), STACK_TRACE_MAX_DEPTH);
for (size_t i = frames_size; i < STACK_TRACE_MAX_DEPTH; ++i)
frames[i] = nullptr;
}
std::string StackTrace::toString() const
std::string StackTrace::toStringImpl(const Frames & frames, size_t frames_size)
{
char ** symbols = backtrace_symbols(frames, frames_size);
char ** symbols = backtrace_symbols(frames.data(), frames_size);
std::stringstream res;
if (!symbols)
@ -72,3 +79,13 @@ std::string StackTrace::toString() const
free(symbols);
return res.str();
}
std::string StackTrace::toString() const
{
/// Calculation of stack trace text is extremely slow.
/// We use simple cache because otherwise the server could be overloaded by trash queries.
static SimpleCache<decltype(StackTrace::toStringImpl), &StackTrace::toStringImpl> func_cached;
return func_cached(frames, frames_size);
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <string>
#include <array>
#define STACK_TRACE_MAX_DEPTH 32
@ -17,6 +18,9 @@ public:
private:
using Frame = void*;
Frame frames[STACK_TRACE_MAX_DEPTH];
using Frames = std::array<Frame, STACK_TRACE_MAX_DEPTH>;
Frames frames;
size_t frames_size;
static std::string toStringImpl(const Frames & frames, size_t frames_size);
};

View File

@ -5,9 +5,10 @@ StopwatchRUsage::Timestamp StopwatchRUsage::Timestamp::current()
{
StopwatchRUsage::Timestamp res;
::rusage rusage;
::rusage rusage {};
#if !defined(__APPLE__)
::getrusage(RUSAGE_THREAD, &rusage);
#endif
res.user_ns = rusage.ru_utime.tv_sec * 1000000000UL + rusage.ru_utime.tv_usec * 1000UL;
res.sys_ns = rusage.ru_stime.tv_sec * 1000000000UL + rusage.ru_stime.tv_usec * 1000UL;
return res;

View File

@ -2,26 +2,27 @@
#include <Common/Exception.h>
#include <Core/Types.h>
#include <unistd.h>
#if defined(__linux__)
#include <common/unaligned.h>
#include <errno.h>
#include <linux/genetlink.h>
#include <linux/netlink.h>
#include <linux/taskstats.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <sys/socket.h>
#include <sys/types.h>
#include <unistd.h>
#include <syscall.h>
#include <linux/genetlink.h>
#include <linux/netlink.h>
#include <linux/taskstats.h>
#include <linux/capability.h>
/// Basic idea is motivated by "iotop" tool.
/// More info: https://www.kernel.org/doc/Documentation/accounting/taskstats.txt
#define GENLMSG_DATA(glh) ((void *)((char*)NLMSG_DATA(glh) + GENL_HDRLEN))
#define GENLMSG_PAYLOAD(glh) (NLMSG_PAYLOAD(glh, 0) - GENL_HDRLEN)
#define NLA_DATA(na) ((void *)((char*)(na) + NLA_HDRLEN))
#define NLA_PAYLOAD(len) (len - NLA_HDRLEN)
namespace DB
{
@ -29,193 +30,253 @@ namespace DB
namespace ErrorCodes
{
extern const int NETLINK_ERROR;
extern const int LOGICAL_ERROR;
}
namespace
{
static size_t constexpr MAX_MSG_SIZE = 1024;
/** The message contains:
* - Netlink protocol header;
* - Generic Netlink (is a sub-protocol of Netlink that we use) protocol header;
* - Payload
* -- that itself is a list of "Attributes" (sub-messages), each of them contains length (including header), type, and its own payload.
* -- and attribute payload may be represented by the list of embedded attributes.
*/
struct NetlinkMessage
{
::nlmsghdr n;
::genlmsghdr g;
char buf[MAX_MSG_SIZE];
static size_t constexpr MAX_MSG_SIZE = 1024;
alignas(NLMSG_ALIGNTO) ::nlmsghdr header;
struct Attribute
{
::nlattr header;
alignas(NLMSG_ALIGNTO) char payload[0];
const Attribute * next() const
{
return reinterpret_cast<const Attribute *>(reinterpret_cast<const char *>(this) + NLA_ALIGN(header.nla_len));
}
};
int sendCommand(
int sock_fd,
UInt16 nlmsg_type,
UInt32 nlmsg_pid,
UInt8 genl_cmd,
UInt16 nla_type,
void * nla_data,
int nla_len) noexcept
union alignas(NLMSG_ALIGNTO)
{
NetlinkMessage msg{};
struct
{
::genlmsghdr generic_header;
msg.n.nlmsg_len = NLMSG_LENGTH(GENL_HDRLEN);
msg.n.nlmsg_type = nlmsg_type;
msg.n.nlmsg_flags = NLM_F_REQUEST;
msg.n.nlmsg_seq = 0;
msg.n.nlmsg_pid = nlmsg_pid;
msg.g.cmd = genl_cmd;
msg.g.version = 1;
union alignas(NLMSG_ALIGNTO)
{
char buf[MAX_MSG_SIZE];
Attribute attribute; /// First attribute. There may be more.
} payload;
};
::nlattr * attr = static_cast<::nlattr *>(GENLMSG_DATA(&msg));
attr->nla_type = nla_type;
attr->nla_len = nla_len + 1 + NLA_HDRLEN;
::nlmsgerr error;
};
memcpy(NLA_DATA(attr), nla_data, nla_len);
msg.n.nlmsg_len += NLMSG_ALIGN(attr->nla_len);
size_t payload_size() const
{
return header.nlmsg_len - sizeof(header) - sizeof(generic_header);
}
char * buf = reinterpret_cast<char *>(&msg);
ssize_t buflen = msg.n.nlmsg_len;
const Attribute * end() const
{
return reinterpret_cast<const Attribute *>(reinterpret_cast<const char *>(this) + header.nlmsg_len);
}
void send(int fd) const
{
const char * request_buf = reinterpret_cast<const char *>(this);
ssize_t request_size = header.nlmsg_len;
::sockaddr_nl nladdr{};
nladdr.nl_family = AF_NETLINK;
while (true)
{
ssize_t r = ::sendto(sock_fd, buf, buflen, 0, reinterpret_cast<const ::sockaddr *>(&nladdr), sizeof(nladdr));
ssize_t bytes_sent = ::sendto(fd, request_buf, request_size, 0, reinterpret_cast<const ::sockaddr *>(&nladdr), sizeof(nladdr));
if (r >= buflen)
if (bytes_sent <= 0)
{
if (errno == EAGAIN)
continue;
else
throwFromErrno("Can't send a Netlink command", ErrorCodes::NETLINK_ERROR);
}
if (bytes_sent > request_size)
throw Exception("Wrong result of sendto system call: bytes_sent is greater than request size", ErrorCodes::NETLINK_ERROR);
if (bytes_sent == request_size)
break;
if (r > 0)
request_buf += bytes_sent;
request_size -= bytes_sent;
}
}
void receive(int fd)
{
buf += r;
buflen -= r;
}
else if (errno != EAGAIN)
return -1;
}
ssize_t bytes_received = ::recv(fd, this, sizeof(*this), 0);
return 0;
if (header.nlmsg_type == NLMSG_ERROR || !NLMSG_OK((&header), bytes_received))
throw Exception("Can't receive Netlink response, error: " + std::to_string(error.error), ErrorCodes::NETLINK_ERROR);
}
};
UInt16 getFamilyId(int nl_sock_fd) noexcept
NetlinkMessage query(
int fd,
UInt16 type,
UInt32 pid,
UInt8 command,
UInt16 attribute_type,
const void * attribute_data,
int attribute_size)
{
struct
NetlinkMessage request;
request.header.nlmsg_len = NLMSG_LENGTH(GENL_HDRLEN); /// Length of both headers.
request.header.nlmsg_type = type;
request.header.nlmsg_flags = NLM_F_REQUEST; /// A request.
request.header.nlmsg_seq = 0;
request.header.nlmsg_pid = pid;
request.generic_header.cmd = command;
request.generic_header.version = 1;
request.payload.attribute.header.nla_type = attribute_type;
request.payload.attribute.header.nla_len = attribute_size + 1 + NLA_HDRLEN;
memcpy(&request.payload.attribute.payload, attribute_data, attribute_size);
request.header.nlmsg_len += NLMSG_ALIGN(request.payload.attribute.header.nla_len);
request.send(fd);
NetlinkMessage response;
response.receive(fd);
return response;
}
UInt16 getFamilyIdImpl(int fd)
{
::nlmsghdr header;
::genlmsghdr ge_header;
char buf[256];
} answer;
NetlinkMessage answer = query(fd, GENL_ID_CTRL, getpid(), CTRL_CMD_GETFAMILY, CTRL_ATTR_FAMILY_NAME, TASKSTATS_GENL_NAME, strlen(TASKSTATS_GENL_NAME) + 1);
static char name[] = TASKSTATS_GENL_NAME;
/// NOTE Why the relevant info is located in the second attribute?
const NetlinkMessage::Attribute * attr = answer.payload.attribute.next();
if (sendCommand(
nl_sock_fd, GENL_ID_CTRL, getpid(), CTRL_CMD_GETFAMILY,
CTRL_ATTR_FAMILY_NAME, (void *) name,
strlen(TASKSTATS_GENL_NAME) + 1))
return 0;
if (attr->header.nla_type != CTRL_ATTR_FAMILY_ID)
throw Exception("Received wrong attribute as an answer to GET_FAMILY Netlink command", ErrorCodes::NETLINK_ERROR);
UInt16 id = 0;
ssize_t rep_len = ::recv(nl_sock_fd, &answer, sizeof(answer), 0);
if (answer.header.nlmsg_type == NLMSG_ERROR || (rep_len < 0) || !NLMSG_OK((&answer.header), rep_len))
return 0;
return unalignedLoad<UInt16>(attr->payload);
}
const ::nlattr * attr;
attr = static_cast<const ::nlattr *>(GENLMSG_DATA(&answer));
attr = reinterpret_cast<const ::nlattr *>(reinterpret_cast<const char *>(attr) + NLA_ALIGN(attr->nla_len));
if (attr->nla_type == CTRL_ATTR_FAMILY_ID)
id = *static_cast<const UInt16 *>(NLA_DATA(attr));
return id;
bool checkPermissionsImpl()
{
/// See man getcap.
__user_cap_header_struct request{};
request.version = _LINUX_CAPABILITY_VERSION_1; /// It's enough to check just single CAP_NET_ADMIN capability we are interested.
request.pid = getpid();
__user_cap_data_struct response{};
/// Avoid dependency on 'libcap'.
if (0 != syscall(SYS_capget, &request, &response))
throwFromErrno("Cannot do 'capget' syscall", ErrorCodes::NETLINK_ERROR);
return (1 << CAP_NET_ADMIN) & response.effective;
}
UInt16 getFamilyId(int fd)
{
/// It is thread and exception safe since C++11 and even before.
static UInt16 res = getFamilyIdImpl(fd);
return res;
}
}
TaskStatsInfoGetter::TaskStatsInfoGetter() = default;
void TaskStatsInfoGetter::init()
bool TaskStatsInfoGetter::checkPermissions()
{
if (netlink_socket_fd >= 0)
return;
static bool res = checkPermissionsImpl();
return res;
}
TaskStatsInfoGetter::TaskStatsInfoGetter()
{
if (!checkPermissions())
throw Exception("Logical error: TaskStatsInfoGetter is not usable without CAP_NET_ADMIN. Check permissions before creating the object.",
ErrorCodes::LOGICAL_ERROR);
netlink_socket_fd = ::socket(PF_NETLINK, SOCK_RAW, NETLINK_GENERIC);
if (netlink_socket_fd < 0)
throwFromErrno("Can't create PF_NETLINK socket");
throwFromErrno("Can't create PF_NETLINK socket", ErrorCodes::NETLINK_ERROR);
/// On some containerized environments, operation on Netlink socket could hang forever.
/// We set reasonably small timeout to overcome this issue.
struct timeval tv;
tv.tv_sec = 0;
tv.tv_usec = 50000;
if (0 != ::setsockopt(netlink_socket_fd, SOL_SOCKET, SO_RCVTIMEO, reinterpret_cast<const char *>(&tv), sizeof(tv)))
throwFromErrno("Can't set timeout on PF_NETLINK socket", ErrorCodes::NETLINK_ERROR);
::sockaddr_nl addr{};
addr.nl_family = AF_NETLINK;
if (::bind(netlink_socket_fd, reinterpret_cast<const ::sockaddr *>(&addr), sizeof(addr)) < 0)
throwFromErrno("Can't bind PF_NETLINK socket");
throwFromErrno("Can't bind PF_NETLINK socket", ErrorCodes::NETLINK_ERROR);
netlink_family_id = getFamilyId(netlink_socket_fd);
taskstats_family_id = getFamilyId(netlink_socket_fd);
}
bool TaskStatsInfoGetter::getStatImpl(int tid, ::taskstats & out_stats, bool throw_on_error)
void TaskStatsInfoGetter::getStat(::taskstats & out_stats, pid_t tid)
{
init();
NetlinkMessage answer = query(netlink_socket_fd, taskstats_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(tid));
if (sendCommand(netlink_socket_fd, netlink_family_id, tid, TASKSTATS_CMD_GET, TASKSTATS_CMD_ATTR_PID, &tid, sizeof(pid_t)))
throwFromErrno("Can't send a Netlink command");
NetlinkMessage msg;
ssize_t rv = ::recv(netlink_socket_fd, &msg, sizeof(msg), 0);
if (msg.n.nlmsg_type == NLMSG_ERROR || !NLMSG_OK((&msg.n), rv))
for (const NetlinkMessage::Attribute * attr = &answer.payload.attribute;
attr < answer.end();
attr = attr->next())
{
const ::nlmsgerr * err = static_cast<const ::nlmsgerr *>(NLMSG_DATA(&msg));
if (throw_on_error)
throw Exception("Can't get Netlink response, error: " + std::to_string(err->error), ErrorCodes::NETLINK_ERROR);
else
return false;
}
rv = GENLMSG_PAYLOAD(&msg.n);
const ::nlattr * attr = static_cast<const ::nlattr *>(GENLMSG_DATA(&msg));
ssize_t len = 0;
while (len < rv)
if (attr->header.nla_type == TASKSTATS_TYPE_AGGR_TGID || attr->header.nla_type == TASKSTATS_TYPE_AGGR_PID)
{
len += NLA_ALIGN(attr->nla_len);
if (attr->nla_type == TASKSTATS_TYPE_AGGR_TGID || attr->nla_type == TASKSTATS_TYPE_AGGR_PID)
for (const NetlinkMessage::Attribute * nested_attr = reinterpret_cast<const NetlinkMessage::Attribute *>(attr->payload);
nested_attr < attr->next();
nested_attr = nested_attr->next())
{
int aggr_len = NLA_PAYLOAD(attr->nla_len);
int len2 = 0;
attr = static_cast<const ::nlattr *>(NLA_DATA(attr));
while (len2 < aggr_len)
if (nested_attr->header.nla_type == TASKSTATS_TYPE_STATS)
{
if (attr->nla_type == TASKSTATS_TYPE_STATS)
out_stats = unalignedLoad<::taskstats>(nested_attr->payload);
return;
}
}
}
}
throw Exception("There is no TASKSTATS_TYPE_STATS attribute in the Netlink response", ErrorCodes::NETLINK_ERROR);
}
pid_t TaskStatsInfoGetter::getCurrentTID()
{
const ::taskstats * ts = static_cast<const ::taskstats *>(NLA_DATA(attr));
out_stats = *ts;
/// This call is always successful. - man gettid
return static_cast<pid_t>(syscall(SYS_gettid));
}
len2 += NLA_ALIGN(attr->nla_len);
attr = reinterpret_cast<const ::nlattr *>(reinterpret_cast<const char *>(attr) + len2);
}
}
attr = reinterpret_cast<const ::nlattr *>(reinterpret_cast<const char *>(GENLMSG_DATA(&msg)) + len);
}
return true;
}
void TaskStatsInfoGetter::getStat(::taskstats & stat, int tid)
{
tid = tid < 0 ? getDefaultTID() : tid;
getStatImpl(tid, stat, true);
}
bool TaskStatsInfoGetter::tryGetStat(::taskstats & stat, int tid)
{
tid = tid < 0 ? getDefaultTID() : tid;
return getStatImpl(tid, stat, false);
}
TaskStatsInfoGetter::~TaskStatsInfoGetter()
{
@ -223,32 +284,43 @@ TaskStatsInfoGetter::~TaskStatsInfoGetter()
close(netlink_socket_fd);
}
int TaskStatsInfoGetter::getCurrentTID()
}
#else
namespace DB
{
/// This call is always successful. - man gettid
return static_cast<int>(syscall(SYS_gettid));
}
int TaskStatsInfoGetter::getDefaultTID()
namespace ErrorCodes
{
if (default_tid < 0)
default_tid = getCurrentTID();
return default_tid;
extern const int NOT_IMPLEMENTED;
}
static bool tryGetTaskStats()
bool TaskStatsInfoGetter::checkPermissions()
{
TaskStatsInfoGetter getter;
::taskstats stat;
return getter.tryGetStat(stat);
return false;
}
bool TaskStatsInfoGetter::checkProcessHasRequiredPermissions()
TaskStatsInfoGetter::TaskStatsInfoGetter()
{
throw Exception("TaskStats are not implemented for this OS.", ErrorCodes::NOT_IMPLEMENTED);
}
void TaskStatsInfoGetter::getStat(::taskstats &, pid_t)
{
}
pid_t TaskStatsInfoGetter::getCurrentTID()
{
return 0;
}
TaskStatsInfoGetter::~TaskStatsInfoGetter()
{
/// It is thread- and exception- safe since C++11
static bool res = tryGetTaskStats();
return res;
}
}
#endif

View File

@ -1,43 +1,34 @@
#pragma once
#include <sys/types.h>
#include <Core/Types.h>
#include <boost/noncopyable.hpp>
struct taskstats;
namespace DB
{
class Exception;
/// Get taskstat info from OS kernel via Netlink protocol.
class TaskStatsInfoGetter
class TaskStatsInfoGetter : private boost::noncopyable
{
public:
TaskStatsInfoGetter();
TaskStatsInfoGetter(const TaskStatsInfoGetter &) = delete;
void getStat(::taskstats & stat, int tid = -1);
bool tryGetStat(::taskstats & stat, int tid = -1);
~TaskStatsInfoGetter();
void getStat(::taskstats & stat, pid_t tid);
/// Make a syscall and returns Linux thread id
static int getCurrentTID();
static pid_t getCurrentTID();
/// Whether the current process has permissions (sudo or cap_net_admin capabilties) to get taskstats info
static bool checkProcessHasRequiredPermissions();
static bool checkPermissions();
#if defined(__linux__)
private:
/// Caches current thread tid to avoid extra sys calls
int getDefaultTID();
int default_tid = -1;
bool getStatImpl(int tid, ::taskstats & out_stats, bool throw_on_error = false);
void init();
int netlink_socket_fd = -1;
UInt16 netlink_family_id = 0;
UInt16 taskstats_family_id = 0;
#endif
};
}

View File

@ -1,11 +1,16 @@
#pragma once
#include <Common/TaskStatsInfoGetter.h>
#include <Common/ProfileEvents.h>
#include <sys/time.h>
#include <sys/resource.h>
#include <pthread.h>
#if defined(__linux__)
#include <linux/taskstats.h>
#else
struct taskstats {};
#endif
namespace ProfileEvents
@ -18,6 +23,7 @@ namespace ProfileEvents
extern const Event VoluntaryContextSwitches;
extern const Event InvoluntaryContextSwitches;
#if defined(__linux__)
extern const Event OSIOWaitMicroseconds;
extern const Event OSCPUWaitMicroseconds;
extern const Event OSCPUVirtualTimeMicroseconds;
@ -25,6 +31,7 @@ namespace ProfileEvents
extern const Event OSWriteChars;
extern const Event OSReadBytes;
extern const Event OSWriteBytes;
#endif
}
@ -82,8 +89,10 @@ struct RUsageCounters
static RUsageCounters current(UInt64 real_time_ = getCurrentTimeNanoseconds())
{
::rusage rusage;
::rusage rusage {};
#if !defined(__APPLE__)
::getrusage(RUSAGE_THREAD, &rusage);
#endif
return RUsageCounters(rusage, real_time_);
}
@ -106,6 +115,8 @@ struct RUsageCounters
};
#if defined(__linux__)
struct TasksStatsCounters
{
::taskstats stat;
@ -141,4 +152,17 @@ struct TasksStatsCounters
}
};
#else
struct TasksStatsCounters
{
::taskstats stat;
static TasksStatsCounters current();
static void incrementProfileEvents(const TasksStatsCounters &, const TasksStatsCounters &, ProfileEvents::Counters &) {}
static void updateProfileEvents(TasksStatsCounters &, ProfileEvents::Counters &) {}
};
#endif
}

View File

@ -1,10 +1,13 @@
#include "ThreadStatus.h"
#include <common/logger_useful.h>
#include <sstream>
#include <common/Types.h>
#include <Common/CurrentThread.h>
#include <Common/Exception.h>
#include <Common/ThreadProfileEvents.h>
#include <Common/TaskStatsInfoGetter.h>
#include <Common/ThreadStatus.h>
#include <Poco/Thread.h>
#include <Poco/Logger.h>
#include <Poco/Ext/ThreadNumber.h>
@ -39,7 +42,6 @@ ThreadStatus::ThreadStatus()
last_rusage = std::make_unique<RUsageCounters>();
last_taskstats = std::make_unique<TasksStatsCounters>();
taskstats_getter = std::make_unique<TaskStatsInfoGetter>();
memory_tracker.setDescription("(for thread)");
log = &Poco::Logger::get("ThreadStatus");
@ -70,17 +72,20 @@ void ThreadStatus::initPerformanceCounters()
++queries_started;
*last_rusage = RUsageCounters::current(query_start_time_nanoseconds);
has_permissions_for_taskstats = TaskStatsInfoGetter::checkProcessHasRequiredPermissions();
if (has_permissions_for_taskstats)
if (TaskStatsInfoGetter::checkPermissions())
{
taskstats_getter = std::make_unique<TaskStatsInfoGetter>();
*last_taskstats = TasksStatsCounters::current();
}
}
void ThreadStatus::updatePerformanceCounters()
{
try
{
RUsageCounters::updateProfileEvents(*last_rusage, performance_counters);
if (has_permissions_for_taskstats)
if (taskstats_getter)
TasksStatsCounters::updateProfileEvents(*last_taskstats, performance_counters);
}
catch (...)

View File

@ -33,7 +33,6 @@ using InternalTextLogsQueueWeakPtr = std::weak_ptr<InternalTextLogsQueue>;
class ThreadGroupStatus
{
public:
mutable std::shared_mutex mutex;
ProfileEvents::Counters performance_counters{VariableContext::Process};
@ -126,7 +125,6 @@ public:
~ThreadStatus();
protected:
ThreadStatus();
void initPerformanceCounters();
@ -160,11 +158,11 @@ protected:
/// Use ptr not to add extra dependencies in the header
std::unique_ptr<RUsageCounters> last_rusage;
std::unique_ptr<TasksStatsCounters> last_taskstats;
/// Set only if we have enough capabilities.
std::unique_ptr<TaskStatsInfoGetter> taskstats_getter;
bool has_permissions_for_taskstats = false;
public:
/// Implicitly finalizes current thread in the destructor
class CurrentThreadScope
{

View File

@ -0,0 +1,169 @@
#include <string.h>
#include <Common/ProfileEvents.h>
#include <Common/ZooKeeper/IKeeper.h>
namespace DB
{
namespace ErrorCodes
{
extern const int KEEPER_EXCEPTION;
}
}
namespace ProfileEvents
{
extern const Event ZooKeeperUserExceptions;
extern const Event ZooKeeperHardwareExceptions;
extern const Event ZooKeeperOtherExceptions;
}
namespace Coordination
{
Exception::Exception(const std::string & msg, const int32_t code, int)
: DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code)
{
if (Coordination::isUserError(code))
ProfileEvents::increment(ProfileEvents::ZooKeeperUserExceptions);
else if (Coordination::isHardwareError(code))
ProfileEvents::increment(ProfileEvents::ZooKeeperHardwareExceptions);
else
ProfileEvents::increment(ProfileEvents::ZooKeeperOtherExceptions);
}
Exception::Exception(const std::string & msg, const int32_t code)
: Exception(msg + " (" + errorMessage(code) + ")", code, 0)
{
}
Exception::Exception(const int32_t code)
: Exception(errorMessage(code), code, 0)
{
}
Exception::Exception(const int32_t code, const std::string & path)
: Exception(std::string{errorMessage(code)} + ", path: " + path, code, 0)
{
}
Exception::Exception(const Exception & exc)
: DB::Exception(exc), code(exc.code)
{
}
using namespace DB;
void addRootPath(String & path, const String & root_path)
{
if (path.empty())
throw Exception("Path cannot be empty", ZBADARGUMENTS);
if (path[0] != '/')
throw Exception("Path must begin with /", ZBADARGUMENTS);
if (root_path.empty())
return;
if (path.size() == 1) /// "/"
path = root_path;
else
path = root_path + path;
}
void removeRootPath(String & path, const String & root_path)
{
if (root_path.empty())
return;
if (path.size() <= root_path.size())
throw Exception("Received path is not longer than root_path", ZDATAINCONSISTENCY);
path = path.substr(root_path.size());
}
const char * errorMessage(int32_t code)
{
switch (code)
{
case ZOK: return "Ok";
case ZSYSTEMERROR: return "System error";
case ZRUNTIMEINCONSISTENCY: return "Run time inconsistency";
case ZDATAINCONSISTENCY: return "Data inconsistency";
case ZCONNECTIONLOSS: return "Connection loss";
case ZMARSHALLINGERROR: return "Marshalling error";
case ZUNIMPLEMENTED: return "Unimplemented";
case ZOPERATIONTIMEOUT: return "Operation timeout";
case ZBADARGUMENTS: return "Bad arguments";
case ZINVALIDSTATE: return "Invalid zhandle state";
case ZAPIERROR: return "API error";
case ZNONODE: return "No node";
case ZNOAUTH: return "Not authenticated";
case ZBADVERSION: return "Bad version";
case ZNOCHILDRENFOREPHEMERALS: return "No children for ephemerals";
case ZNODEEXISTS: return "Node exists";
case ZNOTEMPTY: return "Not empty";
case ZSESSIONEXPIRED: return "Session expired";
case ZINVALIDCALLBACK: return "Invalid callback";
case ZINVALIDACL: return "Invalid ACL";
case ZAUTHFAILED: return "Authentication failed";
case ZCLOSING: return "ZooKeeper is closing";
case ZNOTHING: return "(not error) no server responses to process";
case ZSESSIONMOVED: return "Session moved to another server, so operation is ignored";
}
if (code > 0)
return strerror(code);
return "unknown error";
}
bool isHardwareError(int32_t zk_return_code)
{
return zk_return_code == ZINVALIDSTATE
|| zk_return_code == ZSESSIONEXPIRED
|| zk_return_code == ZSESSIONMOVED
|| zk_return_code == ZCONNECTIONLOSS
|| zk_return_code == ZMARSHALLINGERROR
|| zk_return_code == ZOPERATIONTIMEOUT;
}
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;
}
void CreateRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void RemoveRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void ExistsRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void GetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void SetRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void ListRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void CheckRequest::addRootPath(const String & root_path) { Coordination::addRootPath(path, root_path); }
void MultiRequest::addRootPath(const String & root_path)
{
for (auto & request : requests)
request->addRootPath(root_path);
}
void CreateResponse::removeRootPath(const String & root_path) { Coordination::removeRootPath(path_created, root_path); }
void WatchResponse::removeRootPath(const String & root_path) { Coordination::removeRootPath(path, root_path); }
void MultiResponse::removeRootPath(const String & root_path)
{
for (auto & response : responses)
response->removeRootPath(root_path);
}
}

View File

@ -0,0 +1,389 @@
#pragma once
#include <Core/Types.h>
#include <Common/Exception.h>
#include <vector>
#include <memory>
#include <cstdint>
#include <functional>
/** Generic interface for ZooKeeper-like services.
* Possible examples are:
* - ZooKeeper client itself;
* - fake ZooKeeper client for testing;
* - ZooKeeper emulation layer on top of Etcd, FoundationDB, whatever.
*/
namespace Coordination
{
using namespace DB;
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;
};
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;
};
struct Request;
using RequestPtr = std::shared_ptr<Request>;
using Requests = std::vector<RequestPtr>;
struct Request
{
virtual ~Request() {}
virtual String getPath() const = 0;
virtual void addRootPath(const String & /* root_path */) {}
};
struct Response;
using ResponsePtr = std::shared_ptr<Response>;
using Responses = std::vector<ResponsePtr>;
using ResponseCallback = std::function<void(const Response &)>;
struct Response
{
int32_t error = 0;
virtual ~Response() {}
virtual void removeRootPath(const String & /* root_path */) {}
};
struct WatchResponse : virtual Response
{
int32_t type = 0;
int32_t state = 0;
String path;
void removeRootPath(const String & root_path) override;
};
using WatchCallback = std::function<void(const WatchResponse &)>;
struct CreateRequest : virtual Request
{
String path;
String data;
bool is_ephemeral = false;
bool is_sequential = false;
ACLs acls;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct CreateResponse : virtual Response
{
String path_created;
void removeRootPath(const String & root_path) override;
};
struct RemoveRequest : virtual Request
{
String path;
int32_t version = -1;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct RemoveResponse : virtual Response
{
};
struct ExistsRequest : virtual Request
{
String path;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct ExistsResponse : virtual Response
{
Stat stat;
};
struct GetRequest : virtual Request
{
String path;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct GetResponse : virtual Response
{
String data;
Stat stat;
};
struct SetRequest : virtual Request
{
String path;
String data;
int32_t version = -1;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct SetResponse : virtual Response
{
Stat stat;
};
struct ListRequest : virtual Request
{
String path;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct ListResponse : virtual Response
{
std::vector<String> names;
Stat stat;
};
struct CheckRequest : virtual Request
{
String path;
int32_t version = -1;
void addRootPath(const String & root_path) override;
String getPath() const override { return path; }
};
struct CheckResponse : virtual Response
{
};
struct MultiRequest : virtual Request
{
Requests requests;
void addRootPath(const String & root_path) override;
String getPath() const override { return {}; }
};
struct MultiResponse : virtual Response
{
Responses responses;
void removeRootPath(const String & root_path) override;
};
/// This response may be received only as an element of responses in MultiResponse.
struct ErrorResponse : virtual Response
{
};
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 &)>;
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
};
/// Network errors and similar. You should reinitialize ZooKeeper session in case of these errors
bool isHardwareError(int32_t code);
/// Valid errors sent from the server about database state (like "no node"). Logical and authentication errors (like "bad arguments") are not here.
bool isUserError(int32_t code);
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
};
class Exception : public DB::Exception
{
private:
/// Delegate constructor, used to minimize repetition; last parameter used for overload resolution.
Exception(const std::string & msg, const int32_t code, int);
public:
explicit Exception(const int32_t code);
Exception(const std::string & msg, const int32_t code);
Exception(const int32_t code, const std::string & path);
Exception(const Exception & exc);
const char * name() const throw() override { return "Coordination::Exception"; }
const char * className() const throw() override { return "Coordination::Exception"; }
Exception * clone() const override { return new Exception(*this); }
const int32_t code;
};
/** 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 exception with ZSESSIONEXPIRED code or method isExpired returns true,
* 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 IKeeper
{
public:
virtual ~IKeeper() {}
/// If expired, you can only destroy the object. All other methods will throw exception.
virtual bool isExpired() const = 0;
/// Useful to check owner of ephemeral node.
virtual int64_t getSessionID() const = 0;
/// If the method will throw an exception, callbacks won't be called.
///
/// After the method is executed successfully, you must wait for callbacks
/// (don't destroy callback data before it will be called).
///
/// All callbacks are executed sequentially (the execution of callbacks is serialized).
///
/// If an exception is thrown inside the callback, the session will expire,
/// and all other callbacks will be called with "Session expired" error.
virtual void create(
const String & path,
const String & data,
bool is_ephemeral,
bool is_sequential,
const ACLs & acls,
CreateCallback callback) = 0;
virtual void remove(
const String & path,
int32_t version,
RemoveCallback callback) = 0;
virtual void exists(
const String & path,
ExistsCallback callback,
WatchCallback watch) = 0;
virtual void get(
const String & path,
GetCallback callback,
WatchCallback watch) = 0;
virtual void set(
const String & path,
const String & data,
int32_t version,
SetCallback callback) = 0;
virtual void list(
const String & path,
ListCallback callback,
WatchCallback watch) = 0;
virtual void check(
const String & path,
int32_t version,
CheckCallback callback) = 0;
virtual void multi(
const Requests & requests,
MultiCallback callback) = 0;
};
}

View File

@ -20,7 +20,7 @@ public:
size_t result = 0;
std::string result_str;
zkutil::Stat stat;
Coordination::Stat stat;
bool success = false;
auto zookeeper = zookeeper_holder->getZooKeeper();
@ -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) == ZooKeeperImpl::ZooKeeper::ZOK;
success = zookeeper->trySet(path, std::to_string(result), stat.version) == Coordination::ZOK;
}
else
{
success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZooKeeperImpl::ZooKeeper::ZOK;
success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == Coordination::ZOK;
}
}
while (!success);

View File

@ -1,32 +1,32 @@
#pragma once
#include "Types.h"
#include <Common/ZooKeeper/Types.h>
namespace zkutil
{
using KeeperException = ZooKeeperImpl::Exception;
using KeeperException = Coordination::Exception;
class KeeperMultiException : public KeeperException
{
public:
Requests requests;
Responses responses;
Coordination::Requests requests;
Coordination::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(int32_t code, const Requests & requests, const Responses & responses);
static void check(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses);
KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses);
KeeperMultiException(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses);
private:
static size_t getFailedOpIndex(int32_t code, const Responses & responses);
static size_t getFailedOpIndex(int32_t code, const Coordination::Responses & responses);
};
}

View File

@ -85,8 +85,7 @@ private:
std::string node_path = node->getPath();
node_name = node_path.substr(node_path.find_last_of('/') + 1);
task->activate();
task->schedule();
task->activateAndSchedule();
}
void releaseNode()
@ -123,7 +122,7 @@ private:
{
DB::tryLogCurrentException(log);
if (e.code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED)
if (e.code == Coordination::ZSESSIONEXPIRED)
return;
}
catch (...)

View File

@ -18,17 +18,17 @@ bool Lock::tryLock()
std::string dummy;
int32_t code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy);
if (code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
if (code == Coordination::ZNODEEXISTS)
{
locked.reset(nullptr);
}
else if (code == ZooKeeperImpl::ZooKeeper::ZOK)
else if (code == Coordination::ZOK)
{
locked.reset(new ZooKeeperHandler(zookeeper));
}
else
{
throw zkutil::KeeperException(code);
throw Coordination::Exception(code);
}
}
return bool(locked);
@ -50,7 +50,7 @@ Lock::Status Lock::tryCheck() const
auto zookeeper = zookeeper_holder->getZooKeeper();
Status lock_status;
Stat stat;
Coordination::Stat stat;
std::string dummy;
bool result = zookeeper->tryGet(lock_path, dummy, &stat);
if (!result)

View File

@ -1,16 +1,16 @@
#pragma once
#include <common/Types.h>
#include <future>
#include <memory>
#include <vector>
#include <Common/ZooKeeper/ZooKeeperImpl.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Poco/Event.h>
namespace zkutil
{
using Stat = ZooKeeperImpl::ZooKeeper::Stat;
using Strings = std::vector<std::string>;
@ -24,44 +24,14 @@ namespace CreateMode
using EventPtr = std::shared_ptr<Poco::Event>;
/// 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).
using WatchCallback = ZooKeeperImpl::ZooKeeper::WatchCallback;
using Request = ZooKeeperImpl::ZooKeeper::Request;
using Response = ZooKeeperImpl::ZooKeeper::Response;
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;
/// Gets multiple asynchronous results
/// Each pair, the first is path, the second is response eg. CreateResponse, RemoveResponse
template <typename R>
using AsyncResponses = std::vector<std::pair<std::string, std::future<R>>>;
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);
Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode);
Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version);
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version);
Coordination::RequestPtr makeCheckRequest(const std::string & path, int version);
}

View File

@ -1,4 +1,5 @@
#include "ZooKeeper.h"
#include "ZooKeeperImpl.h"
#include "KeeperException.h"
#include <random>
@ -50,16 +51,16 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
chroot = chroot_;
if (hosts.empty())
throw KeeperException("No addresses passed to ZooKeeper constructor.", ZooKeeperImpl::ZooKeeper::ZBADARGUMENTS);
throw KeeperException("No addresses passed to ZooKeeper constructor.", Coordination::ZBADARGUMENTS);
std::vector<std::string> addresses_strings;
boost::split(addresses_strings, hosts, boost::is_any_of(","));
ZooKeeperImpl::ZooKeeper::Addresses addresses;
Coordination::ZooKeeper::Addresses addresses;
addresses.reserve(addresses_strings.size());
for (const auto & address_string : addresses_strings)
addresses.emplace_back(address_string);
impl = std::make_unique<ZooKeeperImpl::ZooKeeper>(
impl = std::make_unique<Coordination::ZooKeeper>(
addresses,
chroot,
identity_.empty() ? "" : "digest",
@ -71,7 +72,7 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot));
if (!chroot.empty() && !exists("/"))
throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.", ZooKeeperImpl::ZooKeeper::ZNONODE);
throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.", Coordination::ZNONODE);
}
ZooKeeper::ZooKeeper(const std::string & hosts, const std::string & identity,
@ -112,7 +113,7 @@ struct ZooKeeperArgs
chroot = config.getString(config_name + "." + key);
}
else
throw KeeperException(std::string("Unknown key ") + key + " in config file", ZooKeeperImpl::ZooKeeper::ZBADARGUMENTS);
throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::ZBADARGUMENTS);
}
/// Shuffle the hosts to distribute the load among ZooKeeper nodes.
@ -129,7 +130,7 @@ struct ZooKeeperArgs
if (!chroot.empty())
{
if (chroot.front() != '/')
throw KeeperException(std::string("Root path in config file should start with '/', but got ") + chroot, ZooKeeperImpl::ZooKeeper::ZBADARGUMENTS);
throw KeeperException(std::string("Root path in config file should start with '/', but got ") + chroot, Coordination::ZBADARGUMENTS);
if (chroot.back() == '/')
chroot.pop_back();
}
@ -148,22 +149,22 @@ ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std
}
static WatchCallback callbackForEvent(const EventPtr & watch)
static Coordination::WatchCallback callbackForEvent(const EventPtr & watch)
{
if (!watch)
return {};
return [watch](const ZooKeeperImpl::ZooKeeper::WatchResponse &) { watch->set(); };
return [watch](const Coordination::WatchResponse &) { watch->set(); };
}
int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res,
Stat * stat,
WatchCallback watch_callback)
Coordination::Stat * stat,
Coordination::WatchCallback watch_callback)
{
int32_t code = 0;
Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::ListResponse & response)
auto callback = [&](const Coordination::ListResponse & response)
{
code = response.error;
if (!code)
@ -181,7 +182,7 @@ int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res,
}
Strings ZooKeeper::getChildren(
const std::string & path, Stat * stat, const EventPtr & watch)
const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
{
Strings res;
check(tryGetChildren(path, res, stat, watch), path);
@ -189,7 +190,7 @@ Strings ZooKeeper::getChildren(
}
Strings ZooKeeper::getChildrenWatch(
const std::string & path, Stat * stat, WatchCallback watch_callback)
const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{
Strings res;
check(tryGetChildrenWatch(path, res, stat, watch_callback), path);
@ -197,22 +198,22 @@ Strings ZooKeeper::getChildrenWatch(
}
int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res,
Stat * stat, const EventPtr & watch)
Coordination::Stat * stat, const EventPtr & watch)
{
int32_t code = getChildrenImpl(path, res, stat, callbackForEvent(watch));
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE))
if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
throw KeeperException(code, path);
return code;
}
int32_t ZooKeeper::tryGetChildrenWatch(const std::string & path, Strings & res,
Stat * stat, WatchCallback watch_callback)
Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{
int32_t code = getChildrenImpl(path, res, stat, watch_callback);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE))
if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
throw KeeperException(code, path);
return code;
@ -223,7 +224,7 @@ int32_t ZooKeeper::createImpl(const std::string & path, const std::string & data
int32_t code = 0;
Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::CreateResponse & response)
auto callback = [&](const Coordination::CreateResponse & response)
{
code = response.error;
if (!code)
@ -247,10 +248,10 @@ int32_t ZooKeeper::tryCreate(const std::string & path, const std::string & data,
{
int32_t code = createImpl(path, data, mode, path_created);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK ||
code == ZooKeeperImpl::ZooKeeper::ZNONODE ||
code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS ||
code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS))
if (!(code == Coordination::ZOK ||
code == Coordination::ZNONODE ||
code == Coordination::ZNODEEXISTS ||
code == Coordination::ZNOCHILDRENFOREPHEMERALS))
throw KeeperException(code, path);
return code;
@ -267,7 +268,7 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string &
std::string path_created;
int32_t code = createImpl(path, data, CreateMode::Persistent, path_created);
if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS)
return;
else
throw KeeperException(code, path);
@ -291,7 +292,7 @@ int32_t ZooKeeper::removeImpl(const std::string & path, int32_t version)
int32_t code = 0;
Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response)
auto callback = [&](const Coordination::RemoveResponse & response)
{
if (response.error)
code = response.error;
@ -311,20 +312,20 @@ void ZooKeeper::remove(const std::string & path, int32_t version)
int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version)
{
int32_t code = removeImpl(path, version);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK ||
code == ZooKeeperImpl::ZooKeeper::ZNONODE ||
code == ZooKeeperImpl::ZooKeeper::ZBADVERSION ||
code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY))
if (!(code == Coordination::ZOK ||
code == Coordination::ZNONODE ||
code == Coordination::ZBADVERSION ||
code == Coordination::ZNOTEMPTY))
throw KeeperException(code, path);
return code;
}
int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat, WatchCallback watch_callback)
int32_t ZooKeeper::existsImpl(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{
int32_t code = 0;
Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response)
auto callback = [&](const Coordination::ExistsResponse & response)
{
code = response.error;
if (!code && stat)
@ -337,28 +338,28 @@ int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat, WatchCallba
return code;
}
bool ZooKeeper::exists(const std::string & path, Stat * stat, const EventPtr & watch)
bool ZooKeeper::exists(const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
{
return existsWatch(path, stat, callbackForEvent(watch));
}
bool ZooKeeper::existsWatch(const std::string & path, Stat * stat, WatchCallback watch_callback)
bool ZooKeeper::existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{
int32_t code = existsImpl(path, stat, watch_callback);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE))
if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
throw KeeperException(code, path);
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
if (code == Coordination::ZNONODE)
return false;
return true;
}
int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback)
int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{
int32_t code = 0;
Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::GetResponse & response)
auto callback = [&](const Coordination::GetResponse & response)
{
code = response.error;
if (!code)
@ -376,7 +377,7 @@ int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Stat * s
}
std::string ZooKeeper::get(const std::string & path, Stat * stat, const EventPtr & watch)
std::string ZooKeeper::get(const std::string & path, Coordination::Stat * stat, const EventPtr & watch)
{
int32_t code = 0;
std::string res;
@ -386,7 +387,7 @@ std::string ZooKeeper::get(const std::string & path, Stat * stat, const EventPtr
throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code);
}
std::string ZooKeeper::getWatch(const std::string & path, Stat * stat, WatchCallback watch_callback)
std::string ZooKeeper::getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback)
{
int32_t code = 0;
std::string res;
@ -396,31 +397,31 @@ std::string ZooKeeper::getWatch(const std::string & path, Stat * stat, WatchCall
throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code);
}
bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat, const EventPtr & watch, int * return_code)
bool ZooKeeper::tryGet(const std::string & path, std::string & res, Coordination::Stat * stat, const EventPtr & watch, int * return_code)
{
return tryGetWatch(path, res, stat, callbackForEvent(watch), return_code);
}
bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback, int * return_code)
bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback, int * return_code)
{
int32_t code = getImpl(path, res, stat, watch_callback);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE))
if (!(code == Coordination::ZOK || code == Coordination::ZNONODE))
throw KeeperException(code, path);
if (return_code)
*return_code = code;
return code == ZooKeeperImpl::ZooKeeper::ZOK;
return code == Coordination::ZOK;
}
int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data,
int32_t version, Stat * stat)
int32_t version, Coordination::Stat * stat)
{
int32_t code = 0;
Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::SetResponse & response)
auto callback = [&](const Coordination::SetResponse & response)
{
code = response.error;
if (!code && stat)
@ -433,7 +434,7 @@ int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data,
return code;
}
void ZooKeeper::set(const std::string & path, const std::string & data, int32_t version, Stat * stat)
void ZooKeeper::set(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat)
{
check(trySet(path, data, version, stat), path);
}
@ -441,36 +442,36 @@ void ZooKeeper::set(const std::string & path, const std::string & data, int32_t
void ZooKeeper::createOrUpdate(const std::string & path, const std::string & data, int32_t mode)
{
int32_t code = trySet(path, data, -1);
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
if (code == Coordination::ZNONODE)
{
create(path, data, mode);
}
else if (code != ZooKeeperImpl::ZooKeeper::ZOK)
else if (code != Coordination::ZOK)
throw KeeperException(code, path);
}
int32_t ZooKeeper::trySet(const std::string & path, const std::string & data,
int32_t version, Stat * stat)
int32_t version, Coordination::Stat * stat)
{
int32_t code = setImpl(path, data, version, stat);
if (!(code == ZooKeeperImpl::ZooKeeper::ZOK ||
code == ZooKeeperImpl::ZooKeeper::ZNONODE ||
code == ZooKeeperImpl::ZooKeeper::ZBADVERSION))
if (!(code == Coordination::ZOK ||
code == Coordination::ZNONODE ||
code == Coordination::ZBADVERSION))
throw KeeperException(code, path);
return code;
}
int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses)
int32_t ZooKeeper::multiImpl(const Coordination::Requests & requests, Coordination::Responses & responses)
{
if (requests.empty())
return ZooKeeperImpl::ZooKeeper::ZOK;
return Coordination::ZOK;
int32_t code = 0;
Poco::Event event;
auto callback = [&](const ZooKeeperImpl::ZooKeeper::MultiResponse & response)
auto callback = [&](const Coordination::MultiResponse & response)
{
code = response.error;
responses = response.responses;
@ -482,18 +483,18 @@ int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses)
return code;
}
Responses ZooKeeper::multi(const Requests & requests)
Coordination::Responses ZooKeeper::multi(const Coordination::Requests & requests)
{
Responses responses;
Coordination::Responses responses;
int32_t code = multiImpl(requests, responses);
KeeperMultiException::check(code, requests, responses);
return responses;
}
int32_t ZooKeeper::tryMulti(const Requests & requests, Responses & responses)
int32_t ZooKeeper::tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses)
{
int32_t code = multiImpl(requests, responses);
if (code && !ZooKeeperImpl::ZooKeeper::isUserError(code))
if (code && !Coordination::isUserError(code))
throw KeeperException(code);
return code;
}
@ -504,7 +505,7 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path)
Strings children = getChildren(path);
while (!children.empty())
{
Requests ops;
Coordination::Requests ops;
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
{
removeChildrenRecursive(path + "/" + children.back());
@ -518,11 +519,11 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path)
void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path)
{
Strings children;
if (tryGetChildren(path, children) != ZooKeeperImpl::ZooKeeper::ZOK)
if (tryGetChildren(path, children) != Coordination::ZOK)
return;
while (!children.empty())
{
Requests ops;
Coordination::Requests ops;
Strings batch;
for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i)
{
@ -530,17 +531,17 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path)
children.pop_back();
tryRemoveChildrenRecursive(batch.back());
ZooKeeperImpl::ZooKeeper::RemoveRequest request;
Coordination::RemoveRequest request;
request.path = batch.back();
ops.emplace_back(std::make_shared<ZooKeeperImpl::ZooKeeper::RemoveRequest>(std::move(request)));
ops.emplace_back(std::make_shared<Coordination::RemoveRequest>(std::move(request)));
}
/// Try to remove the children with a faster method - in bulk. If this fails,
/// this means someone is concurrently removing these children and we will have
/// to remove them one by one.
Responses responses;
if (tryMulti(ops, responses) != ZooKeeperImpl::ZooKeeper::ZOK)
Coordination::Responses responses;
if (tryMulti(ops, responses) != Coordination::ZOK)
for (const std::string & child : batch)
tryRemove(child);
}
@ -576,14 +577,14 @@ void ZooKeeper::waitForDisappear(const std::string & path)
while (true)
{
auto callback = [state](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response)
auto callback = [state](const Coordination::ExistsResponse & response)
{
state->code = response.error;
if (state->code)
state->event.set();
};
auto watch = [state](const ZooKeeperImpl::ZooKeeper::WatchResponse & response)
auto watch = [state](const Coordination::WatchResponse & response)
{
if (!state->code)
{
@ -599,13 +600,13 @@ void ZooKeeper::waitForDisappear(const std::string & path)
impl->exists(path, callback, watch);
state->event.wait();
if (state->code == ZooKeeperImpl::ZooKeeper::ZNONODE)
if (state->code == Coordination::ZNONODE)
return;
if (state->code)
throw KeeperException(state->code, path);
if (state->event_type == ZooKeeperImpl::ZooKeeper::DELETED)
if (state->event_type == Coordination::DELETED)
return;
}
}
@ -618,7 +619,7 @@ ZooKeeperPtr ZooKeeper::startNewSession() const
std::string ZooKeeper::error2string(int32_t code)
{
return ZooKeeperImpl::ZooKeeper::errorMessage(code);
return Coordination::errorMessage(code);
}
bool ZooKeeper::expired()
@ -632,13 +633,13 @@ Int64 ZooKeeper::getClientID()
}
std::future<ZooKeeperImpl::ZooKeeper::CreateResponse> ZooKeeper::asyncCreate(const std::string & path, const std::string & data, int32_t mode)
std::future<Coordination::CreateResponse> ZooKeeper::asyncCreate(const std::string & path, const std::string & data, int32_t mode)
{
/// https://stackoverflow.com/questions/25421346/how-to-create-an-stdfunction-from-a-move-capturing-lambda-expression
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::CreateResponse>>();
auto promise = std::make_shared<std::promise<Coordination::CreateResponse>>();
auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::CreateResponse & response) mutable
auto callback = [promise, path](const Coordination::CreateResponse & response) mutable
{
if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -651,12 +652,12 @@ std::future<ZooKeeperImpl::ZooKeeper::CreateResponse> ZooKeeper::asyncCreate(con
}
std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncGet(const std::string & path)
std::future<Coordination::GetResponse> ZooKeeper::asyncGet(const std::string & path)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::GetResponse>>();
auto promise = std::make_shared<std::promise<Coordination::GetResponse>>();
auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable
auto callback = [promise, path](const Coordination::GetResponse & response) mutable
{
if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -669,14 +670,14 @@ std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncGet(const std
}
std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncTryGet(const std::string & path)
std::future<Coordination::GetResponse> ZooKeeper::asyncTryGet(const std::string & path)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::GetResponse>>();
auto promise = std::make_shared<std::promise<Coordination::GetResponse>>();
auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable
auto callback = [promise, path](const Coordination::GetResponse & response) mutable
{
if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE)
if (response.error && response.error != Coordination::ZNONODE)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
else
promise->set_value(response);
@ -686,14 +687,14 @@ std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncTryGet(const
return future;
}
std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse> ZooKeeper::asyncExists(const std::string & path)
std::future<Coordination::ExistsResponse> ZooKeeper::asyncExists(const std::string & path)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::ExistsResponse>>();
auto promise = std::make_shared<std::promise<Coordination::ExistsResponse>>();
auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) mutable
auto callback = [promise, path](const Coordination::ExistsResponse & response) mutable
{
if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE)
if (response.error && response.error != Coordination::ZNONODE)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
else
promise->set_value(response);
@ -703,12 +704,12 @@ std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse> ZooKeeper::asyncExists(con
return future;
}
std::future<ZooKeeperImpl::ZooKeeper::SetResponse> ZooKeeper::asyncSet(const std::string & path, const std::string & data, int32_t version)
std::future<Coordination::SetResponse> ZooKeeper::asyncSet(const std::string & path, const std::string & data, int32_t version)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::SetResponse>>();
auto promise = std::make_shared<std::promise<Coordination::SetResponse>>();
auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::SetResponse & response) mutable
auto callback = [promise, path](const Coordination::SetResponse & response) mutable
{
if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -720,12 +721,12 @@ std::future<ZooKeeperImpl::ZooKeeper::SetResponse> ZooKeeper::asyncSet(const std
return future;
}
std::future<ZooKeeperImpl::ZooKeeper::ListResponse> ZooKeeper::asyncGetChildren(const std::string & path)
std::future<Coordination::ListResponse> ZooKeeper::asyncGetChildren(const std::string & path)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::ListResponse>>();
auto promise = std::make_shared<std::promise<Coordination::ListResponse>>();
auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::ListResponse & response) mutable
auto callback = [promise, path](const Coordination::ListResponse & response) mutable
{
if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -737,12 +738,12 @@ std::future<ZooKeeperImpl::ZooKeeper::ListResponse> ZooKeeper::asyncGetChildren(
return future;
}
std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncRemove(const std::string & path, int32_t version)
std::future<Coordination::RemoveResponse> ZooKeeper::asyncRemove(const std::string & path, int32_t version)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::RemoveResponse>>();
auto promise = std::make_shared<std::promise<Coordination::RemoveResponse>>();
auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable
auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable
{
if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
@ -754,14 +755,14 @@ std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncRemove(con
return future;
}
std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncTryRemove(const std::string & path, int32_t version)
std::future<Coordination::RemoveResponse> ZooKeeper::asyncTryRemove(const std::string & path, int32_t version)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::RemoveResponse>>();
auto promise = std::make_shared<std::promise<Coordination::RemoveResponse>>();
auto future = promise->get_future();
auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable
auto callback = [promise, path](const Coordination::RemoveResponse & response) mutable
{
if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE && response.error != ZooKeeperImpl::ZooKeeper::ZBADVERSION && response.error != ZooKeeperImpl::ZooKeeper::ZNOTEMPTY)
if (response.error && response.error != Coordination::ZNONODE && response.error != Coordination::ZBADVERSION && response.error != Coordination::ZNOTEMPTY)
promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error)));
else
promise->set_value(response);
@ -771,12 +772,12 @@ std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncTryRemove(
return future;
}
std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::tryAsyncMulti(const Requests & ops)
std::future<Coordination::MultiResponse> ZooKeeper::tryAsyncMulti(const Coordination::Requests & ops)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::MultiResponse>>();
auto promise = std::make_shared<std::promise<Coordination::MultiResponse>>();
auto future = promise->get_future();
auto callback = [promise](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable
auto callback = [promise](const Coordination::MultiResponse & response) mutable
{
promise->set_value(response);
};
@ -785,12 +786,12 @@ std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::tryAsyncMulti(co
return future;
}
std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::asyncMulti(const Requests & ops)
std::future<Coordination::MultiResponse> ZooKeeper::asyncMulti(const Coordination::Requests & ops)
{
auto promise = std::make_shared<std::promise<ZooKeeperImpl::ZooKeeper::MultiResponse>>();
auto promise = std::make_shared<std::promise<Coordination::MultiResponse>>();
auto future = promise->get_future();
auto callback = [promise](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable
auto callback = [promise](const Coordination::MultiResponse & response) mutable
{
if (response.error)
promise->set_exception(std::make_exception_ptr(KeeperException(response.error)));
@ -802,20 +803,20 @@ std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::asyncMulti(const
return future;
}
int32_t ZooKeeper::tryMultiNoThrow(const Requests & requests, Responses & responses)
int32_t ZooKeeper::tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses)
{
try
{
return multiImpl(requests, responses);
}
catch (const ZooKeeperImpl::Exception & e)
catch (const Coordination::Exception & e)
{
return e.code;
}
}
size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & responses)
size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Coordination::Responses & responses)
{
if (responses.empty())
throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR);
@ -824,7 +825,7 @@ size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & re
if (responses[index]->error)
return index;
if (!ZooKeeperImpl::ZooKeeper::isUserError(code))
if (!Coordination::isUserError(code))
throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(code) + "' is not valid response code for that",
DB::ErrorCodes::LOGICAL_ERROR);
@ -832,7 +833,7 @@ size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & re
}
KeeperMultiException::KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses)
KeeperMultiException::KeeperMultiException(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses)
: KeeperException("Transaction failed", code),
requests(requests), responses(responses), failed_op_index(getFailedOpIndex(code, responses))
{
@ -845,21 +846,21 @@ std::string KeeperMultiException::getPathForFirstFailedOp() const
return requests[failed_op_index]->getPath();
}
void KeeperMultiException::check(int32_t code, const Requests & requests, const Responses & responses)
void KeeperMultiException::check(int32_t code, const Coordination::Requests & requests, const Coordination::Responses & responses)
{
if (!code)
return;
if (ZooKeeperImpl::ZooKeeper::isUserError(code))
if (Coordination::isUserError(code))
throw KeeperMultiException(code, requests, responses);
else
throw KeeperException(code);
}
RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode)
Coordination::RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode)
{
auto request = std::make_shared<CreateRequest>();
auto request = std::make_shared<Coordination::CreateRequest>();
request->path = path;
request->data = data;
request->is_ephemeral = create_mode == CreateMode::Ephemeral || create_mode == CreateMode::EphemeralSequential;
@ -867,26 +868,26 @@ RequestPtr makeCreateRequest(const std::string & path, const std::string & data,
return request;
}
RequestPtr makeRemoveRequest(const std::string & path, int version)
Coordination::RequestPtr makeRemoveRequest(const std::string & path, int version)
{
auto request = std::make_shared<RemoveRequest>();
auto request = std::make_shared<Coordination::RemoveRequest>();
request->path = path;
request->version = version;
return request;
}
RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version)
Coordination::RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version)
{
auto request = std::make_shared<SetRequest>();
auto request = std::make_shared<Coordination::SetRequest>();
request->path = path;
request->data = data;
request->version = version;
return request;
}
RequestPtr makeCheckRequest(const std::string & path, int version)
Coordination::RequestPtr makeCheckRequest(const std::string & path, int version)
{
auto request = std::make_shared<CheckRequest>();
auto request = std::make_shared<Coordination::CheckRequest>();
request->path = path;
request->version = version;
return request;

View File

@ -10,6 +10,7 @@
#include <common/logger_useful.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <port/unistd.h>
@ -109,20 +110,20 @@ public:
/// * The node has children.
int32_t tryRemove(const std::string & path, int32_t version = -1);
bool exists(const std::string & path, Stat * stat = nullptr, const EventPtr & watch = nullptr);
bool existsWatch(const std::string & path, Stat * stat, WatchCallback watch_callback);
bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr);
bool existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
std::string get(const std::string & path, Stat * stat = nullptr, const EventPtr & watch = nullptr);
std::string getWatch(const std::string & path, Stat * stat, WatchCallback watch_callback);
std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr);
std::string getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
/// Doesn't not throw in the following cases:
/// * The node doesn't exist. Returns false in this case.
bool tryGet(const std::string & path, std::string & res, Stat * stat = nullptr, const EventPtr & watch = nullptr, int * code = nullptr);
bool tryGet(const std::string & path, std::string & res, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr, int * code = nullptr);
bool tryGetWatch(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback, int * code = nullptr);
bool tryGetWatch(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback, int * code = nullptr);
void set(const std::string & path, const std::string & data,
int32_t version = -1, Stat * stat = nullptr);
int32_t version = -1, Coordination::Stat * stat = nullptr);
/// Creates the node if it doesn't exist. Updates its contents otherwise.
void createOrUpdate(const std::string & path, const std::string & data, int32_t mode);
@ -131,34 +132,34 @@ public:
/// * The node doesn't exist.
/// * Versions do not match.
int32_t trySet(const std::string & path, const std::string & data,
int32_t version = -1, Stat * stat = nullptr);
int32_t version = -1, Coordination::Stat * stat = nullptr);
Strings getChildren(const std::string & path,
Stat * stat = nullptr,
Coordination::Stat * stat = nullptr,
const EventPtr & watch = nullptr);
Strings getChildrenWatch(const std::string & path,
Stat * stat,
WatchCallback watch_callback);
Coordination::Stat * stat,
Coordination::WatchCallback watch_callback);
/// Doesn't not throw in the following cases:
/// * The node doesn't exist.
int32_t tryGetChildren(const std::string & path, Strings & res,
Stat * stat = nullptr,
Coordination::Stat * stat = nullptr,
const EventPtr & watch = nullptr);
int32_t tryGetChildrenWatch(const std::string & path, Strings & res,
Stat * stat,
WatchCallback watch_callback);
Coordination::Stat * stat,
Coordination::WatchCallback watch_callback);
/// Performs several operations in a transaction.
/// Throws on every error.
Responses multi(const Requests & requests);
Coordination::Responses multi(const Coordination::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 Requests & requests, Responses & responses);
int32_t tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses);
/// Throws nothing (even session expired errors)
int32_t tryMultiNoThrow(const Requests & requests, Responses & responses);
int32_t tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses);
Int64 getClientID();
@ -190,24 +191,24 @@ public:
///
/// Future should not be destroyed before the result is gotten.
using FutureCreate = std::future<ZooKeeperImpl::ZooKeeper::CreateResponse>;
using FutureCreate = std::future<Coordination::CreateResponse>;
FutureCreate asyncCreate(const std::string & path, const std::string & data, int32_t mode);
using FutureGet = std::future<ZooKeeperImpl::ZooKeeper::GetResponse>;
using FutureGet = std::future<Coordination::GetResponse>;
FutureGet asyncGet(const std::string & path);
FutureGet asyncTryGet(const std::string & path);
using FutureExists = std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse>;
using FutureExists = std::future<Coordination::ExistsResponse>;
FutureExists asyncExists(const std::string & path);
using FutureGetChildren = std::future<ZooKeeperImpl::ZooKeeper::ListResponse>;
using FutureGetChildren = std::future<Coordination::ListResponse>;
FutureGetChildren asyncGetChildren(const std::string & path);
using FutureSet = std::future<ZooKeeperImpl::ZooKeeper::SetResponse>;
using FutureSet = std::future<Coordination::SetResponse>;
FutureSet asyncSet(const std::string & path, const std::string & data, int32_t version = -1);
using FutureRemove = std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse>;
using FutureRemove = std::future<Coordination::RemoveResponse>;
FutureRemove asyncRemove(const std::string & path, int32_t version = -1);
/// Doesn't throw in the following cases:
@ -216,11 +217,11 @@ public:
/// * The node has children
FutureRemove asyncTryRemove(const std::string & path, int32_t version = -1);
using FutureMulti = std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>;
FutureMulti asyncMulti(const Requests & ops);
using FutureMulti = std::future<Coordination::MultiResponse>;
FutureMulti asyncMulti(const Coordination::Requests & ops);
/// Like the previous one but don't throw any exceptions on future.get()
FutureMulti tryAsyncMulti(const Requests & ops);
FutureMulti tryAsyncMulti(const Coordination::Requests & ops);
static std::string error2string(int32_t code);
@ -235,13 +236,13 @@ private:
/// 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);
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, Stat * stat);
int32_t getChildrenImpl(const std::string & path, Strings & res, Stat * stat, WatchCallback watch_callback);
int32_t multiImpl(const Requests & requests, Responses & responses);
int32_t existsImpl(const std::string & path, Stat * stat_, WatchCallback watch_callback);
int32_t getImpl(const std::string & path, std::string & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
int32_t setImpl(const std::string & path, const std::string & data, int32_t version, Coordination::Stat * stat);
int32_t getChildrenImpl(const std::string & path, Strings & res, Coordination::Stat * stat, Coordination::WatchCallback watch_callback);
int32_t multiImpl(const Coordination::Requests & requests, Coordination::Responses & responses);
int32_t existsImpl(const std::string & path, Coordination::Stat * stat_, Coordination::WatchCallback watch_callback);
std::unique_ptr<ZooKeeperImpl::ZooKeeper> impl;
std::unique_ptr<Coordination::IKeeper> impl;
std::string hosts;
std::string identity;

File diff suppressed because it is too large Load Diff

View File

@ -3,6 +3,7 @@
#include <Core/Types.h>
#include <Common/ConcurrentBoundedQueue.h>
#include <Common/CurrentMetrics.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
@ -78,349 +79,25 @@ namespace CurrentMetrics
}
namespace ZooKeeperImpl
namespace Coordination
{
using namespace DB;
class Exception : public DB::Exception
{
private:
/// Delegate constructor, used to minimize repetition; last parameter used for overload resolution.
Exception(const std::string & msg, const int32_t code, int);
public:
explicit Exception(const int32_t code);
Exception(const std::string & msg, const int32_t code);
Exception(const int32_t code, const std::string & path);
Exception(const Exception & exc);
const char * name() const throw() override { return "ZooKeeperImpl::Exception"; }
const char * className() const throw() override { return "ZooKeeperImpl::Exception"; }
Exception * clone() const override { return new Exception(*this); }
const int32_t code;
};
struct ZooKeeperRequest;
/** 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 exception with ZSESSIONEXPIRED code or method isExpired returns true,
* 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.
/** Usage scenario: look at the documentation for IKeeper class.
*/
class ZooKeeper
class ZooKeeper : public IKeeper
{
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;
using ResponsePtr = std::shared_ptr<Response>;
using Responses = std::vector<ResponsePtr>;
using ResponseCallback = std::function<void(const Response &)>;
struct Response
{
int32_t error = 0;
virtual ~Response() {}
virtual void readImpl(ReadBuffer &) = 0;
virtual void removeRootPath(const String & /* root_path */) {}
};
struct Request;
using RequestPtr = std::shared_ptr<Request>;
using Requests = std::vector<RequestPtr>;
struct Request
{
XID xid = 0;
bool has_watch = false;
virtual ~Request() {}
virtual RequestPtr clone() const = 0;
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;
};
struct HeartbeatRequest final : Request
{
RequestPtr clone() const override { return std::make_shared<HeartbeatRequest>(*this); }
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;
RequestPtr clone() const override { return std::make_shared<AuthRequest>(*this); }
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
{
RequestPtr clone() const override { return std::make_shared<CloseRequest>(*this); }
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;
RequestPtr clone() const override { return std::make_shared<CreateRequest>(*this); }
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;
RequestPtr clone() const override { return std::make_shared<RemoveRequest>(*this); }
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;
RequestPtr clone() const override { return std::make_shared<ExistsRequest>(*this); }
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;
RequestPtr clone() const override { return std::make_shared<GetRequest>(*this); }
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;
RequestPtr clone() const override { return std::make_shared<SetRequest>(*this); }
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;
RequestPtr clone() const override { return std::make_shared<ListRequest>(*this); }
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;
RequestPtr clone() const override { return std::make_shared<CheckRequest>(*this); }
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;
RequestPtr clone() const override;
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.
@ -438,30 +115,13 @@ public:
/// If expired, you can only destroy the object. All other methods will throw exception.
bool isExpired() const { return expired; }
bool isExpired() const override { return expired; }
/// Useful to check owner of ephemeral node.
int64_t getSessionID() const { return session_id; }
int64_t getSessionID() const override { 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 &)>;
/// If the method will throw an exception, callbacks won't be called.
///
/// After the method is executed successfully, you must wait for callbacks
/// (don't destroy callback data before it will be called).
///
/// All callbacks are executed sequentially (the execution of callbacks is serialized).
///
/// If an exception is thrown inside the callback, the session will expire,
/// and all other callbacks will be called with "Session expired" error.
/// See the documentation about semantics of these methods in IKeeper class.
void create(
const String & path,
@ -469,114 +129,42 @@ public:
bool is_ephemeral,
bool is_sequential,
const ACLs & acls,
CreateCallback callback);
CreateCallback callback) override;
void remove(
const String & path,
int32_t version,
RemoveCallback callback);
RemoveCallback callback) override;
void exists(
const String & path,
ExistsCallback callback,
WatchCallback watch);
WatchCallback watch) override;
void get(
const String & path,
GetCallback callback,
WatchCallback watch);
WatchCallback watch) override;
void set(
const String & path,
const String & data,
int32_t version,
SetCallback callback);
SetCallback callback) override;
void list(
const String & path,
ListCallback callback,
WatchCallback watch);
WatchCallback watch) override;
void check(
const String & path,
int32_t version,
CheckCallback callback);
CheckCallback callback) override;
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
};
/// Network errors and similar. You should reinitialize ZooKeeper session in case of these errors
static bool isHardwareError(int32_t code);
/// Valid errors sent from the server about database state (like "no node"). Logical and authentication errors (like "bad arguments") are not here.
static bool isUserError(int32_t code);
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
};
MultiCallback callback) override;
private:
String root_path;
@ -599,7 +187,7 @@ private:
struct RequestInfo
{
RequestPtr request;
std::shared_ptr<ZooKeeperRequest> request;
ResponseCallback callback;
WatchCallback watch;
clock::time_point time;

View File

@ -41,18 +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](const ZooKeeperImpl::ZooKeeper::WatchResponse & response)
auto watch_callback = [context=context](const Coordination::WatchResponse & response)
{
if (!(response.type != ZooKeeperImpl::ZooKeeper::SESSION || response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION))
if (!(response.type != Coordination::SESSION || response.state == Coordination::EXPIRED_SESSION))
return;
bool changed = false;
{
std::lock_guard<std::mutex> lock(context->mutex);
if (response.type != ZooKeeperImpl::ZooKeeper::SESSION)
if (response.type != Coordination::SESSION)
changed = context->invalidated_paths.emplace(response.path).second;
else if (response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION)
else if (response.state == Coordination::EXPIRED_SESSION)
{
context->zookeeper = nullptr;
context->invalidated_paths.clear();

View File

@ -33,7 +33,7 @@ TEST(zkutil, multi_nice_exception_msg)
{
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
zkutil::Requests ops;
Coordination::Requests ops;
ASSERT_NO_THROW(
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
@ -70,7 +70,7 @@ TEST(zkutil, multi_nice_exception_msg)
TEST(zkutil, multi_async)
{
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
zkutil::Requests ops;
Coordination::Requests ops;
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
@ -88,13 +88,13 @@ TEST(zkutil, multi_async)
ops.clear();
auto res = fut.get();
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZOK);
ASSERT_TRUE(res.error == Coordination::ZOK);
ASSERT_EQ(res.responses.size(), 2);
}
EXPECT_ANY_THROW
(
std::vector<std::future<ZooKeeperImpl::ZooKeeper::MultiResponse>> futures;
std::vector<std::future<Coordination::MultiResponse>> futures;
for (size_t i = 0; i < 10000; ++i)
{
@ -124,7 +124,7 @@ TEST(zkutil, multi_async)
ops.clear();
auto res = fut.get();
ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS);
ASSERT_TRUE(res.error == Coordination::ZNODEEXISTS);
ASSERT_EQ(res.responses.size(), 2);
}
}
@ -176,11 +176,11 @@ TEST(zkutil, multi_create_sequential)
zookeeper->tryRemoveRecursive(base_path);
zookeeper->createAncestors(base_path + "/");
zkutil::Requests ops;
Coordination::Requests ops;
String sequential_node_prefix = base_path + "/queue-";
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));
const auto & sequential_node_result_op = dynamic_cast<const Coordination::CreateResponse &>(*results.at(0));
EXPECT_FALSE(sequential_node_result_op.path_created.empty());
EXPECT_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length());

View File

@ -34,11 +34,11 @@ int main(int argc, char ** argv)
while (true)
{
{
zkutil::Requests ops;
Coordination::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::Responses responses;
Coordination::Responses responses;
int32_t code = zk.tryMultiNoThrow(ops, responses);
std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(code) << std::endl;
@ -57,7 +57,7 @@ int main(int argc, char ** argv)
sleep(1);
}
}
catch (zkutil::KeeperException & e)
catch (Coordination::Exception & e)
{
std::cerr << "KeeperException: " << DB::getCurrentExceptionMessage(true) << std::endl;
return 1;

View File

@ -23,7 +23,7 @@ try
{
while (true)
{
std::vector<std::future<zkutil::GetResponse>> futures;
std::vector<std::future<Coordination::GetResponse>> futures;
for (auto & node : nodes)
futures.push_back(zookeeper.asyncGet("/tmp/" + node));

View File

@ -20,7 +20,7 @@ try
std::cout << "create path" << std::endl;
zk.create("/test", "old", zkutil::CreateMode::Persistent);
zkutil::Stat stat;
Coordination::Stat stat;
zkutil::EventPtr watch = std::make_shared<Poco::Event>();
std::cout << "get path" << std::endl;
@ -38,13 +38,13 @@ try
zk.remove("/test");
zkutil::Requests ops;
Coordination::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;
Coordination::Responses res = zk.multi(ops);
std::cout << "path created: " << dynamic_cast<const Coordination::CreateResponse &>(*res[0]).path_created << std::endl;
return 0;
}

View File

@ -7,7 +7,7 @@
#include <boost/algorithm/string.hpp>
using namespace ZooKeeperImpl;
using namespace Coordination;
int main(int argc, char ** argv)
@ -38,10 +38,10 @@ try
std::cout << "create\n";
zk.create("/test", "old", false, false, {},
[&](const ZooKeeper::CreateResponse & response)
[&](const CreateResponse & response)
{
if (response.error)
std::cerr << "Error (create) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Error (create) " << response.error << ": " << errorMessage(response.error) << '\n';
else
std::cerr << "Created path: " << response.path_created << '\n';
@ -53,19 +53,19 @@ try
std::cout << "get\n";
zk.get("/test",
[&](const ZooKeeper::GetResponse & response)
[&](const GetResponse & response)
{
if (response.error)
std::cerr << "Error (get) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Error (get) " << response.error << ": " << errorMessage(response.error) << '\n';
else
std::cerr << "Value: " << response.data << '\n';
//event.set();
},
[](const ZooKeeper::WatchResponse & response)
[](const WatchResponse & response)
{
if (response.error)
std::cerr << "Watch (get) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Watch (get) on /test, Error " << response.error << ": " << errorMessage(response.error) << '\n';
else
std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n';
});
@ -75,10 +75,10 @@ try
std::cout << "set\n";
zk.set("/test", "new", -1,
[&](const ZooKeeper::SetResponse & response)
[&](const SetResponse & response)
{
if (response.error)
std::cerr << "Error (set) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Error (set) " << response.error << ": " << errorMessage(response.error) << '\n';
else
std::cerr << "Set\n";
@ -90,10 +90,10 @@ try
std::cout << "list\n";
zk.list("/",
[&](const ZooKeeper::ListResponse & response)
[&](const ListResponse & response)
{
if (response.error)
std::cerr << "Error (list) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Error (list) " << response.error << ": " << errorMessage(response.error) << '\n';
else
{
std::cerr << "Children:\n";
@ -103,10 +103,10 @@ try
//event.set();
},
[](const ZooKeeper::WatchResponse & response)
[](const WatchResponse & response)
{
if (response.error)
std::cerr << "Watch (list) on /, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Watch (list) on /, Error " << response.error << ": " << errorMessage(response.error) << '\n';
else
std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n';
});
@ -116,19 +116,19 @@ try
std::cout << "exists\n";
zk.exists("/test",
[&](const ZooKeeper::ExistsResponse & response)
[&](const ExistsResponse & response)
{
if (response.error)
std::cerr << "Error (exists) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Error (exists) " << response.error << ": " << errorMessage(response.error) << '\n';
else
std::cerr << "Exists\n";
//event.set();
},
[](const ZooKeeper::WatchResponse & response)
[](const WatchResponse & response)
{
if (response.error)
std::cerr << "Watch (exists) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Watch (exists) on /test, Error " << response.error << ": " << errorMessage(response.error) << '\n';
else
std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n';
});
@ -137,10 +137,10 @@ try
std::cout << "remove\n";
zk.remove("/test", -1, [&](const ZooKeeper::RemoveResponse & response)
zk.remove("/test", -1, [&](const RemoveResponse & response)
{
if (response.error)
std::cerr << "Error (remove) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Error (remove) " << response.error << ": " << errorMessage(response.error) << '\n';
else
std::cerr << "Removed\n";
@ -151,39 +151,39 @@ try
std::cout << "multi\n";
ZooKeeper::Requests ops;
Requests ops;
{
ZooKeeper::CreateRequest create_request;
CreateRequest create_request;
create_request.path = "/test";
create_request.data = "multi1";
ops.emplace_back(std::make_shared<ZooKeeper::CreateRequest>(std::move(create_request)));
ops.emplace_back(std::make_shared<CreateRequest>(std::move(create_request)));
}
{
ZooKeeper::SetRequest set_request;
SetRequest set_request;
set_request.path = "/test";
set_request.data = "multi2";
ops.emplace_back(std::make_shared<ZooKeeper::SetRequest>(std::move(set_request)));
ops.emplace_back(std::make_shared<SetRequest>(std::move(set_request)));
}
{
ZooKeeper::RemoveRequest remove_request;
RemoveRequest remove_request;
remove_request.path = "/test";
ops.emplace_back(std::make_shared<ZooKeeper::RemoveRequest>(std::move(remove_request)));
ops.emplace_back(std::make_shared<RemoveRequest>(std::move(remove_request)));
}
zk.multi(ops, [&](const ZooKeeper::MultiResponse & response)
zk.multi(ops, [&](const MultiResponse & response)
{
if (response.error)
std::cerr << "Error (multi) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n';
std::cerr << "Error (multi) " << response.error << ": " << 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 << "Error (elem) " << elem->error << ": " << errorMessage(elem->error) << '\n';
std::cerr << "Created path: " << typeid_cast<const ZooKeeper::CreateResponse &>(*response.responses[0]).path_created << '\n';
std::cerr << "Created path: " << dynamic_cast<const CreateResponse &>(*response.responses[0]).path_created << '\n';
}
event.set();

View File

@ -5,12 +5,12 @@
int main()
try
{
ZooKeeperImpl::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}, {0, 50000});
Coordination::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}, {0, 50000});
zookeeper.create("/test", "hello", false, false, {}, [](const ZooKeeperImpl::ZooKeeper::CreateResponse & response)
zookeeper.create("/test", "hello", false, false, {}, [](const Coordination::CreateResponse & response)
{
if (response.error)
std::cerr << "Error " << response.error << ": " << ZooKeeperImpl::ZooKeeper::errorMessage(response.error) << "\n";
std::cerr << "Error " << response.error << ": " << Coordination::errorMessage(response.error) << "\n";
else
std::cerr << "Path created: " << response.path_created << "\n";
});

View File

@ -1,6 +1,5 @@
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <thread>
#include <fstream>
#if defined(__x86_64__)
@ -14,25 +13,6 @@
unsigned getNumberOfPhysicalCPUCores()
{
#if defined(__linux__)
/// On Linux we try to look at Cgroups limit if it is available.
std::ifstream cgroup_read_in("/sys/fs/cgroup/cpu/cpu.cfs_quota_us");
if (cgroup_read_in.is_open())
{
std::string allocated_cpus_share_str{ std::istreambuf_iterator<char>(cgroup_read_in), std::istreambuf_iterator<char>() };
int allocated_cpus_share_int = std::stoi(allocated_cpus_share_str);
cgroup_read_in.close();
// If a valid value is present
if (allocated_cpus_share_int > 0)
{
unsigned allocated_cpus = (allocated_cpus_share_int + 999) / 1000;
return allocated_cpus;
}
}
#endif
#if defined(__x86_64__)
cpu_raw_data_t raw_data;
if (0 != cpuid_get_raw_data(&raw_data))

View File

@ -1,4 +1,5 @@
#pragma once
#include <string>
/// Maps 0..15 to 0..9A..F or 0..9a..f correspondingly.

View File

@ -39,13 +39,17 @@ std::string getThreadName()
{
std::string name(16, '\0');
#if defined(__FreeBSD__) || defined(__APPLE__)
if (pthread_get_name_np(pthread_self(), name.data(), name.size());
throw DB::Exception("Cannot get thread name with pthread_get_name_np()", DB::ErrorCodes::PTHREAD_ERROR);
#if defined(__APPLE__)
if (pthread_getname_np(pthread_self(), name.data(), name.size()))
throw DB::Exception("Cannot get thread name with pthread_getname_np()", DB::ErrorCodes::PTHREAD_ERROR);
#elif defined(__FreeBSD__)
// TODO: make test. freebsd will have this function soon https://freshbsd.org/commit/freebsd/r337983
// if (pthread_get_name_np(pthread_self(), name.data(), name.size()))
// throw DB::Exception("Cannot get thread name with pthread_get_name_np()", DB::ErrorCodes::PTHREAD_ERROR);
#else
if (0 != prctl(PR_GET_NAME, name.data(), 0, 0, 0))
#endif
DB::throwFromErrno("Cannot get thread name with prctl(PR_GET_NAME)");
#endif
name.resize(std::strlen(name.data()));
return name;

View File

@ -43,17 +43,7 @@ bool BackgroundSchedulePool::TaskInfo::schedule()
if (deactivated || scheduled)
return false;
scheduled = true;
if (delayed)
pool.cancelDelayedTask(shared_from_this(), lock);
/// If the task is not executing at the moment, enqueue it for immediate execution.
/// But if it is currently executing, do nothing because it will be enqueued
/// at the end of the execute() method.
if (!executing)
pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));
scheduleImpl(lock);
return true;
}
@ -89,6 +79,18 @@ void BackgroundSchedulePool::TaskInfo::activate()
deactivated = false;
}
bool BackgroundSchedulePool::TaskInfo::activateAndSchedule()
{
std::lock_guard lock(schedule_mutex);
deactivated = false;
if (scheduled)
return false;
scheduleImpl(lock);
return true;
}
void BackgroundSchedulePool::TaskInfo::execute()
{
Stopwatch watch;
@ -129,9 +131,23 @@ void BackgroundSchedulePool::TaskInfo::execute()
}
}
zkutil::WatchCallback BackgroundSchedulePool::TaskInfo::getWatchCallback()
void BackgroundSchedulePool::TaskInfo::scheduleImpl(std::lock_guard<std::mutex> & schedule_mutex_lock)
{
return [t = shared_from_this()](const ZooKeeperImpl::ZooKeeper::WatchResponse &)
scheduled = true;
if (delayed)
pool.cancelDelayedTask(shared_from_this(), schedule_mutex_lock);
/// If the task is not executing at the moment, enqueue it for immediate execution.
/// But if it is currently executing, do nothing because it will be enqueued
/// at the end of the execute() method.
if (!executing)
pool.queue.enqueueNotification(new TaskNotification(shared_from_this()));
}
Coordination::WatchCallback BackgroundSchedulePool::TaskInfo::getWatchCallback()
{
return [t = shared_from_this()](const Coordination::WatchResponse &)
{
t->schedule();
};
@ -143,12 +159,6 @@ BackgroundSchedulePool::BackgroundSchedulePool(size_t size)
{
LOG_INFO(&Logger::get("BackgroundSchedulePool"), "Create BackgroundSchedulePool with " << size << " threads");
/// Put all threads of both thread pools to one thread group
/// The master thread exits immediately
CurrentThread::initializeQuery();
thread_group = CurrentThread::getGroup();
CurrentThread::detachQuery();
threads.resize(size);
for (auto & thread : threads)
thread = std::thread([this] { threadFunction(); });
@ -217,14 +227,29 @@ void BackgroundSchedulePool::cancelDelayedTask(const TaskInfoPtr & task, std::lo
}
void BackgroundSchedulePool::attachToThreadGroup()
{
std::lock_guard lock(delayed_tasks_mutex);
if (thread_group)
{
/// Put all threads to one thread pool
CurrentThread::attachTo(thread_group);
}
else
{
CurrentThread::initializeQuery();
thread_group = CurrentThread::getGroup();
}
}
void BackgroundSchedulePool::threadFunction()
{
setThreadName("BackgrSchedPool");
/// Put all threads to one thread pool
CurrentThread::attachTo(thread_group);
attachToThreadGroup();
SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); });
CurrentThread::getMemoryTracker().setMetric(CurrentMetrics::MemoryTrackingInBackgroundSchedulePool);
while (!shutdown)
@ -242,8 +267,7 @@ void BackgroundSchedulePool::delayExecutionThreadFunction()
{
setThreadName("BckSchPoolDelay");
/// Put all threads to one thread pool
CurrentThread::attachTo(thread_group);
attachToThreadGroup();
SCOPE_EXIT({ CurrentThread::detachQueryIfNotDetached(); });
while (!shutdown)

View File

@ -50,13 +50,16 @@ public:
/// Schedule for execution after specified delay.
bool scheduleAfter(size_t ms);
/// Further attempts to schedule become no-op.
/// Further attempts to schedule become no-op. Will wait till the end of the current execution of the task.
void deactivate();
void activate();
/// get zkutil::WatchCallback needed for notifications from ZooKeeper watches.
zkutil::WatchCallback getWatchCallback();
/// Atomically activate task and schedule it for execution.
bool activateAndSchedule();
/// get Coordination::WatchCallback needed for notifications from ZooKeeper watches.
Coordination::WatchCallback getWatchCallback();
private:
friend class TaskNotification;
@ -64,6 +67,8 @@ public:
void execute();
void scheduleImpl(std::lock_guard<std::mutex> & schedule_mutex_lock);
BackgroundSchedulePool & pool;
std::string log_name;
TaskFunc function;
@ -142,6 +147,8 @@ private:
/// Thread group used for profiling purposes
ThreadGroupStatusPtr thread_group;
void attachToThreadGroup();
};
using BackgroundSchedulePoolPtr = std::shared_ptr<BackgroundSchedulePool>;

View File

@ -21,43 +21,31 @@ namespace ErrorCodes
ColumnGathererStream::ColumnGathererStream(
const String & column_name_, const BlockInputStreams & source_streams, ReadBuffer & row_sources_buf_,
size_t block_preferred_size_)
: column_name(column_name_), row_sources_buf(row_sources_buf_)
: column_name(column_name_), sources(source_streams.size()), row_sources_buf(row_sources_buf_)
, block_preferred_size(block_preferred_size_), log(&Logger::get("ColumnGathererStream"))
{
if (source_streams.empty())
throw Exception("There are no streams to gather", ErrorCodes::EMPTY_DATA_PASSED);
children.assign(source_streams.begin(), source_streams.end());
}
void ColumnGathererStream::init()
{
sources.reserve(children.size());
for (size_t i = 0; i < children.size(); ++i)
{
sources.emplace_back(children[i]->read(), column_name);
Block & block = sources.back().block;
const Block & header = children[i]->getHeader();
/// Sometimes MergeTreeReader injects additional column with partitioning key
if (block.columns() > 2)
if (header.columns() > 2)
throw Exception(
"Block should have 1 or 2 columns, but contains " + toString(block.columns()),
"Block should have 1 or 2 columns, but contains " + toString(header.columns()),
ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
if (!block.has(column_name))
throw Exception(
"Not found column '" + column_name + "' in block.",
ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
if (i == 0)
{
column.name = column_name;
column.type = block.getByName(column_name).type;
column.type = header.getByName(column_name).type;
column.column = column.type->createColumn();
}
if (block.getByName(column_name).column->getName() != column.column->getName())
else if (header.getByName(column_name).column->getName() != column.column->getName())
throw Exception("Column types don't match", ErrorCodes::INCOMPATIBLE_COLUMNS);
}
}
@ -69,10 +57,6 @@ Block ColumnGathererStream::readImpl()
if (children.size() == 1 && row_sources_buf.eof())
return children[0]->read();
/// Initialize first source blocks
if (sources.empty())
init();
if (!source_to_fully_copy && row_sources_buf.eof())
return Block();

View File

@ -76,16 +76,11 @@ private:
/// Cache required fields
struct Source
{
const IColumn * column;
size_t pos;
size_t size;
const IColumn * column = nullptr;
size_t pos = 0;
size_t size = 0;
Block block;
Source(Block && block_, const String & name) : block(std::move(block_))
{
update(name);
}
void update(const String & name)
{
column = block.getByName(name).column.get();
@ -94,7 +89,6 @@ private:
}
};
void init();
void fetchNewBlock(Source & source, size_t source_num);
String column_name;

View File

@ -300,7 +300,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread(ThreadGroupSt
try
{
if (thread_group)
CurrentThread::attachTo(thread_group);
CurrentThread::attachToIfDetached(thread_group);
setThreadName("MergeAggMergThr");
while (!parallel_merge_data->finish)

View File

@ -669,6 +669,10 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams(
}
};
if (!settings.continuous_reading)
state_with_dictionary->num_pending_rows = 0;
bool first_dictionary = true;
while (limit)
{
if (state_with_dictionary->num_pending_rows == 0)
@ -681,8 +685,11 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams(
index_type.deserialize(*indexes_stream);
if (index_type.need_global_dictionary && (!global_dictionary || index_type.need_update_dictionary))
if (index_type.need_global_dictionary && (!global_dictionary || index_type.need_update_dictionary || (first_dictionary && !settings.continuous_reading)))
{
readDictionary();
first_dictionary = false;
}
if (state_with_dictionary->index_type.has_additional_keys)
readAdditionalKeys();

View File

@ -135,6 +135,9 @@ public:
InputStreamGetter getter;
SubstreamPath path;
/// True if continue reading from previous positions in file. False if made fseek to the start of new granule.
bool continuous_reading = true;
bool position_independent_encoding = true;
/// If not zero, may be used to avoid reallocations while reading column of String type.
double avg_value_size_hint = 0;

View File

@ -1,4 +1,5 @@
#pragma once
#include <Columns/ColumnVector.h>
#include <Columns/ColumnString.h>
#include <Columns/IColumn.h>
@ -21,23 +22,24 @@ namespace ErrorCodes
}
/*
* BlockInputStream implementation for external dictionaries
* read() returns single block consisting of the in-memory contents of the dictionaries
/* BlockInputStream implementation for external dictionaries
* read() returns blocks consisting of the in-memory contents of the dictionaries
*/
template <typename DictionaryType, typename Key>
class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase
{
public:
using DictionatyPtr = std::shared_ptr<DictionaryType const>;
using DictionaryPtr = std::shared_ptr<DictionaryType const>;
DictionaryBlockInputStream(std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
PaddedPODArray<Key> && ids, const Names & column_names);
DictionaryBlockInputStream(std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
const std::vector<StringRef> & keys, const Names & column_names);
using GetColumnsFunction =
std::function<ColumnsWithTypeAndName(const Columns &, const std::vector<DictionaryAttribute> & attributes)>;
// Used to separate key columns format for storage and view.
// Calls get_key_columns_function to get key column for dictionary get fuction call
// and get_view_columns_function to get key representation.
@ -59,64 +61,69 @@ private:
// pointer types to getXXX functions
// for single key dictionaries
template <typename Type>
using DictionaryGetter = void (DictionaryType::*)(
const std::string &, const PaddedPODArray<Key> &, PaddedPODArray<Type> &) const;
using DictionaryStringGetter = void (DictionaryType::*)(
const std::string &, const PaddedPODArray<Key> &, ColumnString *) const;
using DictionaryGetter = void (DictionaryType::*)(const std::string &, const PaddedPODArray<Key> &, PaddedPODArray<Type> &) const;
using DictionaryStringGetter = void (DictionaryType::*)(const std::string &, const PaddedPODArray<Key> &, ColumnString *) const;
// for complex complex key dictionaries
template <typename Type>
using GetterByKey = void (DictionaryType::*)(
const std::string &, const Columns &, const DataTypes &, PaddedPODArray<Type> & out) const;
using StringGetterByKey = void (DictionaryType::*)(
const std::string &, const Columns &, const DataTypes &, ColumnString * out) const;
using GetterByKey = void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, PaddedPODArray<Type> & out) const;
using StringGetterByKey = void (DictionaryType::*)(const std::string &, const Columns &, const DataTypes &, ColumnString * out) const;
// call getXXX
// for single key dictionaries
template <typename Type, typename Container>
void callGetter(DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids,
void callGetter(DictionaryGetter<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <typename Container>
void callGetter(DictionaryStringGetter getter, const PaddedPODArray<Key> & ids,
void callGetter(DictionaryStringGetter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
// for complex complex key dictionaries
template <typename Type, typename Container>
void callGetter(GetterByKey<Type> getter, const PaddedPODArray<Key> & ids,
void callGetter(GetterByKey<Type> getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <typename Container>
void callGetter(StringGetterByKey getter, const PaddedPODArray<Key> & ids,
void callGetter(StringGetterByKey getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
Container & container, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <template <typename> class Getter, typename StringGetter>
Block fillBlock(const PaddedPODArray<Key> & ids, const Columns & keys,
Block fillBlock(const PaddedPODArray<Key> & ids_to_fill, const Columns & keys,
const DataTypes & types, ColumnsWithTypeAndName && view) const;
template <typename AttributeType, typename Getter>
ColumnPtr getColumnFromAttribute(Getter getter, const PaddedPODArray<Key> & ids,
ColumnPtr getColumnFromAttribute(Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <typename Getter>
ColumnPtr getColumnFromStringAttribute(Getter getter, const PaddedPODArray<Key> & ids,
ColumnPtr getColumnFromStringAttribute(Getter getter, const PaddedPODArray<Key> & ids_to_fill,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
ColumnPtr getColumnFromIds(const PaddedPODArray<Key> & ids) const;
ColumnPtr getColumnFromIds(const PaddedPODArray<Key> & ids_to_fill) const;
void fillKeyColumns(const std::vector<StringRef> & keys, size_t start, size_t size,
const DictionaryStructure & dictionary_structure, ColumnsWithTypeAndName & columns) const;
DictionatyPtr dictionary;
DictionaryPtr dictionary;
Names column_names;
PaddedPODArray<Key> ids;
ColumnsWithTypeAndName key_columns;
Poco::Logger * logger;
Block (DictionaryBlockInputStream<DictionaryType, Key>::*fillBlockFunction)(
const PaddedPODArray<Key> & ids, const Columns& keys,
using FillBlockFunction = Block (DictionaryBlockInputStream<DictionaryType, Key>::*)(
const PaddedPODArray<Key> & ids_to_fill, const Columns & keys,
const DataTypes & types, ColumnsWithTypeAndName && view) const;
FillBlockFunction fill_block_function;
Columns data_columns;
GetColumnsFunction get_key_columns_function;
GetColumnsFunction get_view_columns_function;
@ -131,6 +138,7 @@ private:
DictionaryKeyType key_type;
};
template <typename DictionaryType, typename Key>
DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
std::shared_ptr<const IDictionaryBase> dictionary, size_t max_block_size,
@ -139,7 +147,7 @@ DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)),
column_names(column_names), ids(std::move(ids)),
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
fillBlockFunction(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<DictionaryGetter, DictionaryStringGetter>),
fill_block_function(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<DictionaryGetter, DictionaryStringGetter>),
key_type(DictionaryKeyType::Id)
{
}
@ -151,7 +159,7 @@ DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
: DictionaryBlockInputStreamBase(keys.size(), max_block_size),
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)), column_names(column_names),
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
fillBlockFunction(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, StringGetterByKey>),
fill_block_function(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, StringGetterByKey>),
key_type(DictionaryKeyType::ComplexKey)
{
const DictionaryStructure & dictionaty_structure = dictionary->getStructure();
@ -167,13 +175,14 @@ DictionaryBlockInputStream<DictionaryType, Key>::DictionaryBlockInputStream(
: DictionaryBlockInputStreamBase(data_columns.front()->size(), max_block_size),
dictionary(std::static_pointer_cast<const DictionaryType>(dictionary)), column_names(column_names),
logger(&Poco::Logger::get("DictionaryBlockInputStream")),
fillBlockFunction(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, StringGetterByKey>),
fill_block_function(&DictionaryBlockInputStream<DictionaryType, Key>::fillBlock<GetterByKey, StringGetterByKey>),
data_columns(data_columns),
get_key_columns_function(get_key_columns_function), get_view_columns_function(get_view_columns_function),
key_type(DictionaryKeyType::Callback)
{
}
template <typename DictionaryType, typename Key>
Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, size_t length) const
{
@ -190,13 +199,15 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, si
columns.emplace_back(column);
view_columns.emplace_back(column, key_column.type, key_column.name);
}
return (this->*fillBlockFunction)({}, columns, {}, std::move(view_columns));
return (this->*fill_block_function)({}, columns, {}, std::move(view_columns));
}
case DictionaryKeyType::Id:
{
PaddedPODArray<Key> block_ids(ids.begin() + start, ids.begin() + start + length);
return (this->*fillBlockFunction)(block_ids, {}, {}, {});
PaddedPODArray<Key> ids_to_fill(ids.begin() + start, ids.begin() + start + length);
return (this->*fill_block_function)(ids_to_fill, {}, {}, {});
}
case DictionaryKeyType::Callback:
{
Columns columns;
@ -214,12 +225,14 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t start, si
columns.push_back(key_column.column);
types.push_back(key_column.type);
}
return (this->*fillBlockFunction)({}, columns, types, std::move(view_with_type_and_name));
return (this->*fill_block_function)({}, columns, types, std::move(view_with_type_and_name));
}
}
throw Exception("Unexpected DictionaryKeyType.", ErrorCodes::LOGICAL_ERROR);
}
template <typename DictionaryType, typename Key>
template <typename Type, typename Container>
void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
@ -260,6 +273,7 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
(dict.*getter)(attribute.name, keys, data_types, container);
}
template <typename DictionaryType, typename Key>
template <template <typename> class Getter, typename StringGetter>
Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
@ -332,7 +346,7 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
case AttributeUnderlyingType::String:
{
column = getColumnFromStringAttribute<StringGetter>(
&DictionaryType::getString, ids, keys, data_types, attribute, *dictionary);
&DictionaryType::getString, ids_to_fill, keys, data_types, attribute, *dictionary);
break;
}
}
@ -343,6 +357,7 @@ Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
return Block(block_columns);
}
template <typename DictionaryType, typename Key>
template <typename AttributeType, typename Getter>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
@ -358,6 +373,7 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribut
return std::move(column_vector);
}
template <typename DictionaryType, typename Key>
template <typename Getter>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAttribute(
@ -371,6 +387,7 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAt
return std::move(column_string);
}
template <typename DictionaryType, typename Key>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(const PaddedPODArray<Key> & ids_to_fill) const
{
@ -381,6 +398,7 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(cons
return std::move(column_vector);
}
template <typename DictionaryType, typename Key>
void DictionaryBlockInputStream<DictionaryType, Key>::fillKeyColumns(
const std::vector<StringRef> & keys, size_t start, size_t size,

View File

@ -4,7 +4,7 @@ namespace DB
{
DictionaryBlockInputStreamBase::DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size)
: rows_count(rows_count), max_block_size(max_block_size), next_row(0)
: rows_count(rows_count), max_block_size(max_block_size)
{
}

View File

@ -1,4 +1,5 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
namespace DB
@ -7,8 +8,6 @@ namespace DB
class DictionaryBlockInputStreamBase : public IProfilingBlockInputStream
{
protected:
//Block block;
DictionaryBlockInputStreamBase(size_t rows_count, size_t max_block_size);
virtual Block getBlock(size_t start, size_t length) const = 0;

View File

@ -1,6 +1,7 @@
#pragma once
#include <Dictionaries/Embedded/GeodataProviders/Types.h>
#include <string>
struct RegionEntry
{

View File

@ -56,8 +56,8 @@ namespace
case Poco::MongoDB::ElementTraits<Int32>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().push_back(static_cast<const Poco::MongoDB::ConcreteElement<Int32> &>(value).value());
break;
case Poco::MongoDB::ElementTraits<Int64>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().push_back(static_cast<const Poco::MongoDB::ConcreteElement<Int64> &>(value).value());
case Poco::MongoDB::ElementTraits<Poco::Int64>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().push_back(static_cast<const Poco::MongoDB::ConcreteElement<Poco::Int64> &>(value).value());
break;
case Poco::MongoDB::ElementTraits<Float64>::TypeId:
static_cast<ColumnVector<T> &>(column).getData().push_back(static_cast<const Poco::MongoDB::ConcreteElement<Float64> &>(value).value());

View File

@ -22,10 +22,10 @@ template <typename DictionaryType, typename Key>
class RangeDictionaryBlockInputStream : public DictionaryBlockInputStreamBase
{
public:
using DictionatyPtr = std::shared_ptr<DictionaryType const>;
using DictionaryPtr = std::shared_ptr<DictionaryType const>;
RangeDictionaryBlockInputStream(
DictionatyPtr dictionary, size_t max_block_size, const Names & column_names, PaddedPODArray<Key> && ids,
DictionaryPtr dictionary, size_t max_block_size, const Names & column_names, PaddedPODArray<Key> && ids,
PaddedPODArray<UInt16> && start_dates, PaddedPODArray<UInt16> && end_dates);
String getName() const override
@ -43,9 +43,9 @@ private:
template <typename AttributeType>
ColumnPtr getColumnFromAttribute(DictionaryGetter<AttributeType> getter,
const PaddedPODArray<Key> & ids, const PaddedPODArray<UInt16> & dates,
const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
ColumnPtr getColumnFromAttributeString(const PaddedPODArray<Key> & ids, const PaddedPODArray<UInt16> & dates,
ColumnPtr getColumnFromAttributeString(const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const;
template <typename T>
ColumnPtr getColumnFromPODArray(const PaddedPODArray<T> & array) const;
@ -56,13 +56,13 @@ private:
const std::string & default_name, const std::unordered_set<std::string> & column_names,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const;
Block fillBlock(const PaddedPODArray<Key> & ids,
Block fillBlock(const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const;
PaddedPODArray<UInt16> makeDateKey(
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const;
DictionatyPtr dictionary;
DictionaryPtr dictionary;
Names column_names;
PaddedPODArray<Key> ids;
PaddedPODArray<UInt16> start_dates;
@ -72,7 +72,7 @@ private:
template <typename DictionaryType, typename Key>
RangeDictionaryBlockInputStream<DictionaryType, Key>::RangeDictionaryBlockInputStream(
DictionatyPtr dictionary, size_t max_column_size, const Names & column_names, PaddedPODArray<Key> && ids,
DictionaryPtr dictionary, size_t max_column_size, const Names & column_names, PaddedPODArray<Key> && ids,
PaddedPODArray<UInt16> && start_dates, PaddedPODArray<UInt16> && end_dates)
: DictionaryBlockInputStreamBase(ids.size(), max_column_size),
dictionary(dictionary), column_names(column_names),
@ -103,21 +103,21 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t star
template <typename DictionaryType, typename Key>
template <typename AttributeType>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
DictionaryGetter<AttributeType> getter, const PaddedPODArray<Key> & ids,
DictionaryGetter<AttributeType> getter, const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<UInt16> & dates, const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
{
auto column_vector = ColumnVector<AttributeType>::create(ids.size());
(dictionary.*getter)(attribute.name, ids, dates, column_vector->getData());
auto column_vector = ColumnVector<AttributeType>::create(ids_to_fill.size());
(dictionary.*getter)(attribute.name, ids_to_fill, dates, column_vector->getData());
return std::move(column_vector);
}
template <typename DictionaryType, typename Key>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttributeString(
const PaddedPODArray<Key> & ids, const PaddedPODArray<UInt16> & dates,
const PaddedPODArray<Key> & ids_to_fill, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute & attribute, const DictionaryType & dictionary) const
{
auto column_string = ColumnString::create();
dictionary.getString(attribute.name, ids, dates, column_string.get());
dictionary.getString(attribute.name, ids_to_fill, dates, column_string.get());
return std::move(column_string);
}
@ -167,7 +167,7 @@ PaddedPODArray<UInt16> RangeDictionaryBlockInputStream<DictionaryType, Key>::mak
template <typename DictionaryType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const PaddedPODArray<Key> & ids,
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const
{
ColumnsWithTypeAndName columns;
@ -175,7 +175,7 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
std::unordered_set<std::string> names(column_names.begin(), column_names.end());
addSpecialColumn(structure.id, std::make_shared<DataTypeUInt64>(), "ID", names, ids, columns);
addSpecialColumn(structure.id, std::make_shared<DataTypeUInt64>(), "ID", names, ids_to_fill, columns);
addSpecialColumn(structure.range_min, std::make_shared<DataTypeDate>(), "Range Start", names, start_dates, columns);
addSpecialColumn(structure.range_max, std::make_shared<DataTypeDate>(), "Range End", names, end_dates, columns);
@ -188,7 +188,7 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
{
ColumnPtr column;
#define GET_COLUMN_FORM_ATTRIBUTE(TYPE)\
column = getColumnFromAttribute<TYPE>(&DictionaryType::get##TYPE, ids, date_key, attribute, *dictionary)
column = getColumnFromAttribute<TYPE>(&DictionaryType::get##TYPE, ids_to_fill, date_key, attribute, *dictionary)
switch (attribute.underlying_type)
{
case AttributeUnderlyingType::UInt8:
@ -225,7 +225,7 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
GET_COLUMN_FORM_ATTRIBUTE(Float64);
break;
case AttributeUnderlyingType::String:
column = getColumnFromAttributeString(ids, date_key, attribute, *dictionary);
column = getColumnFromAttributeString(ids_to_fill, date_key, attribute, *dictionary);
break;
}

View File

@ -945,14 +945,13 @@ void FunctionArrayEnumerate::executeImpl(Block & block, const ColumnNumbers & ar
ColumnUInt32::Container & res_values = res_nested->getData();
res_values.resize(array->getData().size());
size_t prev_off = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
ColumnArray::Offset prev_off = 0;
for (ColumnArray::Offset i = 0; i < offsets.size(); ++i)
{
ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j)
res_values[j] = j - prev_off + 1;
}
prev_off = off;
}
@ -1101,13 +1100,13 @@ bool FunctionArrayUniq::executeNumber(const ColumnArray * array, const IColumn *
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
Set set;
size_t prev_off = 0;
ColumnArray::Offset prev_off = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
set.clear();
bool found_null = false;
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j)
{
if (null_map_data && ((*null_map_data)[j] == 1))
found_null = true;
@ -1147,13 +1146,13 @@ bool FunctionArrayUniq::executeString(const ColumnArray * array, const IColumn *
null_map_data = &static_cast<const ColumnUInt8 *>(null_map)->getData();
Set set;
size_t prev_off = 0;
ColumnArray::Offset prev_off = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
set.clear();
bool found_null = false;
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j)
{
if (null_map_data && ((*null_map_data)[j] == 1))
found_null = true;
@ -1209,26 +1208,26 @@ bool FunctionArrayUniq::execute128bit(
/// Each binary blob is inserted into a hash table.
///
Set set;
size_t prev_off = 0;
for (size_t i = 0; i < offsets.size(); ++i)
ColumnArray::Offset prev_off = 0;
for (ColumnArray::Offset i = 0; i < offsets.size(); ++i)
{
set.clear();
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j)
{
if (has_nullable_columns)
{
KeysNullMap<UInt128> bitmap{};
for (size_t i = 0; i < columns.size(); ++i)
for (ColumnArray::Offset i = 0; i < columns.size(); ++i)
{
if (null_maps[i])
{
const auto & null_map = static_cast<const ColumnUInt8 &>(*null_maps[i]).getData();
if (null_map[j] == 1)
{
size_t bucket = i / 8;
size_t offset = i % 8;
ColumnArray::Offset bucket = i / 8;
ColumnArray::Offset offset = i % 8;
bitmap[bucket] |= UInt8(1) << offset;
}
}
@ -1257,12 +1256,12 @@ void FunctionArrayUniq::executeHashed(
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
Set set;
size_t prev_off = 0;
for (size_t i = 0; i < offsets.size(); ++i)
ColumnArray::Offset prev_off = 0;
for (ColumnArray::Offset i = 0; i < offsets.size(); ++i)
{
set.clear();
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
ColumnArray::Offset off = offsets[i];
for (ColumnArray::Offset j = prev_off; j < off; ++j)
set.insert(hash128(j, count, columns));
res_values[i] = set.size();
@ -1308,9 +1307,6 @@ void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arg
const IColumn & src_data = array->getData();
const ColumnArray::Offsets & offsets = array->getOffsets();
ColumnRawPtrs original_data_columns;
original_data_columns.push_back(&src_data);
IColumn & res_data = res.getData();
ColumnArray::Offsets & res_offsets = res.getOffsets();
@ -1339,13 +1335,14 @@ void FunctionArrayDistinct::executeImpl(Block & block, const ColumnNumbers & arg
|| executeNumber<Float32>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|| executeNumber<Float64>(*inner_col, offsets, res_data, res_offsets, nullable_col)
|| executeString(*inner_col, offsets, res_data, res_offsets, nullable_col)))
executeHashed(offsets, original_data_columns, res_data, res_offsets, nullable_col);
executeHashed(*inner_col, offsets, res_data, res_offsets, nullable_col);
block.getByPosition(result).column = std::move(res_ptr);
}
template <typename T>
bool FunctionArrayDistinct::executeNumber(const IColumn & src_data,
bool FunctionArrayDistinct::executeNumber(
const IColumn & src_data,
const ColumnArray::Offsets & src_offsets,
IColumn & res_data_col,
ColumnArray::Offsets & res_offsets,
@ -1364,9 +1361,7 @@ bool FunctionArrayDistinct::executeNumber(const IColumn & src_data,
const PaddedPODArray<UInt8> * src_null_map = nullptr;
if (nullable_col)
{
src_null_map = &static_cast<const ColumnUInt8 *>(&nullable_col->getNullMapColumn())->getData();
}
using Set = ClearableHashSet<T,
DefaultHash<T>,
@ -1374,22 +1369,31 @@ bool FunctionArrayDistinct::executeNumber(const IColumn & src_data,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>;
Set set;
size_t prev_off = 0;
for (size_t i = 0; i < src_offsets.size(); ++i)
ColumnArray::Offset prev_src_offset = 0;
ColumnArray::Offset res_offset = 0;
for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i)
{
set.clear();
size_t off = src_offsets[i];
for (size_t j = prev_off; j < off; ++j)
ColumnArray::Offset curr_src_offset = src_offsets[i];
for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j)
{
if ((set.find(values[j]) == set.end()) && (!nullable_col || (*src_null_map)[j] == 0))
if (nullable_col && (*src_null_map)[j])
continue;
if (set.find(values[j]) == set.end())
{
res_data.emplace_back(values[j]);
set.insert(values[j]);
}
}
res_offsets.emplace_back(set.size() + prev_off);
prev_off = off;
res_offset += set.size();
res_offsets.emplace_back(res_offset);
prev_src_offset = curr_src_offset;
}
return true;
}
@ -1404,9 +1408,7 @@ bool FunctionArrayDistinct::executeString(
const ColumnString * src_data_concrete = checkAndGetColumn<ColumnString>(&src_data);
if (!src_data_concrete)
{
return false;
}
ColumnString & res_data_column_string = typeid_cast<ColumnString &>(res_data_col);
@ -1418,70 +1420,86 @@ bool FunctionArrayDistinct::executeString(
const PaddedPODArray<UInt8> * src_null_map = nullptr;
if (nullable_col)
{
src_null_map = &static_cast<const ColumnUInt8 *>(&nullable_col->getNullMapColumn())->getData();
}
Set set;
size_t prev_off = 0;
for (size_t i = 0; i < src_offsets.size(); ++i)
ColumnArray::Offset prev_src_offset = 0;
ColumnArray::Offset res_offset = 0;
for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i)
{
set.clear();
size_t off = src_offsets[i];
for (size_t j = prev_off; j < off; ++j)
ColumnArray::Offset curr_src_offset = src_offsets[i];
for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j)
{
if (nullable_col && (*src_null_map)[j])
continue;
StringRef str_ref = src_data_concrete->getDataAt(j);
if (set.find(str_ref) == set.end() && (!nullable_col || (*src_null_map)[j] == 0))
if (set.find(str_ref) == set.end())
{
set.insert(str_ref);
res_data_column_string.insertData(str_ref.data, str_ref.size);
}
}
res_offsets.emplace_back(set.size() + prev_off);
prev_off = off;
res_offset += set.size();
res_offsets.emplace_back(res_offset);
prev_src_offset = curr_src_offset;
}
return true;
}
void FunctionArrayDistinct::executeHashed(
const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns,
const IColumn & src_data,
const ColumnArray::Offsets & src_offsets,
IColumn & res_data_col,
ColumnArray::Offsets & res_offsets,
const ColumnNullable * nullable_col)
{
size_t count = columns.size();
using Set = ClearableHashSet<UInt128, UInt128TrivialHash, HashTableGrower<INITIAL_SIZE_DEGREE>,
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
const PaddedPODArray<UInt8> * src_null_map = nullptr;
if (nullable_col)
{
src_null_map = &static_cast<const ColumnUInt8 *>(&nullable_col->getNullMapColumn())->getData();
}
Set set;
size_t prev_off = 0;
for (size_t i = 0; i < offsets.size(); ++i)
ColumnArray::Offset prev_src_offset = 0;
ColumnArray::Offset res_offset = 0;
for (ColumnArray::Offset i = 0; i < src_offsets.size(); ++i)
{
set.clear();
size_t off = offsets[i];
for (size_t j = prev_off; j < off; ++j)
ColumnArray::Offset curr_src_offset = src_offsets[i];
for (ColumnArray::Offset j = prev_src_offset; j < curr_src_offset; ++j)
{
auto hash = hash128(j, count, columns);
if (set.find(hash) == set.end() && (!nullable_col || (*src_null_map)[j] == 0))
if (nullable_col && (*src_null_map)[j])
continue;
UInt128 hash;
SipHash hash_function;
src_data.updateHashWithValue(j, hash_function);
hash_function.get128(reinterpret_cast<char *>(&hash));
if (set.find(hash) == set.end())
{
set.insert(hash);
res_data_col.insertFrom(*columns[0], j);
res_data_col.insertFrom(src_data, j);
}
}
res_offsets.emplace_back(set.size() + prev_off);
prev_off = off;
res_offset += set.size();
res_offsets.emplace_back(res_offset);
prev_src_offset = curr_src_offset;
}
}

View File

@ -1252,8 +1252,8 @@ private:
const ColumnNullable * nullable_col);
void executeHashed(
const ColumnArray::Offsets & offsets,
const ColumnRawPtrs & columns,
const IColumn & src_data,
const ColumnArray::Offsets & src_offsets,
IColumn & res_data_col,
ColumnArray::Offsets & res_offsets,
const ColumnNullable * nullable_col);

View File

@ -72,7 +72,7 @@ struct HalfMD5Impl
union
{
unsigned char char_data[16];
Poco::UInt64 uint64_data;
uint64_t uint64_data;
} buf;
MD5_CTX ctx;

View File

@ -108,7 +108,8 @@ private:
inner,
outer,
singleLine,
pairOfLinesSinglePolygon,
pairOfLinesSingleConvexPolygon,
pairOfLinesSingleNonConvexPolygons,
pairOfLinesDifferentPolygons,
complexPolygon
};
@ -180,6 +181,9 @@ private:
/// Returns a list of half-planes were formed from intersection edges without box edges.
inline std::vector<HalfPlane> findHalfPlanes(const Box & box, const Polygon & intersection);
/// Check that polygon.outer() is convex.
inline bool isConvex(const Polygon & polygon);
using Distance = typename boost::geometry::default_comparable_distance_result<Point, Segment>::type;
/// min(distance(point, edge) : edge in polygon)
@ -306,9 +310,10 @@ bool PointInPolygonWithGrid<CoordinateType>::contains(CoordinateType x, Coordina
return false;
case CellType::singleLine:
return cell.half_planes[0].contains(x, y);
case CellType::pairOfLinesSinglePolygon:
case CellType::pairOfLinesSingleConvexPolygon:
return cell.half_planes[0].contains(x, y) && cell.half_planes[1].contains(x, y);
case CellType::pairOfLinesDifferentPolygons:
case CellType::pairOfLinesSingleNonConvexPolygons:
return cell.half_planes[0].contains(x, y) || cell.half_planes[1].contains(x, y);
case CellType::complexPolygon:
return boost::geometry::within(Point(x, y), polygons[cell.index_of_inner_polygon]);
@ -335,6 +340,35 @@ PointInPolygonWithGrid<CoordinateType>::distance(
return distance;
}
template <typename CoordinateType>
bool PointInPolygonWithGrid<CoordinateType>::isConvex(const PointInPolygonWithGrid<CoordinateType>::Polygon & polygon)
{
const auto & outer = polygon.outer();
/// Segment or point.
if (outer.size() < 4)
return false;
auto vecProduct = [](const Point & from, const Point & to) { return from.x() * to.y() - from.y() * to.x(); };
auto getVector = [](const Point & from, const Point & to) -> Point
{
return Point(to.x() - from.x(), to.y() - from.y());
};
Point first = getVector(outer[0], outer[1]);
Point prev = first;
for (auto i : ext::range(1, outer.size() - 1))
{
Point cur = getVector(outer[i], outer[i + 1]);
if (vecProduct(prev, cur) < 0)
return false;
prev = cur;
}
return vecProduct(prev, first) >= 0;
}
template <typename CoordinateType>
std::vector<typename PointInPolygonWithGrid<CoordinateType>::HalfPlane>
PointInPolygonWithGrid<CoordinateType>::findHalfPlanes(
@ -423,7 +457,8 @@ void PointInPolygonWithGrid<CoordinateType>::addCell(
}
else if (half_planes.size() == 2)
{
cells[index].type = CellType::pairOfLinesSinglePolygon;
cells[index].type = isConvex(intersection) ? CellType::pairOfLinesSingleConvexPolygon
: CellType::pairOfLinesSingleNonConvexPolygons;
cells[index].half_planes[0] = half_planes[0];
cells[index].half_planes[1] = half_planes[1];
}

View File

@ -234,8 +234,8 @@ void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, co
executeImpl(block, args, result, input_rows_count);
}
static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(Block & block, const ColumnNumbers & args,
bool can_be_executed_on_default_arguments)
static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(
Block & block, const ColumnNumbers & args, bool can_be_executed_on_default_arguments)
{
size_t num_rows = 0;
ColumnPtr indexes;

View File

@ -1,4 +1,4 @@
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#if defined(__linux__)
#include <boost/noncopyable.hpp>
#include <Common/Exception.h>

View File

@ -1,6 +1,8 @@
#pragma once
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#if defined(__linux__)
#include <boost/noncopyable.hpp>
/// https://stackoverflow.com/questions/20759750/resolving-redefinition-of-timespec-in-time-h
#define timespec linux_timespec

View File

@ -1,4 +1,4 @@
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#if defined(__linux__)
#include <Common/Exception.h>
#include <common/logger_useful.h>

View File

@ -1,6 +1,6 @@
#pragma once
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#if defined(__linux__)
#include <ext/singleton.h>
#include <condition_variable>

View File

@ -25,8 +25,7 @@ private:
/// Swap the main and duplicate buffers.
void swapBuffers()
{
buffer().swap(out.buffer());
std::swap(position(), out.position());
swap(out);
}
void nextImpl() override

View File

@ -80,6 +80,13 @@ public:
/// How many bytes are available for read/write
inline size_t available() const { return size_t(working_buffer.end() - pos); }
inline void swap(BufferBase & other)
{
internal_buffer.swap(other.internal_buffer);
working_buffer.swap(other.working_buffer);
std::swap(pos, other.pos);
}
/** How many bytes have been read/written, counting those that are still in the buffer. */
size_t count() const
{

View File

@ -13,14 +13,14 @@ class LimitReadBuffer : public ReadBuffer
{
private:
ReadBuffer & in;
UInt64 limit;
size_t limit;
bool throw_exception;
std::string exception_message;
bool nextImpl() override;
public:
LimitReadBuffer(ReadBuffer & in, UInt64 limit, bool throw_exception, std::string exception_message = {});
LimitReadBuffer(ReadBuffer & in, size_t limit, bool throw_exception, std::string exception_message = {});
~LimitReadBuffer() override;
};

View File

@ -1,4 +1,4 @@
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#if defined(__linux__)
#include <IO/ReadBufferAIO.h>
#include <IO/AIOContextPool.h>
@ -293,9 +293,7 @@ void ReadBufferAIO::finalize()
is_eof = true;
/// Swap the main and duplicate buffers.
internalBuffer().swap(fill_buffer.internalBuffer());
buffer().swap(fill_buffer.buffer());
std::swap(position(), fill_buffer.position());
swap(fill_buffer);
}
}

View File

@ -1,6 +1,6 @@
#pragma once
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#if defined(__linux__)
#include <IO/ReadBufferFromFileBase.h>
#include <IO/ReadBuffer.h>

View File

@ -67,8 +67,9 @@ bool ReadBufferFromFileDescriptor::nextImpl()
if (res > 0)
bytes_read += res;
/// NOTE: it is quite inaccurate on high loads since the thread could be replaced by another one and we will count cpu time of other thread
/// It is better to use taskstats::blkio_delay_total, but it is quite expensive to get it (TaskStatsInfoGetter has about 500K RPS)
/// It reports real time spent including the time spent while thread was preempted doing nothing.
/// And it is Ok for the purpose of this watch (it is used to lower the number of threads to read from tables).
/// Sometimes it is better to use taskstats::blkio_delay_total, but it is quite expensive to get it (TaskStatsInfoGetter has about 500K RPS).
watch.stop();
ProfileEvents::increment(ProfileEvents::DiskReadElapsedMicroseconds, watch.elapsedMicroseconds());

View File

@ -175,10 +175,10 @@ void readStringInto(Vector & s, ReadBuffer & buf)
{
while (!buf.eof())
{
const char * next_pos = find_first_symbols<'\t', '\n'>(buf.position(), buf.buffer().end());
char * next_pos = find_first_symbols<'\t', '\n'>(buf.position(), buf.buffer().end());
appendToStringOrVector(s, buf.position(), next_pos);
buf.position() += next_pos - buf.position(); /// Code looks complicated, because "buf.position() = next_pos" doens't work due to const-ness.
buf.position() = next_pos;
if (buf.hasPendingData())
return;
@ -199,10 +199,8 @@ void readStringUntilEOFInto(Vector & s, ReadBuffer & buf)
{
while (!buf.eof())
{
size_t bytes = buf.buffer().end() - buf.position();
appendToStringOrVector(s, buf.position(), buf.position() + bytes);
buf.position() += bytes;
appendToStringOrVector(s, buf.position(), buf.buffer().end());
buf.position() = buf.buffer().end();
if (buf.hasPendingData())
return;
@ -370,10 +368,10 @@ void readEscapedStringInto(Vector & s, ReadBuffer & buf)
{
while (!buf.eof())
{
const char * next_pos = find_first_symbols<'\t', '\n', '\\'>(buf.position(), buf.buffer().end());
char * next_pos = find_first_symbols<'\t', '\n', '\\'>(buf.position(), buf.buffer().end());
appendToStringOrVector(s, buf.position(), next_pos);
buf.position() += next_pos - buf.position(); /// Code looks complicated, because "buf.position() = next_pos" doens't work due to const-ness.
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;
@ -412,10 +410,10 @@ static void readAnyQuotedStringInto(Vector & s, ReadBuffer & buf)
while (!buf.eof())
{
const char * next_pos = find_first_symbols<'\\', quote>(buf.position(), buf.buffer().end());
char * next_pos = find_first_symbols<'\\', quote>(buf.position(), buf.buffer().end());
appendToStringOrVector(s, buf.position(), next_pos);
buf.position() += next_pos - buf.position();
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;
@ -522,13 +520,13 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
/// The quoted case. We are looking for the next quotation mark.
while (!buf.eof())
{
const char * next_pos = reinterpret_cast<const char *>(memchr(buf.position(), maybe_quote, buf.buffer().end() - buf.position()));
char * next_pos = reinterpret_cast<char *>(memchr(buf.position(), maybe_quote, buf.buffer().end() - buf.position()));
if (nullptr == next_pos)
next_pos = buf.buffer().end();
appendToStringOrVector(s, buf.position(), next_pos);
buf.position() += next_pos - buf.position();
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;
@ -553,13 +551,13 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
/// Unquoted case. Look for delimiter or \r or \n.
while (!buf.eof())
{
const char * next_pos = buf.position();
char * next_pos = buf.position();
while (next_pos < buf.buffer().end()
&& *next_pos != delimiter && *next_pos != '\r' && *next_pos != '\n') /// NOTE You can make a SIMD version.
++next_pos;
appendToStringOrVector(s, buf.position(), next_pos);
buf.position() += next_pos - buf.position();
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;
@ -606,10 +604,10 @@ ReturnType readJSONStringInto(Vector & s, ReadBuffer & buf)
while (!buf.eof())
{
const char * next_pos = find_first_symbols<'\\', '"'>(buf.position(), buf.buffer().end());
char * next_pos = find_first_symbols<'\\', '"'>(buf.position(), buf.buffer().end());
appendToStringOrVector(s, buf.position(), next_pos);
buf.position() += next_pos - buf.position();
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;
@ -870,8 +868,10 @@ void readException(Exception & e, ReadBuffer & buf, const String & additional_me
if (name != "DB::Exception")
out << name << ". ";
out << message
<< ". Stack trace:\n\n" << stack_trace;
out << message << ".";
if (!stack_trace.empty())
out << " Stack trace:\n\n" << stack_trace;
if (has_nested)
{
@ -895,8 +895,8 @@ void skipToNextLineOrEOF(ReadBuffer & buf)
{
while (!buf.eof())
{
const char * next_pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end());
buf.position() += next_pos - buf.position();
char * next_pos = find_first_symbols<'\n'>(buf.position(), buf.buffer().end());
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;
@ -914,8 +914,8 @@ void skipToUnescapedNextLineOrEOF(ReadBuffer & buf)
{
while (!buf.eof())
{
const char * next_pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end());
buf.position() += next_pos - buf.position();
char * next_pos = find_first_symbols<'\n', '\\'>(buf.position(), buf.buffer().end());
buf.position() = next_pos;
if (!buf.hasPendingData())
continue;

View File

@ -118,11 +118,13 @@ inline void throwReadAfterEOF()
throw Exception("Attempt to read after eof", ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF);
}
inline void readVarUInt(UInt64 & x, ReadBuffer & istr)
template <bool fast>
inline void readVarUIntImpl(UInt64 & x, ReadBuffer & istr)
{
x = 0;
for (size_t i = 0; i < 9; ++i)
{
if constexpr (!fast)
if (istr.eof())
throwReadAfterEOF();
@ -135,6 +137,13 @@ inline void readVarUInt(UInt64 & x, ReadBuffer & istr)
}
}
inline void readVarUInt(UInt64 & x, ReadBuffer & istr)
{
if (istr.buffer().end() - istr.position() >= 9)
return readVarUIntImpl<true>(x, istr);
return readVarUIntImpl<false>(x, istr);
}
inline void readVarUInt(UInt64 & x, std::istream & istr)
{

View File

@ -1,4 +1,4 @@
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#if defined(__linux__)
#include <IO/WriteBufferAIO.h>
#include <Common/ProfileEvents.h>
@ -206,8 +206,7 @@ bool WriteBufferAIO::waitForAIOCompletion()
void WriteBufferAIO::prepare()
{
/// Swap the main and duplicate buffers.
buffer().swap(flush_buffer.buffer());
std::swap(position(), flush_buffer.position());
swap(flush_buffer);
truncation_count = 0;

View File

@ -1,6 +1,6 @@
#pragma once
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#if defined(__linux__)
#include <IO/WriteBufferFromFileBase.h>
#include <IO/WriteBuffer.h>

View File

@ -49,18 +49,22 @@ void formatUUID(std::reverse_iterator<const UInt8 *> src16, UInt8 * dst36)
void writeException(const Exception & e, WriteBuffer & buf)
void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trace)
{
writeBinary(e.code(), buf);
writeBinary(String(e.name()), buf);
writeBinary(e.displayText(), buf);
if (with_stack_trace)
writeBinary(e.getStackTrace().toString(), buf);
else
writeBinary(String(), buf);
bool has_nested = e.nested() != nullptr;
writeBinary(has_nested, buf);
if (has_nested)
writeException(Exception(*e.nested()), buf);
writeException(Exception(*e.nested()), buf, with_stack_trace);
}
}

View File

@ -823,7 +823,7 @@ void writeText(const std::vector<T> & x, WriteBuffer & buf)
/// Serialize exception (so that it can be transferred over the network)
void writeException(const Exception & e, WriteBuffer & buf);
void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trace);
/// An easy-to-use method for converting something to a string in text form.

View File

@ -1,6 +1,6 @@
#include <IO/createReadBufferFromFileBase.h>
#include <IO/ReadBufferFromFile.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER)
#if defined(__linux__)
#include <IO/ReadBufferAIO.h>
#endif
#include <Common/ProfileEvents.h>
@ -14,7 +14,7 @@ namespace ProfileEvents
namespace DB
{
#if defined(__APPLE__) || defined(__FreeBSD__) || defined(_MSC_VER)
#if !defined(__linux__)
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
@ -31,7 +31,7 @@ std::unique_ptr<ReadBufferFromFileBase> createReadBufferFromFileBase(const std::
}
else
{
#if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER)
#if defined(__linux__)
ProfileEvents::increment(ProfileEvents::CreatedReadBufferAIO);
return std::make_unique<ReadBufferAIO>(filename_, buffer_size_, flags_, existing_memory_);
#else

View File

@ -1,6 +1,6 @@
#include <IO/createWriteBufferFromFileBase.h>
#include <IO/WriteBufferFromFile.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER)
#if defined(__linux__)
#include <IO/WriteBufferAIO.h>
#endif
#include <Common/ProfileEvents.h>
@ -15,7 +15,7 @@ namespace ProfileEvents
namespace DB
{
#if defined(__APPLE__) || defined(__FreeBSD__) || defined(_MSC_VER)
#if !defined(__linux__)
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
@ -33,7 +33,7 @@ WriteBufferFromFileBase * createWriteBufferFromFileBase(const std::string & file
}
else
{
#if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER)
#if defined(__linux__)
ProfileEvents::increment(ProfileEvents::CreatedWriteBufferAIO);
return new WriteBufferAIO(filename_, buffer_size_, flags_, mode, existing_memory_);
#else

View File

@ -632,7 +632,7 @@ void Context::checkDatabaseAccessRightsImpl(const std::string & database_name) c
return;
}
if (!shared->security_manager->hasAccessToDatabase(client_info.current_user, database_name))
throw Exception("Access denied to database " + database_name, ErrorCodes::DATABASE_ACCESS_DENIED);
throw Exception("Access denied to database " + database_name + " for user " + client_info.current_user , ErrorCodes::DATABASE_ACCESS_DENIED);
}
void Context::addDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where)

View File

@ -545,19 +545,19 @@ void DDLWorker::processTask(DDLTask & task)
String finished_node_path = task.entry_path + "/finished/" + task.host_id_str;
auto code = zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy);
if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
if (code == Coordination::ZOK || code == Coordination::ZNODEEXISTS)
{
// Ok
}
else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
else if (code == Coordination::ZNONODE)
{
/// There is no parent
createStatusDirs(task.entry_path);
if (ZooKeeperImpl::ZooKeeper::ZOK != zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy))
throw zkutil::KeeperException(code, active_node_path);
if (Coordination::ZOK != zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy))
throw Coordination::Exception(code, active_node_path);
}
else
throw zkutil::KeeperException(code, active_node_path);
throw Coordination::Exception(code, active_node_path);
if (!task.was_executed)
{
@ -578,7 +578,7 @@ void DDLWorker::processTask(DDLTask & task)
tryExecuteQuery(rewritten_query, task, task.execution_status);
}
}
catch (const zkutil::KeeperException &)
catch (const Coordination::Exception &)
{
throw;
}
@ -594,7 +594,7 @@ 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::Requests ops;
Coordination::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);
@ -734,7 +734,7 @@ void DDLWorker::cleanupQueue()
String node_path = queue_dir + "/" + node_name;
String lock_path = node_path + "/lock";
zkutil::Stat stat;
Coordination::Stat stat;
String dummy;
try
@ -784,7 +784,7 @@ void DDLWorker::cleanupQueue()
}
/// Remove the lock node and its parent atomically
zkutil::Requests ops;
Coordination::Requests ops;
ops.emplace_back(zkutil::makeRemoveRequest(lock_path, -1));
ops.emplace_back(zkutil::makeRemoveRequest(node_path, -1));
zookeeper->multi(ops);
@ -803,21 +803,21 @@ void DDLWorker::cleanupQueue()
/// Try to create nonexisting "status" dirs for a node
void DDLWorker::createStatusDirs(const std::string & node_path)
{
zkutil::Requests ops;
Coordination::Requests ops;
{
zkutil::CreateRequest request;
Coordination::CreateRequest request;
request.path = node_path + "/active";
ops.emplace_back(std::make_shared<zkutil::CreateRequest>(std::move(request)));
ops.emplace_back(std::make_shared<Coordination::CreateRequest>(std::move(request)));
}
{
zkutil::CreateRequest request;
Coordination::CreateRequest request;
request.path = node_path + "/finished";
ops.emplace_back(std::make_shared<zkutil::CreateRequest>(std::move(request)));
ops.emplace_back(std::make_shared<Coordination::CreateRequest>(std::move(request)));
}
zkutil::Responses responses;
Coordination::Responses responses;
int code = zookeeper->tryMulti(ops, responses);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
throw zkutil::KeeperException(code);
if (code && code != Coordination::ZNODEEXISTS)
throw Coordination::Exception(code);
}
@ -862,9 +862,9 @@ void DDLWorker::run()
zookeeper->createAncestors(queue_dir + "/");
initialized = true;
}
catch (const zkutil::KeeperException & e)
catch (const Coordination::Exception & e)
{
if (!ZooKeeperImpl::ZooKeeper::isHardwareError(e.code))
if (!Coordination::isHardwareError(e.code))
throw;
}
}
@ -890,9 +890,9 @@ void DDLWorker::run()
/// TODO: it might delay the execution, move it to separate thread.
cleanupQueue();
}
catch (zkutil::KeeperException & e)
catch (Coordination::Exception & e)
{
if (ZooKeeperImpl::ZooKeeper::isHardwareError(e.code))
if (Coordination::isHardwareError(e.code))
{
LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false));
@ -1066,8 +1066,8 @@ private:
{
Strings res;
int code = zookeeper->tryGetChildren(node_path, res);
if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE)
throw zkutil::KeeperException(code, node_path);
if (code && code != Coordination::ZNONODE)
throw Coordination::Exception(code, node_path);
return res;
}

View File

@ -66,6 +66,7 @@
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Interpreters/evaluateQualified.h>
namespace DB
@ -164,35 +165,6 @@ void removeDuplicateColumns(NamesAndTypesList & columns)
}
String DatabaseAndTableWithAlias::getQualifiedNamePrefix() const
{
return (!alias.empty() ? alias : (database + '.' + table)) + '.';
}
void DatabaseAndTableWithAlias::makeQualifiedName(const ASTPtr & ast) const
{
if (auto identifier = typeid_cast<ASTIdentifier *>(ast.get()))
{
String prefix = getQualifiedNamePrefix();
identifier->name.insert(identifier->name.begin(), prefix.begin(), prefix.end());
Names qualifiers;
if (!alias.empty())
qualifiers.push_back(alias);
else
{
qualifiers.push_back(database);
qualifiers.push_back(table);
}
for (const auto & qualifier : qualifiers)
identifier->children.emplace_back(std::make_shared<ASTIdentifier>(qualifier));
}
}
ExpressionAnalyzer::ExpressionAnalyzer(
const ASTPtr & ast_,
const Context & context_,
@ -274,7 +246,7 @@ ExpressionAnalyzer::ExpressionAnalyzer(
getArrayJoinedColumns();
/// Push the predicate expression down to the subqueries.
rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings).optimize();
rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
/// Delete the unnecessary from `source_columns` list. Create `unknown_required_source_columns`. Form `columns_added_by_join`.
collectUsedColumns();
@ -293,46 +265,6 @@ ExpressionAnalyzer::ExpressionAnalyzer(
analyzeAggregation();
}
static DatabaseAndTableWithAlias getTableNameWithAliasFromTableExpression(const ASTTableExpression & table_expression,
const Context & context)
{
DatabaseAndTableWithAlias database_and_table_with_alias;
if (table_expression.database_and_table_name)
{
const auto & identifier = static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name);
database_and_table_with_alias.alias = identifier.tryGetAlias();
if (table_expression.database_and_table_name->children.empty())
{
database_and_table_with_alias.database = context.getCurrentDatabase();
database_and_table_with_alias.table = identifier.name;
}
else
{
if (table_expression.database_and_table_name->children.size() != 2)
throw Exception("Logical error: number of components in table expression not equal to two", ErrorCodes::LOGICAL_ERROR);
database_and_table_with_alias.database = static_cast<const ASTIdentifier &>(*identifier.children[0]).name;
database_and_table_with_alias.table = static_cast<const ASTIdentifier &>(*identifier.children[1]).name;
}
}
else if (table_expression.table_function)
{
database_and_table_with_alias.alias = table_expression.table_function->tryGetAlias();
}
else if (table_expression.subquery)
{
database_and_table_with_alias.alias = table_expression.subquery->tryGetAlias();
}
else
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
return database_and_table_with_alias;
}
void ExpressionAnalyzer::translateQualifiedNames()
{
if (!select_query || !select_query->tables || select_query->tables->children.empty())
@ -357,80 +289,6 @@ void ExpressionAnalyzer::translateQualifiedNames()
translateQualifiedNamesImpl(ast, tables);
}
/// Get the number of components of identifier which are correspond to 'alias.', 'table.' or 'databas.table.' from names.
static size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier,
const DatabaseAndTableWithAlias & names)
{
size_t num_qualifiers_to_strip = 0;
auto get_identifier_name = [](const ASTPtr & ast) { return static_cast<const ASTIdentifier &>(*ast).name; };
/// It is compound identifier
if (!identifier.children.empty())
{
size_t num_components = identifier.children.size();
/// database.table.column
if (num_components >= 3
&& !names.database.empty()
&& get_identifier_name(identifier.children[0]) == names.database
&& get_identifier_name(identifier.children[1]) == names.table)
{
num_qualifiers_to_strip = 2;
}
/// table.column or alias.column. If num_components > 2, it is like table.nested.column.
if (num_components >= 2
&& ((!names.table.empty() && get_identifier_name(identifier.children[0]) == names.table)
|| (!names.alias.empty() && get_identifier_name(identifier.children[0]) == names.alias)))
{
num_qualifiers_to_strip = 1;
}
}
return num_qualifiers_to_strip;
}
/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left.
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'.
static void stripIdentifier(ASTPtr & ast, size_t num_qualifiers_to_strip)
{
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(ast.get());
if (!identifier)
throw Exception("ASTIdentifier expected for stripIdentifier", ErrorCodes::LOGICAL_ERROR);
if (num_qualifiers_to_strip)
{
size_t num_components = identifier->children.size();
/// plain column
if (num_components - num_qualifiers_to_strip == 1)
{
String node_alias = identifier->tryGetAlias();
ast = identifier->children.back();
if (!node_alias.empty())
ast->setAlias(node_alias);
}
else
/// nested column
{
identifier->children.erase(identifier->children.begin(), identifier->children.begin() + num_qualifiers_to_strip);
String new_name;
for (const auto & child : identifier->children)
{
if (!new_name.empty())
new_name += '.';
new_name += static_cast<const ASTIdentifier &>(*child.get()).name;
}
identifier->name = new_name;
}
}
}
void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const std::vector<DatabaseAndTableWithAlias> & tables)
{
if (auto * identifier = typeid_cast<ASTIdentifier *>(ast.get()))
@ -509,7 +367,6 @@ void ExpressionAnalyzer::translateQualifiedNamesImpl(ASTPtr & ast, const std::ve
}
}
void ExpressionAnalyzer::optimizeIfWithConstantCondition()
{
optimizeIfWithConstantConditionImpl(ast, aliases);
@ -566,10 +423,13 @@ void ExpressionAnalyzer::optimizeIfWithConstantConditionImpl(ASTPtr & current_as
optimizeIfWithConstantConditionImpl(function_node->arguments, aliases);
ASTExpressionList * args = typeid_cast<ASTExpressionList *>(function_node->arguments.get());
ASTPtr condition_expr = args->children.at(0);
ASTPtr then_expr = args->children.at(1);
ASTPtr else_expr = args->children.at(2);
if (args->children.size() != 3)
throw Exception("Wrong number of arguments for function 'if' (" + toString(args->children.size()) + " instead of 3)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTPtr condition_expr = args->children[0];
ASTPtr then_expr = args->children[1];
ASTPtr else_expr = args->children[2];
bool condition;
if (tryExtractConstValueFromCondition(condition_expr, condition))
@ -765,23 +625,6 @@ void ExpressionAnalyzer::findExternalTables(ASTPtr & ast)
external_tables[node->name] = external_storage;
}
static std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier)
{
std::pair<String, String> res;
res.second = identifier.name;
if (!identifier.children.empty())
{
if (identifier.children.size() != 2)
throw Exception("Qualified table name could have only two components", ErrorCodes::LOGICAL_ERROR);
res.first = typeid_cast<const ASTIdentifier &>(*identifier.children[0]).name;
res.second = typeid_cast<const ASTIdentifier &>(*identifier.children[1]).name;
}
return res;
}
static std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
const ASTPtr & subquery_or_table_name, const Context & context, size_t subquery_depth, const Names & required_source_columns)
{

View File

@ -4,6 +4,7 @@
#include <Interpreters/Settings.h>
#include <Core/Block.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/evaluateQualified.h>
#include <Interpreters/ProjectionManipulation.h>
#include <Parsers/StringRange.h>
#include <Parsers/ASTTablesInSelectQuery.h>
@ -91,19 +92,6 @@ struct ScopeStack
const Block & getSampleBlock() const;
};
struct DatabaseAndTableWithAlias
{
String database;
String table;
String alias;
/// "alias." or "database.table." if alias is empty
String getQualifiedNamePrefix() const;
/// If ast is ASTIdentifier, prepend getQualifiedNamePrefix() to it's name.
void makeQualifiedName(const ASTPtr & ast) const;
};
/** Transforms an expression from a syntax tree into a sequence of actions to execute it.
*
* NOTE: if `ast` is a SELECT query from a table, the structure of this table should not change during the lifetime of ExpressionAnalyzer.

View File

@ -60,6 +60,7 @@ namespace ErrorCodes
extern const int TOO_MANY_COLUMNS;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int PARAMETER_OUT_OF_BOUND;
}
InterpreterSelectQuery::InterpreterSelectQuery(
@ -695,6 +696,9 @@ void InterpreterSelectQuery::executeFetchColumns(
size_t max_block_size = settings.max_block_size;
if (!max_block_size)
throw Exception("Setting 'max_block_size' cannot be zero", ErrorCodes::PARAMETER_OUT_OF_BOUND);
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY but LIMIT is specified, and limit + offset < max_block_size,
* then as the block size we will use limit + offset (not to read more from the table than requested),
* and also set the number of threads to 1.

View File

@ -2,11 +2,11 @@
#include <Storages/IStorage.h>
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/queryToString.h>
#include <iostream>
#include <Parsers/ASTAsterisk.h>
#include <Parsers/ASTQualifiedAsterisk.h>
#include <Parsers/queryToString.h>
namespace DB
{
@ -14,18 +14,18 @@ namespace DB
static constexpr auto and_function_name = "and";
PredicateExpressionsOptimizer::PredicateExpressionsOptimizer(
ASTSelectQuery * ast_select_, const Settings & settings_)
: ast_select(ast_select_), settings(settings_)
ASTSelectQuery * ast_select_, const Settings & settings_, const Context & context_)
: ast_select(ast_select_), settings(settings_), context(context_)
{
}
bool PredicateExpressionsOptimizer::optimize()
{
if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables)
if (!settings.enable_optimize_predicate_expression || !ast_select || !ast_select->tables || ast_select->tables->children.empty())
return false;
SubqueriesProjectionColumns all_subquery_projection_columns;
getAllSubqueryProjectionColumns(ast_select->tables.get(), all_subquery_projection_columns);
getAllSubqueryProjectionColumns(all_subquery_projection_columns);
bool is_rewrite_subqueries = false;
if (!all_subquery_projection_columns.empty())
@ -42,11 +42,16 @@ bool PredicateExpressionsOptimizer::optimizeImpl(
/// split predicate with `and`
PredicateExpressions outer_predicate_expressions = splitConjunctionPredicate(outer_expression);
std::vector<ASTTableExpression *> tables_expression = getSelectTablesExpression(ast_select);
std::vector<DatabaseAndTableWithAlias> database_and_table_with_aliases;
for (const auto & table_expression : tables_expression)
database_and_table_with_aliases.emplace_back(getTableNameWithAliasFromTableExpression(*table_expression, context));
bool is_rewrite_subquery = false;
for (const auto & outer_predicate : outer_predicate_expressions)
{
ASTs outer_predicate_dependent;
getExpressionDependentColumns(outer_predicate, outer_predicate_dependent);
IdentifiersWithQualifiedNameSet outer_predicate_dependencies;
getDependenciesAndQualifiedOfExpression(outer_predicate, outer_predicate_dependencies, database_and_table_with_aliases);
/// TODO: remove origin expression
for (const auto & subquery_projection_columns : subqueries_projection_columns)
@ -55,10 +60,10 @@ bool PredicateExpressionsOptimizer::optimizeImpl(
const ProjectionsWithAliases projection_columns = subquery_projection_columns.second;
OptimizeKind optimize_kind = OptimizeKind::NONE;
if (!cannotPushDownOuterPredicate(projection_columns, subquery, outer_predicate_dependent, is_prewhere, optimize_kind))
if (!cannotPushDownOuterPredicate(projection_columns, subquery, outer_predicate_dependencies, is_prewhere, optimize_kind))
{
ASTPtr inner_predicate;
cloneOuterPredicateForInnerPredicate(outer_predicate, projection_columns, outer_predicate_dependent, inner_predicate);
cloneOuterPredicateForInnerPredicate(outer_predicate, projection_columns, database_and_table_with_aliases, inner_predicate);
switch(optimize_kind)
{
@ -109,34 +114,57 @@ PredicateExpressions PredicateExpressionsOptimizer::splitConjunctionPredicate(AS
return predicate_expressions;
}
void PredicateExpressionsOptimizer::getExpressionDependentColumns(const ASTPtr & expression, ASTs & expression_dependent_columns)
void PredicateExpressionsOptimizer::getDependenciesAndQualifiedOfExpression(const ASTPtr & expression,
IdentifiersWithQualifiedNameSet & dependencies_and_qualified,
std::vector<DatabaseAndTableWithAlias> & tables_with_aliases)
{
if (!typeid_cast<ASTIdentifier *>(expression.get()))
if (const auto identifier = typeid_cast<ASTIdentifier *>(expression.get()))
{
for (const auto & child : expression->children)
getExpressionDependentColumns(child, expression_dependent_columns);
if (!identifier->children.empty())
dependencies_and_qualified.emplace_back(std::pair(identifier, expression->getAliasOrColumnName()));
else
{
size_t best_table_pos = 0;
size_t max_num_qualifiers_to_strip = 0;
return;
/// translate qualifiers for dependent columns
for (size_t table_pos = 0; table_pos < tables_with_aliases.size(); ++table_pos)
{
const auto & table = tables_with_aliases[table_pos];
auto num_qualifiers_to_strip = getNumComponentsToStripInOrderToTranslateQualifiedName(*identifier, table);
if (num_qualifiers_to_strip > max_num_qualifiers_to_strip)
{
max_num_qualifiers_to_strip = num_qualifiers_to_strip;
best_table_pos = table_pos;
}
}
expression_dependent_columns.emplace_back(expression);
String qualified_name = tables_with_aliases[best_table_pos].getQualifiedNamePrefix() + expression->getAliasOrColumnName();
dependencies_and_qualified.emplace_back(std::pair(identifier, qualified_name));
}
}
else
{
for (const auto & child : expression->children)
getDependenciesAndQualifiedOfExpression(child, dependencies_and_qualified, tables_with_aliases);
}
}
bool PredicateExpressionsOptimizer::cannotPushDownOuterPredicate(
const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery,
ASTs & expression_dependent_columns, bool & is_prewhere, OptimizeKind & optimize_kind)
IdentifiersWithQualifiedNameSet & outer_predicate_dependencies, bool & is_prewhere, OptimizeKind & optimize_kind)
{
if (subquery->final() || subquery->limit_by_expression_list || subquery->limit_offset || subquery->with_expression_list)
if (subquery->final() || subquery->limit_by_expression_list || subquery->limit_length || subquery->with_expression_list)
return true;
for (auto & dependent_column : expression_dependent_columns)
for (auto & predicate_dependency : outer_predicate_dependencies)
{
bool is_found = false;
String dependent_column_name = dependent_column->getAliasOrColumnName();
for (auto projection_column : subquery_projection_columns)
{
if (projection_column.second == dependent_column_name)
if (projection_column.second == predicate_dependency.second)
{
is_found = true;
optimize_kind = isAggregateFunction(projection_column.first) ? OptimizeKind::PUSH_TO_HAVING : optimize_kind;
@ -168,39 +196,21 @@ bool PredicateExpressionsOptimizer::isAggregateFunction(ASTPtr & node)
return false;
}
void PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns(IAST * node, SubqueriesProjectionColumns & all_subquery_projection_columns)
{
if (auto ast_subquery = typeid_cast<ASTSubquery *>(node))
{
ASTs output_projection;
IAST * subquery = ast_subquery->children.at(0).get();
getSubqueryProjectionColumns(subquery, all_subquery_projection_columns, output_projection);
return;
}
for (auto & child : node->children)
getAllSubqueryProjectionColumns(child.get(), all_subquery_projection_columns);
}
void PredicateExpressionsOptimizer::cloneOuterPredicateForInnerPredicate(
const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns, ASTs & predicate_dependent_columns,
ASTPtr & inner_predicate)
const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns,
std::vector<DatabaseAndTableWithAlias> & tables, ASTPtr & inner_predicate)
{
inner_predicate = outer_predicate->clone();
ASTs new_expression_require_columns;
new_expression_require_columns.reserve(predicate_dependent_columns.size());
getExpressionDependentColumns(inner_predicate, new_expression_require_columns);
IdentifiersWithQualifiedNameSet new_expression_requires;
getDependenciesAndQualifiedOfExpression(inner_predicate, new_expression_requires, tables);
for (auto & expression : new_expression_require_columns)
{
if (auto identifier = typeid_cast<ASTIdentifier *>(expression.get()))
for (auto & require : new_expression_requires)
{
for (auto projection : projection_columns)
{
if (identifier->name == projection.second)
identifier->name = projection.first->getAliasOrColumnName();
}
if (require.second == projection.second)
require.first->name = projection.first->getAliasOrColumnName();
}
}
}
@ -221,32 +231,159 @@ bool PredicateExpressionsOptimizer::optimizeExpression(const ASTPtr & outer_expr
return true;
}
void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(IAST * subquery, SubqueriesProjectionColumns & all_subquery_projection_columns, ASTs & output_projections)
void PredicateExpressionsOptimizer::getAllSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns)
{
if (auto * with_union_subquery = typeid_cast<ASTSelectWithUnionQuery *>(subquery))
for (auto & select : with_union_subquery->list_of_selects->children)
getSubqueryProjectionColumns(select.get(), all_subquery_projection_columns, output_projections);
const auto tables_expression = getSelectTablesExpression(ast_select);
if (auto * without_union_subquery = typeid_cast<ASTSelectQuery *>(subquery))
for (const auto & table_expression : tables_expression)
{
const auto expression_list = without_union_subquery->select_expression_list->children;
if (table_expression->subquery)
{
/// Use qualifiers to translate the columns of subqueries
const auto database_and_table_with_alias = getTableNameWithAliasFromTableExpression(*table_expression, context);
String qualified_name_prefix = database_and_table_with_alias.getQualifiedNamePrefix();
getSubqueryProjectionColumns(all_subquery_projection_columns, qualified_name_prefix,
static_cast<const ASTSubquery *>(table_expression->subquery.get())->children[0]);
}
}
}
/// use first projection as the output projection
if (output_projections.empty())
output_projections = expression_list;
if (output_projections.size() != expression_list.size())
throw Exception("Number of columns doesn't match", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
void PredicateExpressionsOptimizer::getSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns,
String & qualified_name_prefix, const ASTPtr & subquery)
{
ASTs select_with_union_projections;
auto select_with_union_query = static_cast<ASTSelectWithUnionQuery *>(subquery.get());
for (auto & select_without_union_query : select_with_union_query->list_of_selects->children)
{
ProjectionsWithAliases subquery_projections;
subquery_projections.reserve(expression_list.size());
auto select_projection_columns = getSelectQueryProjectionColumns(select_without_union_query);
for (size_t idx = 0; idx < expression_list.size(); idx++)
subquery_projections.emplace_back(std::pair(expression_list.at(idx), output_projections.at(idx)->getAliasOrColumnName()));
if (!select_projection_columns.empty())
{
if (select_with_union_projections.empty())
select_with_union_projections = select_projection_columns;
all_subquery_projection_columns.insert(std::pair(subquery, subquery_projections));
for (size_t i = 0; i < select_projection_columns.size(); i++)
subquery_projections.emplace_back(std::pair(select_projection_columns[i],
qualified_name_prefix + select_with_union_projections[i]->getAliasOrColumnName()));
all_subquery_projection_columns.insert(std::pair(select_without_union_query.get(), subquery_projections));
}
}
}
ASTs PredicateExpressionsOptimizer::getSelectQueryProjectionColumns(ASTPtr & ast)
{
ASTs projection_columns;
auto select_query = static_cast<ASTSelectQuery *>(ast.get());
for (const auto & projection_column : select_query->select_expression_list->children)
{
if (typeid_cast<ASTAsterisk *>(projection_column.get()) || typeid_cast<ASTQualifiedAsterisk *>(projection_column.get()))
{
ASTs evaluated_columns = evaluateAsterisk(select_query, projection_column);
for (const auto & column : evaluated_columns)
projection_columns.emplace_back(column);
continue;
}
projection_columns.emplace_back(projection_column);
}
return projection_columns;
}
ASTs PredicateExpressionsOptimizer::evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk)
{
/// SELECT *, SELECT dummy, SELECT 1 AS id
if (!select_query->tables || select_query->tables->children.empty())
return {};
std::vector<ASTTableExpression *> tables_expression = getSelectTablesExpression(select_query);
if (const auto qualified_asterisk = typeid_cast<ASTQualifiedAsterisk *>(asterisk.get()))
{
if (qualified_asterisk->children.size() != 1)
throw Exception("Logical error: qualified asterisk must have exactly one child", ErrorCodes::LOGICAL_ERROR);
ASTIdentifier * ident = typeid_cast<ASTIdentifier *>(qualified_asterisk->children[0].get());
if (!ident)
throw Exception("Logical error: qualified asterisk must have identifier as its child", ErrorCodes::LOGICAL_ERROR);
size_t num_components = ident->children.size();
if (num_components > 2)
throw Exception("Qualified asterisk cannot have more than two qualifiers", ErrorCodes::UNKNOWN_ELEMENT_IN_AST);
for (auto it = tables_expression.begin(); it != tables_expression.end(); ++it)
{
const ASTTableExpression * table_expression = *it;
const auto database_and_table_with_alias = getTableNameWithAliasFromTableExpression(*table_expression, context);
/// database.table.*
if (num_components == 2 && !database_and_table_with_alias.database.empty()
&& static_cast<const ASTIdentifier &>(*ident->children[0]).name == database_and_table_with_alias.database
&& static_cast<const ASTIdentifier &>(*ident->children[1]).name == database_and_table_with_alias.table)
continue;
/// table.* or alias.*
else if (num_components == 0
&& ((!database_and_table_with_alias.table.empty() && ident->name == database_and_table_with_alias.table)
|| (!database_and_table_with_alias.alias.empty() && ident->name == database_and_table_with_alias.alias)))
continue;
else
/// It's not a required table
tables_expression.erase(it);
}
}
ASTs projection_columns;
for (auto & table_expression : tables_expression)
{
if (table_expression->subquery)
{
const auto subquery = static_cast<const ASTSubquery *>(table_expression->subquery.get());
const auto select_with_union_query = static_cast<ASTSelectWithUnionQuery *>(subquery->children[0].get());
const auto subquery_projections = getSelectQueryProjectionColumns(select_with_union_query->list_of_selects->children[0]);
projection_columns.insert(projection_columns.end(), subquery_projections.begin(), subquery_projections.end());
}
else
{
StoragePtr storage;
if (table_expression->table_function)
storage = const_cast<Context &>(context).executeTableFunction(table_expression->table_function);
else if (table_expression->database_and_table_name)
{
const auto database_and_table_ast = static_cast<ASTIdentifier*>(table_expression->database_and_table_name.get());
const auto database_and_table_name = getDatabaseAndTableNameFromIdentifier(*database_and_table_ast);
storage = context.tryGetTable(database_and_table_name.first, database_and_table_name.second);
}
const auto block = storage->getSampleBlock();
for (size_t idx = 0; idx < block.columns(); idx++)
projection_columns.emplace_back(std::make_shared<ASTIdentifier>(block.getByPosition(idx).name));
}
}
return projection_columns;
}
std::vector<ASTTableExpression *> PredicateExpressionsOptimizer::getSelectTablesExpression(ASTSelectQuery * select_query)
{
if (!select_query->tables)
return {};
std::vector<ASTTableExpression *> tables_expression;
const ASTTablesInSelectQuery & tables_in_select_query = static_cast<const ASTTablesInSelectQuery &>(*select_query->tables);
for (const auto & child : tables_in_select_query.children)
{
ASTTablesInSelectQueryElement * tables_element = static_cast<ASTTablesInSelectQueryElement *>(child.get());
if (tables_element->table_expression)
tables_expression.emplace_back(static_cast<ASTTableExpression *>(tables_element->table_expression.get()));
}
return tables_expression;
}
}

View File

@ -7,6 +7,9 @@
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/evaluateQualified.h>
namespace DB
{
@ -21,6 +24,8 @@ using PredicateExpressions = std::vector<ASTPtr>;
using ProjectionWithAlias = std::pair<ASTPtr, String>;
using ProjectionsWithAliases = std::vector<ProjectionWithAlias>;
using SubqueriesProjectionColumns = std::map<IAST *, ProjectionsWithAliases>;
using IdentifierWithQualifiedName = std::pair<ASTIdentifier *, String>;
using IdentifiersWithQualifiedNameSet = std::vector<IdentifierWithQualifiedName>;
/** This class provides functions for Push-Down predicate expressions
@ -37,13 +42,14 @@ using SubqueriesProjectionColumns = std::map<IAST *, ProjectionsWithAliases>;
class PredicateExpressionsOptimizer
{
public:
PredicateExpressionsOptimizer(ASTSelectQuery * ast_select_, const Settings & settings_);
PredicateExpressionsOptimizer(ASTSelectQuery * ast_select_, const Settings & settings_, const Context & context_);
bool optimize();
private:
ASTSelectQuery * ast_select;
const Settings & settings;
const Context & context;
enum OptimizeKind
{
@ -57,24 +63,29 @@ private:
PredicateExpressions splitConjunctionPredicate(ASTPtr & predicate_expression);
void getExpressionDependentColumns(const ASTPtr & expression, ASTs & expression_dependent_columns);
void getDependenciesAndQualifiedOfExpression(const ASTPtr & expression, IdentifiersWithQualifiedNameSet & dependencies_and_qualified,
std::vector<DatabaseAndTableWithAlias> & tables_with_aliases);
bool optimizeExpression(const ASTPtr & outer_expression, ASTPtr & subquery_expression, ASTSelectQuery * subquery);
bool optimizeImpl(ASTPtr & outer_expression, SubqueriesProjectionColumns & subqueries_projection_columns, bool is_prewhere);
bool cannotPushDownOuterPredicate(
const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery,
ASTs & expression_dependent_columns, bool & is_prewhere, OptimizeKind & optimize_kind);
bool cannotPushDownOuterPredicate(const ProjectionsWithAliases & subquery_projection_columns, ASTSelectQuery * subquery,
IdentifiersWithQualifiedNameSet & outer_predicate_dependencies, bool & is_prewhere, OptimizeKind & optimize_kind);
void cloneOuterPredicateForInnerPredicate(
const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns, ASTs & predicate_dependent_columns,
ASTPtr & inner_predicate);
void cloneOuterPredicateForInnerPredicate(const ASTPtr & outer_predicate, const ProjectionsWithAliases & projection_columns,
std::vector<DatabaseAndTableWithAlias> & tables, ASTPtr & inner_predicate);
void getAllSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns);
void getAllSubqueryProjectionColumns(IAST * node, SubqueriesProjectionColumns & all_subquery_projection_columns);
void getSubqueryProjectionColumns(SubqueriesProjectionColumns & all_subquery_projection_columns,
String & qualified_name_prefix, const ASTPtr & subquery);
void getSubqueryProjectionColumns(IAST * subquery, SubqueriesProjectionColumns & all_subquery_projection_columns, ASTs & output_projections);
ASTs getSelectQueryProjectionColumns(ASTPtr & ast);
std::vector<ASTTableExpression *> getSelectTablesExpression(ASTSelectQuery * select_query);
ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk);
};
}

View File

@ -283,6 +283,7 @@ struct Settings
M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.") \
M(SettingBool, asterisk_left_columns_only, 0, "If it is set to true, the asterisk only return left of join query.") \
M(SettingUInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).") \
M(SettingBool, calculate_text_stack_trace, 1, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.") \
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \

View File

@ -51,7 +51,7 @@ namespace DB
*/
#define DBMS_SYSTEM_LOG_QUEUE_SIZE 1024
#define DBMS_SYSTEM_LOG_QUEUE_SIZE 1048576
class Context;
class QueryLog;

View File

@ -0,0 +1,160 @@
#include <Interpreters/evaluateQualified.h>
#include <Interpreters/Context.h>
#include <Common/typeid_cast.h>
namespace DB
{
/// Checks that ast is ASTIdentifier and remove num_qualifiers_to_strip components from left.
/// Example: 'database.table.name' -> (num_qualifiers_to_strip = 2) -> 'name'.
void stripIdentifier(DB::ASTPtr & ast, size_t num_qualifiers_to_strip)
{
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(ast.get());
if (!identifier)
throw DB::Exception("ASTIdentifier expected for stripIdentifier", DB::ErrorCodes::LOGICAL_ERROR);
if (num_qualifiers_to_strip)
{
size_t num_components = identifier->children.size();
/// plain column
if (num_components - num_qualifiers_to_strip == 1)
{
DB::String node_alias = identifier->tryGetAlias();
ast = identifier->children.back();
if (!node_alias.empty())
ast->setAlias(node_alias);
}
else
/// nested column
{
identifier->children.erase(identifier->children.begin(), identifier->children.begin() + num_qualifiers_to_strip);
DB::String new_name;
for (const auto & child : identifier->children)
{
if (!new_name.empty())
new_name += '.';
new_name += static_cast<const ASTIdentifier &>(*child.get()).name;
}
identifier->name = new_name;
}
}
}
DatabaseAndTableWithAlias getTableNameWithAliasFromTableExpression(const ASTTableExpression & table_expression,
const Context & context)
{
DatabaseAndTableWithAlias database_and_table_with_alias;
if (table_expression.database_and_table_name)
{
const auto & identifier = static_cast<const ASTIdentifier &>(*table_expression.database_and_table_name);
database_and_table_with_alias.alias = identifier.tryGetAlias();
if (table_expression.database_and_table_name->children.empty())
{
database_and_table_with_alias.database = context.getCurrentDatabase();
database_and_table_with_alias.table = identifier.name;
}
else
{
if (table_expression.database_and_table_name->children.size() != 2)
throw Exception("Logical error: number of components in table expression not equal to two", ErrorCodes::LOGICAL_ERROR);
database_and_table_with_alias.database = static_cast<const ASTIdentifier &>(*identifier.children[0]).name;
database_and_table_with_alias.table = static_cast<const ASTIdentifier &>(*identifier.children[1]).name;
}
}
else if (table_expression.table_function)
{
database_and_table_with_alias.alias = table_expression.table_function->tryGetAlias();
}
else if (table_expression.subquery)
{
database_and_table_with_alias.alias = table_expression.subquery->tryGetAlias();
}
else
throw Exception("Logical error: no known elements in ASTTableExpression", ErrorCodes::LOGICAL_ERROR);
return database_and_table_with_alias;
}
/// Get the number of components of identifier which are correspond to 'alias.', 'table.' or 'databas.table.' from names.
size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier,
const DatabaseAndTableWithAlias & names)
{
size_t num_qualifiers_to_strip = 0;
auto get_identifier_name = [](const ASTPtr & ast) { return static_cast<const ASTIdentifier &>(*ast).name; };
/// It is compound identifier
if (!identifier.children.empty())
{
size_t num_components = identifier.children.size();
/// database.table.column
if (num_components >= 3
&& !names.database.empty()
&& get_identifier_name(identifier.children[0]) == names.database
&& get_identifier_name(identifier.children[1]) == names.table)
{
num_qualifiers_to_strip = 2;
}
/// table.column or alias.column. If num_components > 2, it is like table.nested.column.
if (num_components >= 2
&& ((!names.table.empty() && get_identifier_name(identifier.children[0]) == names.table)
|| (!names.alias.empty() && get_identifier_name(identifier.children[0]) == names.alias)))
{
num_qualifiers_to_strip = 1;
}
}
return num_qualifiers_to_strip;
}
std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier)
{
std::pair<String, String> res;
res.second = identifier.name;
if (!identifier.children.empty())
{
if (identifier.children.size() != 2)
throw Exception("Qualified table name could have only two components", ErrorCodes::LOGICAL_ERROR);
res.first = typeid_cast<const ASTIdentifier &>(*identifier.children[0]).name;
res.second = typeid_cast<const ASTIdentifier &>(*identifier.children[1]).name;
}
return res;
}
String DatabaseAndTableWithAlias::getQualifiedNamePrefix() const
{
return (!alias.empty() ? alias : (database + '.' + table)) + '.';
}
void DatabaseAndTableWithAlias::makeQualifiedName(const ASTPtr & ast) const
{
if (auto identifier = typeid_cast<ASTIdentifier *>(ast.get()))
{
String prefix = getQualifiedNamePrefix();
identifier->name.insert(identifier->name.begin(), prefix.begin(), prefix.end());
Names qualifiers;
if (!alias.empty())
qualifiers.push_back(alias);
else
{
qualifiers.push_back(database);
qualifiers.push_back(table);
}
for (const auto & qualifier : qualifiers)
identifier->children.emplace_back(std::make_shared<ASTIdentifier>(qualifier));
}
}
}

View File

@ -0,0 +1,34 @@
#pragma once
#include <Parsers/IAST.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Interpreters/Context.h>
namespace DB
{
struct DatabaseAndTableWithAlias
{
String database;
String table;
String alias;
/// "alias." or "database.table." if alias is empty
String getQualifiedNamePrefix() const;
/// If ast is ASTIdentifier, prepend getQualifiedNamePrefix() to it's name.
void makeQualifiedName(const ASTPtr & ast) const;
};
void stripIdentifier(DB::ASTPtr & ast, size_t num_qualifiers_to_strip);
DatabaseAndTableWithAlias getTableNameWithAliasFromTableExpression(const ASTTableExpression & table_expression,
const Context & context);
size_t getNumComponentsToStripInOrderToTranslateQualifiedName(const ASTIdentifier & identifier,
const DatabaseAndTableWithAlias & names);
std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier);
}

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