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

This commit is contained in:
Ivan Blinkov 2018-04-04 00:42:16 +03:00
commit 079cf3ce16
63 changed files with 435 additions and 520 deletions

View File

@ -9,7 +9,7 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/poco/CMakeLists.txt")
endif () endif ()
if (NOT USE_INTERNAL_POCO_LIBRARY) if (NOT USE_INTERNAL_POCO_LIBRARY)
find_package (Poco COMPONENTS Net NetSSL XML Data Crypto DataODBC MongoDB) find_package (Poco COMPONENTS Net NetSSL XML SQL Data Crypto DataODBC MongoDB)
endif () endif ()
if (Poco_INCLUDE_DIRS AND Poco_Foundation_LIBRARY) if (Poco_INCLUDE_DIRS AND Poco_Foundation_LIBRARY)
@ -24,6 +24,15 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
set (ENABLE_DATA_SQLITE 0 CACHE BOOL "") set (ENABLE_DATA_SQLITE 0 CACHE BOOL "")
set (ENABLE_DATA_MYSQL 0 CACHE BOOL "") set (ENABLE_DATA_MYSQL 0 CACHE BOOL "")
set (ENABLE_DATA_POSTGRESQL 0 CACHE BOOL "") set (ENABLE_DATA_POSTGRESQL 0 CACHE BOOL "")
# new after 2.0.0:
set (POCO_ENABLE_ZIP 0 CACHE BOOL "")
set (POCO_ENABLE_PAGECOMPILER 0 CACHE BOOL "")
set (POCO_ENABLE_PAGECOMPILER_FILE2PAGE 0 CACHE BOOL "")
set (POCO_ENABLE_REDIS 0 CACHE BOOL "")
set (POCO_ENABLE_SQL_SQLITE 0 CACHE BOOL "")
set (POCO_ENABLE_SQL_MYSQL 0 CACHE BOOL "")
set (POCO_ENABLE_SQL_POSTGRESQL 0 CACHE BOOL "")
set (POCO_UNBUNDLED 1 CACHE BOOL "") set (POCO_UNBUNDLED 1 CACHE BOOL "")
set (POCO_UNBUNDLED_PCRE 0 CACHE BOOL "") set (POCO_UNBUNDLED_PCRE 0 CACHE BOOL "")
set (POCO_UNBUNDLED_EXPAT 0 CACHE BOOL "") set (POCO_UNBUNDLED_EXPAT 0 CACHE BOOL "")
@ -44,9 +53,25 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
endif () endif ()
if (ODBC_FOUND) if (ODBC_FOUND)
if (EXISTS "${ClickHouse_SOURCE_DIR}/contrib/poco/SQL/ODBC/include/")
set (Poco_SQL_FOUND 1)
set (Poco_SQLODBC_FOUND 1)
set (Poco_SQL_INCLUDE_DIRS
"${ClickHouse_SOURCE_DIR}/contrib/poco/SQL/include"
"${ClickHouse_SOURCE_DIR}/contrib/poco/Data/include"
)
set (Poco_SQLODBC_INCLUDE_DIRS
"${ClickHouse_SOURCE_DIR}/contrib/poco/SQL/ODBC/include/"
"${ClickHouse_SOURCE_DIR}/contrib/poco/Data/ODBC/include/"
)
set (Poco_SQL_LIBRARY PocoSQL)
set (Poco_SQLODBC_LIBRARY PocoSQLODBC ${ODBC_LIBRARIES} ${LTDL_LIBRARY})
else ()
set (Poco_DataODBC_FOUND 1) set (Poco_DataODBC_FOUND 1)
set (Poco_DataODBC_INCLUDE_DIRS "${ClickHouse_SOURCE_DIR}/contrib/poco/Data/ODBC/include/" "${ClickHouse_SOURCE_DIR}/contrib/poco/Data/include")
set (Poco_Data_LIBRARY PocoData)
set (Poco_DataODBC_LIBRARY PocoDataODBC ${ODBC_LIBRARIES} ${LTDL_LIBRARY}) set (Poco_DataODBC_LIBRARY PocoDataODBC ${ODBC_LIBRARIES} ${LTDL_LIBRARY})
set (Poco_DataODBC_INCLUDE_DIRS "${ClickHouse_SOURCE_DIR}/contrib/poco/Data/ODBC/include/") endif ()
endif () endif ()
# TODO! fix internal ssl # TODO! fix internal ssl
@ -66,7 +91,6 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
set (Poco_Foundation_LIBRARY PocoFoundation) set (Poco_Foundation_LIBRARY PocoFoundation)
set (Poco_Util_LIBRARY PocoUtil) set (Poco_Util_LIBRARY PocoUtil)
set (Poco_Net_LIBRARY PocoNet) set (Poco_Net_LIBRARY PocoNet)
set (Poco_Data_LIBRARY PocoData)
set (Poco_XML_LIBRARY PocoXML) set (Poco_XML_LIBRARY PocoXML)
endif () endif ()

View File

@ -1,202 +0,0 @@
# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements. See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership. The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
cmake_minimum_required(VERSION 3.0)
# Modified version of CMakeLists.txt for ClickHouse. Doesn't link the library to libm.
# Otherwise we have extra dependency when compiling with the most fresh libc version.
# How to check:
# readelf -s ./clickhouse | grep -F 2.23
SET(LIBRARY_DIR ${ClickHouse_SOURCE_DIR}/contrib/zookeeper/src/c)
project(zookeeper VERSION 3.5.3)
set(email user@zookeeper.apache.org)
set(description "zookeeper C client")
# general options
if(UNIX)
add_compile_options(-Wall -fPIC)
elseif(WIN32)
add_compile_options(/W3)
endif()
add_definitions(-DUSE_STATIC_LIB)
# TODO: Enable /WX and /W4 on Windows. Currently there are ~1000 warnings.
# TODO: Add Solaris support.
# TODO: Add a shared library option.
# TODO: Specify symbols to export.
# TODO: Generate doxygen documentation.
# Sync API option
option(WANT_SYNCAPI "Enables Sync API support" ON)
if(WANT_SYNCAPI)
add_definitions(-DTHREADED)
if(WIN32)
# Note that the generator expression ensures that `/MTd` is used when Debug
# configurations are built.
add_compile_options(/MT$<$<CONFIG:Debug>:d>)
endif()
endif()
# CppUnit option
if(WIN32 OR APPLE)
# The tests do not yet compile on Windows or macOS,
# so we set this to off by default.
#
# Note that CMake does not have expressions except in conditionals,
# so we're left with this if/else/endif pattern.
set(DEFAULT_WANT_CPPUNIT OFF)
else()
set(DEFAULT_WANT_CPPUNIT ON)
endif()
option(WANT_CPPUNIT "Enables CppUnit and tests" ${DEFAULT_WANT_CPPUNIT})
# The function `to_have(in out)` converts a header name like `arpa/inet.h`
# into an Autotools style preprocessor definition `HAVE_ARPA_INET_H`.
# This is then set or unset in `configure_file()` step.
#
# Note that CMake functions do not have return values; instead an "out"
# variable must be passed, and explicitly set with parent scope.
function(to_have in out)
string(TOUPPER ${in} str)
string(REGEX REPLACE "/|\\." "_" str ${str})
set(${out} "HAVE_${str}" PARENT_SCOPE)
endfunction()
# include file checks
foreach(f generated/zookeeper.jute.h generated/zookeeper.jute.c)
if(EXISTS "${LIBRARY_DIR}/${f}")
to_have(${f} name)
set(${name} 1)
else()
message(FATAL_ERROR
"jute files are missing!\n"
"Please run 'ant compile_jute' while in the ZooKeeper top level directory.")
endif()
endforeach()
# header checks
include(CheckIncludeFile)
set(check_headers
arpa/inet.h
dlfcn.h
fcntl.h
inttypes.h
memory.h
netdb.h
netinet/in.h
stdint.h
stdlib.h
string.h
strings.h
sys/socket.h
sys/stat.h
sys/time.h
sys/types.h
unistd.h
sys/utsname.h)
foreach(f ${check_headers})
to_have(${f} name)
check_include_file(${f} ${name})
endforeach()
# function checks
include(CheckFunctionExists)
set(check_functions
getcwd
gethostbyname
gethostname
getlogin
getpwuid_r
gettimeofday
getuid
memmove
memset
poll
socket
strchr
strdup
strerror
strtol)
foreach(fn ${check_functions})
to_have(${fn} name)
check_function_exists(${fn} ${name})
endforeach()
# library checks
set(check_libraries rt m pthread)
foreach(lib ${check_libraries})
to_have("lib${lib}" name)
find_library(${name} ${lib})
endforeach()
# IPv6 check
include(CheckStructHasMember)
check_struct_has_member("struct sockaddr_in6" sin6_addr "netinet/in.h" ZOO_IPV6_ENABLED)
# configure
configure_file(${LIBRARY_DIR}/cmake_config.h.in ${CMAKE_CURRENT_BINARY_DIR}/include/config.h)
# hashtable library
set(hashtable_sources ${LIBRARY_DIR}/src/hashtable/hashtable_itr.c ${LIBRARY_DIR}/src/hashtable/hashtable.c)
add_library(hashtable STATIC ${hashtable_sources})
# zookeeper library
set(zookeeper_sources
${LIBRARY_DIR}/src/zookeeper.c
${LIBRARY_DIR}/src/recordio.c
${LIBRARY_DIR}/generated/zookeeper.jute.c
${LIBRARY_DIR}/src/zk_log.c
${LIBRARY_DIR}/src/zk_hashtable.c
)
# src/addrvec.c
if(WANT_SYNCAPI)
list(APPEND zookeeper_sources ${LIBRARY_DIR}/src/mt_adaptor.c)
else()
list(APPEND zookeeper_sources ${LIBRARY_DIR}/src/st_adaptor.c)
endif()
if(WIN32)
list(APPEND zookeeper_sources ${LIBRARY_DIR}/src/winport.c)
endif()
add_library(zookeeper STATIC ${zookeeper_sources})
target_include_directories(zookeeper BEFORE PUBLIC ${LIBRARY_DIR}/include ${CMAKE_CURRENT_BINARY_DIR}/include ${LIBRARY_DIR}/generated)
target_link_libraries(zookeeper PUBLIC
hashtable
$<$<PLATFORM_ID:Linux>:rt> # clock_gettime
$<$<PLATFORM_ID:Windows>:ws2_32>) # Winsock 2.0
if(WANT_SYNCAPI AND NOT WIN32)
find_package(Threads REQUIRED)
target_link_libraries(zookeeper PUBLIC Threads::Threads)
endif()
# cli executable
add_executable(cli ${LIBRARY_DIR}/src/cli.c)
target_link_libraries(cli zookeeper)
target_link_libraries(cli $<$<PLATFORM_ID:Linux>:m>)
# load_gen executable
if(WANT_SYNCAPI AND NOT WIN32)
add_executable(load_gen ${LIBRARY_DIR}/src/load_gen.c)
target_link_libraries(load_gen zookeeper)
target_link_libraries(load_gen $<$<PLATFORM_ID:Linux>:m>)
endif()

View File

@ -161,11 +161,20 @@ if (NOT USE_INTERNAL_BOOST_LIBRARY)
target_include_directories (clickhouse_common_io BEFORE PUBLIC ${Boost_INCLUDE_DIRS}) target_include_directories (clickhouse_common_io BEFORE PUBLIC ${Boost_INCLUDE_DIRS})
endif () endif ()
if (Poco_DataODBC_FOUND) if (Poco_SQLODBC_FOUND)
target_link_libraries (dbms ${Poco_DataODBC_LIBRARY}) target_link_libraries (clickhouse_common_io ${Poco_SQL_LIBRARY})
target_include_directories (dbms PRIVATE ${ODBC_INCLUDE_DIRECTORIES}) target_include_directories (clickhouse_common_io PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQL_INCLUDE_DIRS})
target_link_libraries (dbms ${Poco_SQLODBC_LIBRARY} ${Poco_SQL_LIBRARY})
target_include_directories (dbms PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIRS} PUBLIC ${Poco_SQL_INCLUDE_DIRS})
endif() endif()
if (Poco_DataODBC_FOUND)
target_link_libraries (clickhouse_common_io ${Poco_Data_LIBRARY})
target_link_libraries (dbms ${Poco_DataODBC_LIBRARY})
target_include_directories (dbms PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIRS})
endif()
if (Poco_MongoDB_FOUND) if (Poco_MongoDB_FOUND)
target_link_libraries (dbms ${Poco_MongoDB_LIBRARY}) target_link_libraries (dbms ${Poco_MongoDB_LIBRARY})
endif() endif()
@ -212,6 +221,7 @@ endif ()
target_include_directories (dbms PUBLIC ${DBMS_INCLUDE_DIR}) target_include_directories (dbms PUBLIC ${DBMS_INCLUDE_DIR})
target_include_directories (clickhouse_common_io PUBLIC ${DBMS_INCLUDE_DIR}) target_include_directories (clickhouse_common_io PUBLIC ${DBMS_INCLUDE_DIR})
target_include_directories (clickhouse_common_io PUBLIC ${PCG_RANDOM_INCLUDE_DIR}) target_include_directories (clickhouse_common_io PUBLIC ${PCG_RANDOM_INCLUDE_DIR})
target_include_directories (clickhouse_common_io PUBLIC ${Poco_DataODBC_INCLUDE_DIRS})
target_include_directories (clickhouse_common_io BEFORE PUBLIC ${DOUBLE_CONVERSION_INCLUDE_DIR}) target_include_directories (clickhouse_common_io BEFORE PUBLIC ${DOUBLE_CONVERSION_INCLUDE_DIR})
# also for copy_headers.sh: # also for copy_headers.sh:

View File

@ -1,6 +1,6 @@
# This strings autochanged from release_lib.sh: # This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54371-testing) set(VERSION_DESCRIBE v1.1.54372-testing)
set(VERSION_REVISION 54371) set(VERSION_REVISION 54372)
# end of autochange # end of autochange
set (VERSION_MAJOR 1) set (VERSION_MAJOR 1)

View File

@ -280,8 +280,7 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead
read_list.push_back(*connection->socket); read_list.push_back(*connection->socket);
} }
/// If no data was found, then we check if there are any connections /// If no data was found, then we check if there are any connections ready for reading.
/// ready for reading.
if (read_list.empty()) if (read_list.empty())
{ {
Poco::Net::Socket::SocketList write_list; Poco::Net::Socket::SocketList write_list;
@ -300,6 +299,9 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead
throw Exception("Timeout exceeded while reading from " + dumpAddressesUnlocked(), ErrorCodes::TIMEOUT_EXCEEDED); throw Exception("Timeout exceeded while reading from " + dumpAddressesUnlocked(), ErrorCodes::TIMEOUT_EXCEEDED);
} }
/// TODO Absolutely wrong code: read_list could be empty; rand() is not thread safe and has low quality; motivation of rand is unclear.
/// This code path is disabled by default.
auto & socket = read_list[rand() % read_list.size()]; auto & socket = read_list[rand() % read_list.size()];
if (fd_to_replica_state_idx.empty()) if (fd_to_replica_state_idx.empty())
{ {

View File

@ -11,6 +11,7 @@
#include <Poco/DOM/Comment.h> #include <Poco/DOM/Comment.h>
#include <Poco/Util/XMLConfiguration.h> #include <Poco/Util/XMLConfiguration.h>
#include <Common/ZooKeeper/ZooKeeperNodeCache.h> #include <Common/ZooKeeper/ZooKeeperNodeCache.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#define PREPROCESSED_SUFFIX "-preprocessed" #define PREPROCESSED_SUFFIX "-preprocessed"
@ -382,17 +383,19 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string &
Poco::File merge_dir(merge_dir_name); Poco::File merge_dir(merge_dir_name);
if (!merge_dir.exists() || !merge_dir.isDirectory()) if (!merge_dir.exists() || !merge_dir.isDirectory())
continue; continue;
for (Poco::DirectoryIterator it(merge_dir_name); it != Poco::DirectoryIterator(); ++it) for (Poco::DirectoryIterator it(merge_dir_name); it != Poco::DirectoryIterator(); ++it)
{ {
Poco::File & file = *it; Poco::File & file = *it;
if (file.isFile() Poco::Path path(file.path());
&& (endsWith(file.path(), ".xml") || endsWith(file.path(), ".conf")) std::string extension = path.getExtension();
&& !startsWith(file.path(), ".")) // skip temporary files std::string base_name = path.getBaseName();
{
// Skip non-config and temporary files
if (file.isFile() && (extension == "xml" || extension == "conf") && !startsWith(base_name, "."))
files.push_back(file.path()); files.push_back(file.path());
} }
} }
}
std::sort(files.begin(), files.end()); std::sort(files.begin(), files.end());

View File

@ -67,6 +67,7 @@ void ConfigReloader::run()
catch (...) catch (...)
{ {
tryLogCurrentException(log, __PRETTY_FUNCTION__); tryLogCurrentException(log, __PRETTY_FUNCTION__);
std::this_thread::sleep_for(reload_interval);
} }
} }
} }

View File

@ -28,7 +28,9 @@
M(MemoryTrackingForMerges) \ M(MemoryTrackingForMerges) \
M(LeaderElection) \ M(LeaderElection) \
M(EphemeralNode) \ M(EphemeralNode) \
M(ZooKeeperSession) \
M(ZooKeeperWatch) \ M(ZooKeeperWatch) \
M(ZooKeeperRequest) \
M(DelayedInserts) \ M(DelayedInserts) \
M(ContextLockWait) \ M(ContextLockWait) \
M(StorageBufferRows) \ M(StorageBufferRows) \

View File

@ -56,14 +56,20 @@
\ \
M(ZooKeeperInit) \ M(ZooKeeperInit) \
M(ZooKeeperTransactions) \ M(ZooKeeperTransactions) \
M(ZooKeeperGetChildren) \ M(ZooKeeperList) \
M(ZooKeeperCreate) \ M(ZooKeeperCreate) \
M(ZooKeeperRemove) \ M(ZooKeeperRemove) \
M(ZooKeeperExists) \ M(ZooKeeperExists) \
M(ZooKeeperGet) \ M(ZooKeeperGet) \
M(ZooKeeperSet) \ M(ZooKeeperSet) \
M(ZooKeeperMulti) \ M(ZooKeeperMulti) \
M(ZooKeeperCheck) \
M(ZooKeeperClose) \
M(ZooKeeperWatchResponse) \
M(ZooKeeperExceptions) \ M(ZooKeeperExceptions) \
M(ZooKeeperWaitMicroseconds) \
M(ZooKeeperBytesSent) \
M(ZooKeeperBytesReceived) \
\ \
M(DistributedConnectionFailTry) \ M(DistributedConnectionFailTry) \
M(DistributedConnectionMissingTable) \ M(DistributedConnectionMissingTable) \

View File

@ -1,21 +1,6 @@
#pragma once #pragma once
#include <Common/Exception.h>
#include "Types.h" #include "Types.h"
#include <Common/ProfileEvents.h>
namespace DB
{
namespace ErrorCodes
{
extern const int KEEPER_EXCEPTION;
}
}
namespace ProfileEvents
{
extern const Event ZooKeeperExceptions;
}
namespace zkutil namespace zkutil
@ -43,42 +28,7 @@ inline bool isUserError(int32_t zk_return_code)
} }
class KeeperException : public DB::Exception using KeeperException = ZooKeeperImpl::Exception;
{
private:
/// delegate constructor, used to minimize repetition; last parameter used for overload resolution
KeeperException(const std::string & msg, const int32_t code, int)
: DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code) { incrementEventCounter(); }
public:
KeeperException(const std::string & msg, const int32_t code)
: KeeperException(msg + " (" + ZooKeeperImpl::ZooKeeper::errorMessage(code) + ")", code, 0) {}
explicit KeeperException(const int32_t code) : KeeperException(ZooKeeperImpl::ZooKeeper::errorMessage(code), code, 0) {}
KeeperException(const int32_t code, const std::string & path)
: KeeperException(std::string{ZooKeeperImpl::ZooKeeper::errorMessage(code)} + ", path: " + path, code, 0) {}
KeeperException(const KeeperException & exc) : DB::Exception(exc), code(exc.code) { incrementEventCounter(); }
const char * name() const throw() override { return "zkutil::KeeperException"; }
const char * className() const throw() override { return "zkutil::KeeperException"; }
KeeperException * clone() const override { return new KeeperException(*this); }
/// Any error related with network or master election
/// In case of these errors you should reinitialize ZooKeeper session.
bool isHardwareError() const
{
return zkutil::isHardwareError(code);
}
const int32_t code;
private:
static void incrementEventCounter()
{
ProfileEvents::increment(ProfileEvents::ZooKeeperExceptions);
}
};
class KeeperMultiException : public KeeperException class KeeperMultiException : public KeeperException

View File

@ -1,5 +1,7 @@
#include "KeeperException.h"
#include "Lock.h" #include "Lock.h"
using namespace zkutil; using namespace zkutil;
bool Lock::tryLock() bool Lock::tryLock()

View File

@ -40,7 +40,7 @@ namespace zkutil
{ {
unlock(); unlock();
} }
catch (const zkutil::KeeperException & e) catch (...)
{ {
DB::tryLogCurrentException(__PRETTY_FUNCTION__); DB::tryLogCurrentException(__PRETTY_FUNCTION__);
} }

View File

@ -1,4 +1,5 @@
#include "ZooKeeper.h" #include "ZooKeeper.h"
#include "KeeperException.h"
#include <random> #include <random>
#include <pcg_random.hpp> #include <pcg_random.hpp>
@ -6,7 +7,6 @@
#include <boost/algorithm/string.hpp> #include <boost/algorithm/string.hpp>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/ProfileEvents.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <Common/PODArray.h> #include <Common/PODArray.h>
#include <Common/randomSeed.h> #include <Common/randomSeed.h>
@ -15,25 +15,6 @@
#define ZOOKEEPER_OPERATION_TIMEOUT_MS 1000 #define ZOOKEEPER_OPERATION_TIMEOUT_MS 1000
namespace ProfileEvents
{
extern const Event ZooKeeperInit;
extern const Event ZooKeeperTransactions;
extern const Event ZooKeeperCreate;
extern const Event ZooKeeperRemove;
extern const Event ZooKeeperExists;
extern const Event ZooKeeperMulti;
extern const Event ZooKeeperGet;
extern const Event ZooKeeperSet;
extern const Event ZooKeeperGetChildren;
}
namespace CurrentMetrics
{
extern const Metric ZooKeeperWatch;
}
namespace DB namespace DB
{ {
namespace ErrorCodes namespace ErrorCodes
@ -84,8 +65,6 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000), Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000),
Poco::Timespan(0, ZOOKEEPER_OPERATION_TIMEOUT_MS * 1000)); Poco::Timespan(0, ZOOKEEPER_OPERATION_TIMEOUT_MS * 1000));
ProfileEvents::increment(ProfileEvents::ZooKeeperInit);
LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot)); LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot));
if (!chroot.empty() && !exists("/")) if (!chroot.empty() && !exists("/"))
@ -232,9 +211,6 @@ int32_t ZooKeeper::createImpl(const std::string & path, const std::string & data
impl->create(path, data, mode & 1, mode & 2, {}, callback); /// TODO better mode impl->create(path, data, mode & 1, mode & 2, {}, callback); /// TODO better mode
event.wait(); event.wait();
ProfileEvents::increment(ProfileEvents::ZooKeeperCreate);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return code; return code;
} }
@ -302,9 +278,6 @@ int32_t ZooKeeper::removeImpl(const std::string & path, int32_t version)
impl->remove(path, version, callback); impl->remove(path, version, callback);
event.wait(); event.wait();
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return code; return code;
} }
@ -339,9 +312,6 @@ int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat, WatchCallba
impl->exists(path, callback, watch_callback); impl->exists(path, callback, watch_callback);
event.wait(); event.wait();
ProfileEvents::increment(ProfileEvents::ZooKeeperExists);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return code; return code;
} }
@ -380,9 +350,6 @@ int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Stat * s
impl->get(path, callback, watch_callback); impl->get(path, callback, watch_callback);
event.wait(); event.wait();
ProfileEvents::increment(ProfileEvents::ZooKeeperGet);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return code; return code;
} }
@ -431,9 +398,6 @@ int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data,
impl->set(path, data, version, callback); impl->set(path, data, version, callback);
event.wait(); event.wait();
ProfileEvents::increment(ProfileEvents::ZooKeeperSet);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return code; return code;
} }
@ -483,9 +447,6 @@ int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses)
impl->multi(requests, callback); impl->multi(requests, callback);
event.wait(); event.wait();
ProfileEvents::increment(ProfileEvents::ZooKeeperMulti);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return code; return code;
} }
@ -592,9 +553,6 @@ void ZooKeeper::waitForDisappear(const std::string & path)
impl->exists(path, callback, watch); impl->exists(path, callback, watch);
event.wait(); event.wait();
ProfileEvents::increment(ProfileEvents::ZooKeeperExists);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) if (code == ZooKeeperImpl::ZooKeeper::ZNONODE)
return; return;
@ -643,9 +601,6 @@ std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncGet(const std
}; };
impl->get(path, std::move(callback), {}); impl->get(path, std::move(callback), {});
ProfileEvents::increment(ProfileEvents::ZooKeeperGet);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return future; return future;
} }
@ -664,9 +619,6 @@ std::future<ZooKeeperImpl::ZooKeeper::GetResponse> ZooKeeper::asyncTryGet(const
}; };
impl->get(path, std::move(callback), {}); impl->get(path, std::move(callback), {});
ProfileEvents::increment(ProfileEvents::ZooKeeperGet);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return future; return future;
} }
@ -684,9 +636,6 @@ std::future<ZooKeeperImpl::ZooKeeper::ExistsResponse> ZooKeeper::asyncExists(con
}; };
impl->exists(path, std::move(callback), {}); impl->exists(path, std::move(callback), {});
ProfileEvents::increment(ProfileEvents::ZooKeeperExists);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return future; return future;
} }
@ -705,9 +654,6 @@ std::future<ZooKeeperImpl::ZooKeeper::ListResponse> ZooKeeper::asyncGetChildren(
}; };
impl->list(path, std::move(callback), {}); impl->list(path, std::move(callback), {});
ProfileEvents::increment(ProfileEvents::ZooKeeperGetChildren);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return future; return future;
} }
@ -725,9 +671,6 @@ std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncRemove(con
}; };
impl->remove(path, version, std::move(callback)); impl->remove(path, version, std::move(callback));
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return future; return future;
} }
@ -745,9 +688,6 @@ std::future<ZooKeeperImpl::ZooKeeper::RemoveResponse> ZooKeeper::asyncTryRemove(
}; };
impl->remove(path, version, std::move(callback)); impl->remove(path, version, std::move(callback));
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return future; return future;
} }
@ -762,9 +702,6 @@ std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::tryAsyncMulti(co
}; };
impl->multi(ops, std::move(callback)); impl->multi(ops, std::move(callback));
ProfileEvents::increment(ProfileEvents::ZooKeeperMulti);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return future; return future;
} }
@ -782,9 +719,6 @@ std::future<ZooKeeperImpl::ZooKeeper::MultiResponse> ZooKeeper::asyncMulti(const
}; };
impl->multi(ops, std::move(callback)); impl->multi(ops, std::move(callback));
ProfileEvents::increment(ProfileEvents::ZooKeeperMulti);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
return future; return future;
} }

View File

@ -1,7 +1,6 @@
#pragma once #pragma once
#include "Types.h" #include "Types.h"
#include "KeeperException.h"
#include <Poco/Util/LayeredConfiguration.h> #include <Poco/Util/LayeredConfiguration.h>
#include <unordered_set> #include <unordered_set>
#include <future> #include <future>
@ -280,7 +279,7 @@ public:
{ {
zookeeper.tryRemove(path); zookeeper.tryRemove(path);
} }
catch (const KeeperException & e) catch (...)
{ {
ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode); ProfileEvents::increment(ProfileEvents::CannotRemoveEphemeralNode);
DB::tryLogCurrentException(__PRETTY_FUNCTION__); DB::tryLogCurrentException(__PRETTY_FUNCTION__);

View File

@ -1,9 +1,11 @@
#include <Common/ZooKeeper/ZooKeeperImpl.h> #include <Common/ZooKeeper/ZooKeeperImpl.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/ProfileEvents.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <Poco/Exception.h> #include <Poco/Exception.h>
@ -11,7 +13,40 @@
#include <array> #include <array>
//#include <iostream>
namespace DB
{
namespace ErrorCodes
{
extern const int KEEPER_EXCEPTION;
}
}
namespace ProfileEvents
{
extern const Event ZooKeeperExceptions;
extern const Event ZooKeeperInit;
extern const Event ZooKeeperTransactions;
extern const Event ZooKeeperCreate;
extern const Event ZooKeeperRemove;
extern const Event ZooKeeperExists;
extern const Event ZooKeeperMulti;
extern const Event ZooKeeperGet;
extern const Event ZooKeeperSet;
extern const Event ZooKeeperList;
extern const Event ZooKeeperCheck;
extern const Event ZooKeeperClose;
extern const Event ZooKeeperWaitMicroseconds;
extern const Event ZooKeeperBytesSent;
extern const Event ZooKeeperBytesReceived;
extern const Event ZooKeeperWatchResponse;
}
namespace CurrentMetrics
{
extern const Metric ZooKeeperRequest;
extern const Metric ZooKeeperWatch;
}
/** ZooKeeper wire protocol. /** ZooKeeper wire protocol.
@ -228,6 +263,33 @@ after:
namespace ZooKeeperImpl namespace ZooKeeperImpl
{ {
Exception::Exception(const std::string & msg, const int32_t code, int)
: DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code)
{
ProfileEvents::increment(ProfileEvents::ZooKeeperExceptions);
}
Exception::Exception(const std::string & msg, const int32_t code)
: Exception(msg + " (" + ZooKeeperImpl::ZooKeeper::errorMessage(code) + ")", code, 0)
{
}
Exception::Exception(const int32_t code)
: Exception(ZooKeeperImpl::ZooKeeper::errorMessage(code), code, 0)
{
}
Exception::Exception(const int32_t code, const std::string & path)
: Exception(std::string{ZooKeeperImpl::ZooKeeper::errorMessage(code)} + ", path: " + path, code, 0)
{
}
Exception::Exception(const Exception & exc)
: DB::Exception(exc), code(exc.code)
{
}
using namespace DB; using namespace DB;
@ -304,10 +366,10 @@ void read(String & s, ReadBuffer & in)
static constexpr int32_t max_string_size = 1 << 20; static constexpr int32_t max_string_size = 1 << 20;
int32_t size = 0; int32_t size = 0;
read(size, in); read(size, in);
if (size < 0) if (size < 0) /// TODO Actually it means that zookeeper node has NULL value. Maybe better to treat it like empty string.
throw Exception("Negative size"); /// TODO Actually it means that zookeeper node have NULL value. Maybe better to treat it like empty string. throw Exception("Negative size while reading string from ZooKeeper", ZooKeeper::ZMARSHALLINGERROR);
if (size > max_string_size) if (size > max_string_size)
throw Exception("Too large string size"); /// TODO error code throw Exception("Too large string size while reading from ZooKeeper", ZooKeeper::ZMARSHALLINGERROR);
s.resize(size); s.resize(size);
in.read(&s[0], size); in.read(&s[0], size);
} }
@ -317,7 +379,7 @@ template <size_t N> void read(std::array<char, N> & s, ReadBuffer & in)
int32_t size = 0; int32_t size = 0;
read(size, in); read(size, in);
if (size != N) if (size != N)
throw Exception("Unexpected array size"); /// TODO error code throw Exception("Unexpected array size while reading from ZooKeeper", ZooKeeper::ZMARSHALLINGERROR);
in.read(&s[0], N); in.read(&s[0], N);
} }
@ -347,9 +409,9 @@ template <typename T> void read(std::vector<T> & arr, ReadBuffer & in)
int32_t size = 0; int32_t size = 0;
read(size, in); read(size, in);
if (size < 0) if (size < 0)
throw Exception("Negative size"); throw Exception("Negative size while reading array from ZooKeeper", ZooKeeper::ZMARSHALLINGERROR);
if (size > max_array_size) if (size > max_array_size)
throw Exception("Too large array size"); /// TODO error code throw Exception("Too large array size while reading from ZooKeeper", ZooKeeper::ZMARSHALLINGERROR);
arr.resize(size); arr.resize(size);
for (auto & elem : arr) for (auto & elem : arr)
read(elem, in); read(elem, in);
@ -478,6 +540,8 @@ ZooKeeper::ZooKeeper(
send_thread = std::thread([this] { sendThread(); }); send_thread = std::thread([this] { sendThread(); });
receive_thread = std::thread([this] { receiveThread(); }); receive_thread = std::thread([this] { receiveThread(); });
ProfileEvents::increment(ProfileEvents::ZooKeeperInit);
} }
@ -488,6 +552,7 @@ void ZooKeeper::connect(
static constexpr size_t num_tries = 3; static constexpr size_t num_tries = 3;
bool connected = false; bool connected = false;
WriteBufferFromOwnString fail_reasons;
for (size_t try_no = 0; try_no < num_tries; ++try_no) for (size_t try_no = 0; try_no < num_tries; ++try_no)
{ {
for (const auto & address : addresses) for (const auto & address : addresses)
@ -500,10 +565,11 @@ void ZooKeeper::connect(
} }
catch (const Poco::Net::NetException & e) catch (const Poco::Net::NetException & e)
{ {
/// TODO log exception fail_reasons << "\n" << getCurrentExceptionMessage(false);
} }
catch (const Poco::TimeoutException & e) catch (const Poco::TimeoutException & e)
{ {
fail_reasons << "\n" << getCurrentExceptionMessage(false);
} }
} }
@ -512,7 +578,22 @@ void ZooKeeper::connect(
} }
if (!connected) if (!connected)
throw Exception("All connection tries failed"); /// TODO more info; error code {
WriteBufferFromOwnString out;
out << "All connection tries failed while connecting to ZooKeeper. Addresses: ";
bool first = true;
for (const auto & address : addresses)
{
if (first)
first = false;
else
out << ", ";
out << address.toString();
}
out << fail_reasons.str();
throw Exception(out.str(), ZCONNECTIONLOSS);
}
socket.setReceiveTimeout(operation_timeout); socket.setReceiveTimeout(operation_timeout);
socket.setSendTimeout(operation_timeout); socket.setSendTimeout(operation_timeout);
@ -553,15 +634,15 @@ void ZooKeeper::receiveHandshake()
read(handshake_length); read(handshake_length);
if (handshake_length != 36) if (handshake_length != 36)
throw Exception("Unexpected handshake length received: " + toString(handshake_length)); throw Exception("Unexpected handshake length received: " + toString(handshake_length), ZMARSHALLINGERROR);
read(protocol_version_read); read(protocol_version_read);
if (protocol_version_read != protocol_version) if (protocol_version_read != protocol_version)
throw Exception("Unexpected protocol version: " + toString(protocol_version_read)); throw Exception("Unexpected protocol version: " + toString(protocol_version_read), ZMARSHALLINGERROR);
read(timeout); read(timeout);
if (timeout != session_timeout.totalMilliseconds()) if (timeout != session_timeout.totalMilliseconds())
throw Exception("Received different session timeout from server: " + toString(timeout)); throw Exception("Received different session timeout from server: " + toString(timeout), ZMARSHALLINGERROR);
read(session_id); read(session_id);
read(passwd); read(passwd);
@ -588,14 +669,17 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data)
read(err); read(err);
if (xid != auth_xid) if (xid != auth_xid)
throw Exception("Unexpected event recievent in reply to auth request: " + toString(xid)); throw Exception("Unexpected event recieved in reply to auth request: " + toString(xid),
ZMARSHALLINGERROR);
int32_t actual_length = in->count() - count_before_event; int32_t actual_length = in->count() - count_before_event;
if (length != actual_length) if (length != actual_length)
throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length)); throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length),
ZMARSHALLINGERROR);
if (err) if (err)
throw Exception("Error received in reply to auth request. Code: " + toString(err) + ". Message: " + String(errorMessage(err))); throw Exception("Error received in reply to auth request. Code: " + toString(err) + ". Message: " + String(errorMessage(err)),
ZMARSHALLINGERROR);
} }
@ -607,6 +691,8 @@ void ZooKeeper::sendThread()
{ {
while (!expired) while (!expired)
{ {
auto prev_bytes_sent = out->count();
auto now = clock::now(); auto now = clock::now();
auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(session_timeout.totalMilliseconds() / 3); auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(session_timeout.totalMilliseconds() / 3);
@ -635,6 +721,8 @@ void ZooKeeper::sendThread()
request.xid = ping_xid; request.xid = ping_xid;
request.write(*out); request.write(*out);
} }
ProfileEvents::increment(ProfileEvents::ZooKeeperBytesSent, out->count() - prev_bytes_sent);
} }
} }
catch (...) catch (...)
@ -657,19 +745,21 @@ void ZooKeeper::receiveThread()
Int64 waited = 0; Int64 waited = 0;
while (!expired) while (!expired)
{ {
auto prev_bytes_received = in->count();
clock::time_point now = clock::now(); clock::time_point now = clock::now();
UInt64 max_wait = operation_timeout.totalMicroseconds(); UInt64 max_wait = operation_timeout.totalMicroseconds();
bool has_operations = false; std::optional<RequestInfo> earliest_operation;
{ {
std::lock_guard lock(operations_mutex); std::lock_guard lock(operations_mutex);
if (!operations.empty()) if (!operations.empty())
{ {
/// Operations are ordered by xid (and consequently, by time). /// Operations are ordered by xid (and consequently, by time).
has_operations = true; earliest_operation = operations.begin()->second;
auto earliest_operation_deadline = operations.begin()->second.time + std::chrono::microseconds(operation_timeout.totalMicroseconds()); auto earliest_operation_deadline = earliest_operation->time + std::chrono::microseconds(operation_timeout.totalMicroseconds());
if (now > earliest_operation_deadline) if (now > earliest_operation_deadline)
throw Exception("Operation timeout"); throw Exception("Operation timeout (deadline already expired) for path: " + earliest_operation->request->getPath(), ZOPERATIONTIMEOUT);
max_wait = std::chrono::duration_cast<std::chrono::microseconds>(earliest_operation_deadline - now).count(); max_wait = std::chrono::duration_cast<std::chrono::microseconds>(earliest_operation_deadline - now).count();
} }
} }
@ -684,13 +774,15 @@ void ZooKeeper::receiveThread()
} }
else else
{ {
if (has_operations) if (earliest_operation)
throw Exception("Operation timeout"); throw Exception("Operation timeout (no response) for path: " + earliest_operation->request->getPath(), ZOPERATIONTIMEOUT);
waited += max_wait; waited += max_wait;
if (waited > session_timeout.totalMicroseconds()) if (waited > session_timeout.totalMicroseconds())
throw Exception("Nothing is received in session timeout"); throw Exception("Nothing is received in session timeout", ZOPERATIONTIMEOUT);
} }
ProfileEvents::increment(ProfileEvents::ZooKeeperBytesReceived, in->count() - prev_bytes_received);
} }
} }
catch (...) catch (...)
@ -729,14 +821,17 @@ ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { return st
void addRootPath(String & path, const String & root_path) void addRootPath(String & path, const String & root_path)
{ {
if (path.empty()) if (path.empty())
throw Exception("Path cannot be empty"); throw Exception("Path cannot be empty", ZooKeeper::ZBADARGUMENTS);
if (path[0] != '/') if (path[0] != '/')
throw Exception("Path must begin with /"); throw Exception("Path must begin with /", ZooKeeper::ZBADARGUMENTS);
if (root_path.empty()) if (root_path.empty())
return; return;
if (path.size() == 1) /// "/"
path = root_path;
else
path = root_path + path; path = root_path + path;
} }
@ -746,7 +841,7 @@ void removeRootPath(String & path, const String & root_path)
return; return;
if (path.size() <= root_path.size()) if (path.size() <= root_path.size())
throw Exception("Received path is not longer than root_path"); throw Exception("Received path is not longer than root_path", ZooKeeper::ZDATAINCONSISTENCY);
path = path.substr(root_path.size()); path = path.substr(root_path.size());
} }
@ -795,14 +890,13 @@ void ZooKeeper::receiveEvent()
if (xid == ping_xid) if (xid == ping_xid)
{ {
if (err) if (err)
throw Exception("Received error in heartbeat response: " + String(errorMessage(err))); throw Exception("Received error in heartbeat response: " + String(errorMessage(err)), ZRUNTIMEINCONSISTENCY);
response = std::make_shared<HeartbeatResponse>(); response = std::make_shared<HeartbeatResponse>();
// std::cerr << "Received heartbeat\n";
} }
else if (xid == watch_xid) else if (xid == watch_xid)
{ {
ProfileEvents::increment(ProfileEvents::ZooKeeperWatchResponse);
response = std::make_shared<WatchResponse>(); response = std::make_shared<WatchResponse>();
request_info.callback = [this](const Response & response) request_info.callback = [this](const Response & response)
@ -829,11 +923,10 @@ void ZooKeeper::receiveEvent()
if (callback) if (callback)
callback(watch_response); /// NOTE We may process callbacks not under mutex. callback(watch_response); /// NOTE We may process callbacks not under mutex.
CurrentMetrics::sub(CurrentMetrics::ZooKeeperWatch, it->second.size());
watches.erase(it); watches.erase(it);
} }
}; };
// std::cerr << "Received watch\n";
} }
else else
{ {
@ -842,14 +935,13 @@ void ZooKeeper::receiveEvent()
auto it = operations.find(xid); auto it = operations.find(xid);
if (it == operations.end()) if (it == operations.end())
throw Exception("Received response for unknown xid"); throw Exception("Received response for unknown xid", ZRUNTIMEINCONSISTENCY);
request_info = std::move(it->second); request_info = std::move(it->second);
operations.erase(it); operations.erase(it);
CurrentMetrics::sub(CurrentMetrics::ZooKeeperRequest);
} }
// std::cerr << "Received response: " << request_info.request->getOpNum() << "\n";
response = request_info.request->makeResponse(); response = request_info.request->makeResponse();
} }
@ -863,7 +955,10 @@ void ZooKeeper::receiveEvent()
int32_t actual_length = in->count() - count_before_event; int32_t actual_length = in->count() - count_before_event;
if (length != actual_length) if (length != actual_length)
throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length)); throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length), ZMARSHALLINGERROR);
auto elapsed_microseconds = std::chrono::duration_cast<std::chrono::microseconds>(clock::now() - request_info.time).count();
ProfileEvents::increment(ProfileEvents::ZooKeeperWaitMicroseconds, elapsed_microseconds);
if (request_info.callback) if (request_info.callback)
request_info.callback(*response); request_info.callback(*response);
@ -902,6 +997,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive)
request_info.callback(*response); request_info.callback(*response);
} }
CurrentMetrics::sub(CurrentMetrics::ZooKeeperRequest, operations.size());
operations.clear(); operations.clear();
} }
@ -920,6 +1016,7 @@ void ZooKeeper::finalize(bool error_send, bool error_receive)
callback(response); callback(response);
} }
CurrentMetrics::sub(CurrentMetrics::ZooKeeperWatch, watches.size());
watches.clear(); watches.clear();
} }
} }
@ -1055,12 +1152,13 @@ void ZooKeeper::ErrorResponse::readImpl(ReadBuffer & in)
ZooKeeperImpl::read(read_error, in); ZooKeeperImpl::read(read_error, in);
if (read_error != error) if (read_error != error)
throw Exception("Error code in ErrorResponse (" + toString(read_error) + ") doesn't match error code in header (" + toString(error) + ")"); throw Exception("Error code in ErrorResponse (" + toString(read_error) + ") doesn't match error code in header (" + toString(error) + ")",
ZMARSHALLINGERROR);
} }
void ZooKeeper::CloseResponse::readImpl(ReadBuffer &) void ZooKeeper::CloseResponse::readImpl(ReadBuffer &)
{ {
throw Exception("Received response for close request"); throw Exception("Received response for close request", ZRUNTIMEINCONSISTENCY);
} }
ZooKeeper::MultiResponse::MultiResponse(const Requests & requests) ZooKeeper::MultiResponse::MultiResponse(const Requests & requests)
@ -1083,10 +1181,8 @@ void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in)
ZooKeeperImpl::read(done, in); ZooKeeperImpl::read(done, in);
ZooKeeperImpl::read(op_error, in); ZooKeeperImpl::read(op_error, in);
// std::cerr << "Received result for multi: " << op_num << "\n";
if (done) if (done)
throw Exception("Not enough results received for multi transaction"); throw Exception("Not enough results received for multi transaction", ZMARSHALLINGERROR);
/// op_num == -1 is special for multi transaction. /// op_num == -1 is special for multi transaction.
/// For unknown reason, error code is duplicated in header and in response body. /// For unknown reason, error code is duplicated in header and in response body.
@ -1120,11 +1216,11 @@ void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in)
ZooKeeperImpl::read(error, in); ZooKeeperImpl::read(error, in);
if (!done) if (!done)
throw Exception("Too many results received for multi transaction"); throw Exception("Too many results received for multi transaction", ZMARSHALLINGERROR);
if (op_num != -1) if (op_num != -1)
throw Exception("Unexpected op_num received at the end of results for multi transaction"); throw Exception("Unexpected op_num received at the end of results for multi transaction", ZMARSHALLINGERROR);
if (error != -1) if (error != -1)
throw Exception("Unexpected error value received at the end of results for multi transaction"); throw Exception("Unexpected error value received at the end of results for multi transaction", ZMARSHALLINGERROR);
} }
} }
@ -1133,7 +1229,7 @@ void ZooKeeper::pushRequest(RequestInfo && info)
{ {
/// If the request is close request, we push it even after session is expired - because it will signal sending thread to stop. /// If the request is close request, we push it even after session is expired - because it will signal sending thread to stop.
if (expired && info.request->xid != close_xid) if (expired && info.request->xid != close_xid)
throw Exception("Session expired"); throw Exception("Session expired", ZSESSIONEXPIRED);
info.request->addRootPath(root_path); info.request->addRootPath(root_path);
@ -1143,10 +1239,13 @@ void ZooKeeper::pushRequest(RequestInfo && info)
{ {
info.request->xid = xid.fetch_add(1); info.request->xid = xid.fetch_add(1);
if (info.request->xid < 0) if (info.request->xid < 0)
throw Exception("XID overflow"); throw Exception("XID overflow", ZSESSIONEXPIRED);
} }
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
{ {
CurrentMetrics::add(CurrentMetrics::ZooKeeperRequest);
std::lock_guard lock(operations_mutex); std::lock_guard lock(operations_mutex);
operations[info.request->xid] = info; operations[info.request->xid] = info;
} }
@ -1154,12 +1253,13 @@ void ZooKeeper::pushRequest(RequestInfo && info)
if (info.watch) if (info.watch)
{ {
info.request->has_watch = true; info.request->has_watch = true;
CurrentMetrics::add(CurrentMetrics::ZooKeeperWatch);
std::lock_guard lock(watches_mutex); std::lock_guard lock(watches_mutex);
watches[info.request->getPath()].emplace_back(std::move(info.watch)); watches[info.request->getPath()].emplace_back(std::move(info.watch));
} }
if (!requests.tryPush(info.request, session_timeout.totalMilliseconds())) if (!requests.tryPush(info.request, operation_timeout.totalMilliseconds()))
throw Exception("Cannot push request to queue within session timeout"); throw Exception("Cannot push request to queue within operation timeout", ZOPERATIONTIMEOUT);
} }
@ -1183,6 +1283,7 @@ void ZooKeeper::create(
request_info.callback = [callback](const Response & response) { callback(typeid_cast<const CreateResponse &>(response)); }; request_info.callback = [callback](const Response & response) { callback(typeid_cast<const CreateResponse &>(response)); };
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperCreate);
} }
@ -1200,6 +1301,7 @@ void ZooKeeper::remove(
request_info.callback = [callback](const Response & response) { callback(typeid_cast<const RemoveResponse &>(response)); }; request_info.callback = [callback](const Response & response) { callback(typeid_cast<const RemoveResponse &>(response)); };
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperRemove);
} }
@ -1217,6 +1319,7 @@ void ZooKeeper::exists(
request_info.watch = watch; request_info.watch = watch;
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperExists);
} }
@ -1234,6 +1337,7 @@ void ZooKeeper::get(
request_info.watch = watch; request_info.watch = watch;
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperGet);
} }
@ -1253,6 +1357,7 @@ void ZooKeeper::set(
request_info.callback = [callback](const Response & response) { callback(typeid_cast<const SetResponse &>(response)); }; request_info.callback = [callback](const Response & response) { callback(typeid_cast<const SetResponse &>(response)); };
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperSet);
} }
@ -1270,6 +1375,7 @@ void ZooKeeper::list(
request_info.watch = watch; request_info.watch = watch;
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperList);
} }
@ -1287,6 +1393,7 @@ void ZooKeeper::check(
request_info.callback = [callback](const Response & response) { callback(typeid_cast<const CheckResponse &>(response)); }; request_info.callback = [callback](const Response & response) { callback(typeid_cast<const CheckResponse &>(response)); };
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperCheck);
} }
@ -1307,6 +1414,7 @@ void ZooKeeper::multi(
request_info.callback = [callback](const Response & response) { callback(typeid_cast<const MultiResponse &>(response)); }; request_info.callback = [callback](const Response & response) { callback(typeid_cast<const MultiResponse &>(response)); };
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperMulti);
} }
@ -1319,6 +1427,7 @@ void ZooKeeper::close()
request_info.request = std::make_shared<CloseRequest>(std::move(request)); request_info.request = std::make_shared<CloseRequest>(std::move(request));
pushRequest(std::move(request_info)); pushRequest(std::move(request_info));
ProfileEvents::increment(ProfileEvents::ZooKeeperClose);
} }

View File

@ -2,6 +2,7 @@
#include <Core/Types.h> #include <Core/Types.h>
#include <Common/ConcurrentBoundedQueue.h> #include <Common/ConcurrentBoundedQueue.h>
#include <Common/CurrentMetrics.h>
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
@ -23,12 +24,38 @@
#include <functional> #include <functional>
namespace CurrentMetrics
{
extern const Metric ZooKeeperSession;
}
namespace ZooKeeperImpl namespace ZooKeeperImpl
{ {
using namespace DB; 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;
};
/** Usage scenario: /** Usage scenario:
* - create an object and issue commands; * - create an object and issue commands;
* - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap: * - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap:
@ -543,6 +570,8 @@ private:
template <typename T> template <typename T>
void read(T &); void read(T &);
CurrentMetrics::Increment metric_increment{CurrentMetrics::ZooKeeperSession};
}; };
}; };

View File

@ -1,5 +1,6 @@
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <iostream> #include <iostream>
#include <chrono> #include <chrono>

View File

@ -1,5 +1,6 @@
#include <iostream> #include <iostream>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Poco/ConsoleChannel.h> #include <Poco/ConsoleChannel.h>
#include <Common/Exception.h> #include <Common/Exception.h>

View File

@ -1,4 +1,5 @@
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <iostream> #include <iostream>
#include <port/unistd.h> #include <port/unistd.h>

View File

@ -9,6 +9,7 @@
#cmakedefine01 USE_RDKAFKA #cmakedefine01 USE_RDKAFKA
#cmakedefine01 USE_CAPNP #cmakedefine01 USE_CAPNP
#cmakedefine01 USE_EMBEDDED_COMPILER #cmakedefine01 USE_EMBEDDED_COMPILER
#cmakedefine01 Poco_SQLODBC_FOUND
#cmakedefine01 Poco_DataODBC_FOUND #cmakedefine01 Poco_DataODBC_FOUND
#cmakedefine01 Poco_MongoDB_FOUND #cmakedefine01 Poco_MongoDB_FOUND
#cmakedefine01 Poco_NetSSL_FOUND #cmakedefine01 Poco_NetSSL_FOUND

View File

@ -34,6 +34,7 @@ const char * auto_config_build[]
"USE_VECTORCLASS", "@USE_VECTORCLASS@", "USE_VECTORCLASS", "@USE_VECTORCLASS@",
"USE_RDKAFKA", "@USE_RDKAFKA@", "USE_RDKAFKA", "@USE_RDKAFKA@",
"USE_CAPNP", "@USE_CAPNP@", "USE_CAPNP", "@USE_CAPNP@",
"USE_Poco_SQLODBC", "@Poco_SQLODBC_FOUND@",
"USE_Poco_DataODBC", "@Poco_DataODBC_FOUND@", "USE_Poco_DataODBC", "@Poco_DataODBC_FOUND@",
"USE_Poco_MongoDB", "@Poco_MongoDB_FOUND@", "USE_Poco_MongoDB", "@Poco_MongoDB_FOUND@",
"USE_Poco_NetSSL", "@Poco_NetSSL_FOUND@", "USE_Poco_NetSSL", "@Poco_NetSSL_FOUND@",

View File

@ -42,7 +42,7 @@ Block IProfilingBlockInputStream::read()
if (isCancelledOrThrowIfKilled()) if (isCancelledOrThrowIfKilled())
return res; return res;
if (!checkTimeLimits()) if (!checkTimeLimit())
limit_exceeded_need_break = true; limit_exceeded_need_break = true;
if (!limit_exceeded_need_break) if (!limit_exceeded_need_break)
@ -191,7 +191,7 @@ static bool handleOverflowMode(OverflowMode mode, const String & message, int co
}; };
bool IProfilingBlockInputStream::checkTimeLimits() bool IProfilingBlockInputStream::checkTimeLimit()
{ {
if (limits.max_execution_time != 0 if (limits.max_execution_time != 0
&& info.total_stopwatch.elapsed() > static_cast<UInt64>(limits.max_execution_time.totalMicroseconds()) * 1000) && info.total_stopwatch.elapsed() > static_cast<UInt64>(limits.max_execution_time.totalMicroseconds()) * 1000)

View File

@ -234,10 +234,10 @@ private:
void updateExtremes(Block & block); void updateExtremes(Block & block);
/** Check constraints and quotas. /** Check limits and quotas.
* But only those that can be tested within each separate source. * But only those that can be checked within each separate stream.
*/ */
bool checkTimeLimits(); bool checkTimeLimit();
void checkQuota(Block & block); void checkQuota(Block & block);

View File

@ -19,7 +19,7 @@
#if Poco_MongoDB_FOUND #if Poco_MongoDB_FOUND
#include <Dictionaries/MongoDBDictionarySource.h> #include <Dictionaries/MongoDBDictionarySource.h>
#endif #endif
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
#pragma GCC diagnostic push #pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wunused-parameter" #pragma GCC diagnostic ignored "-Wunused-parameter"
#include <Poco/Data/ODBC/Connector.h> #include <Poco/Data/ODBC/Connector.h>
@ -89,7 +89,7 @@ Block createSampleBlock(const DictionaryStructure & dict_struct)
DictionarySourceFactory::DictionarySourceFactory() DictionarySourceFactory::DictionarySourceFactory()
: log(&Poco::Logger::get("DictionarySourceFactory")) : log(&Poco::Logger::get("DictionarySourceFactory"))
{ {
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
Poco::Data::ODBC::Connector::registerConnector(); Poco::Data::ODBC::Connector::registerConnector();
#endif #endif
} }
@ -154,7 +154,7 @@ DictionarySourcePtr DictionarySourceFactory::create(
} }
else if ("odbc" == source_type) else if ("odbc" == source_type)
{ {
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
return std::make_unique<ODBCDictionarySource>(dict_struct, config, config_prefix + ".odbc", sample_block, context); return std::make_unique<ODBCDictionarySource>(dict_struct, config, config_prefix + ".odbc", sample_block, context);
#else #else
throw Exception{"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.", throw Exception{"Dictionary source of type `odbc` is disabled because poco library was built without ODBC support.",

View File

@ -1,17 +1,15 @@
#pragma once #pragma once
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wunused-parameter"
#include <Poco/Data/SessionPool.h>
#pragma GCC diagnostic pop
#include <Dictionaries/IDictionarySource.h> #include <Dictionaries/IDictionarySource.h>
#include <Dictionaries/ExternalQueryBuilder.h> #include <Dictionaries/ExternalQueryBuilder.h>
#include <Dictionaries/DictionaryStructure.h> #include <Dictionaries/DictionaryStructure.h>
namespace Poco namespace Poco
{ {
namespace Data
{
class SessionPool;
}
namespace Util namespace Util
{ {
class AbstractConfiguration; class AbstractConfiguration;

View File

@ -515,12 +515,6 @@ void Context::setConfig(const ConfigurationPtr & config)
shared->config = config; shared->config = config;
} }
ConfigurationPtr Context::getConfig() const
{
auto lock = getLock();
return shared->config;
}
Poco::Util::AbstractConfiguration & Context::getConfigRef() const Poco::Util::AbstractConfiguration & Context::getConfigRef() const
{ {
auto lock = getLock(); auto lock = getLock();
@ -689,10 +683,10 @@ void Context::assertTableExists(const String & database_name, const String & tab
Databases::const_iterator it = shared->databases.find(db); Databases::const_iterator it = shared->databases.find(db);
if (shared->databases.end() == it) if (shared->databases.end() == it)
throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); throw Exception("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
if (!it->second->isTableExist(*this, table_name)) if (!it->second->isTableExist(*this, table_name))
throw Exception("Table " + db + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); throw Exception("Table " + backQuoteIfNeed(db) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
} }
@ -706,7 +700,7 @@ void Context::assertTableDoesntExist(const String & database_name, const String
Databases::const_iterator it = shared->databases.find(db); Databases::const_iterator it = shared->databases.find(db);
if (shared->databases.end() != it && it->second->isTableExist(*this, table_name)) if (shared->databases.end() != it && it->second->isTableExist(*this, table_name))
throw Exception("Table " + db + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); throw Exception("Table " + backQuoteIfNeed(db) + "." + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
} }
@ -719,7 +713,7 @@ void Context::assertDatabaseExists(const String & database_name, bool check_data
checkDatabaseAccessRights(db); checkDatabaseAccessRights(db);
if (shared->databases.end() == shared->databases.find(db)) if (shared->databases.end() == shared->databases.find(db))
throw Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); throw Exception("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
} }
@ -731,7 +725,7 @@ void Context::assertDatabaseDoesntExist(const String & database_name) const
checkDatabaseAccessRights(db); checkDatabaseAccessRights(db);
if (shared->databases.end() != shared->databases.find(db)) if (shared->databases.end() != shared->databases.find(db))
throw Exception("Database " + db + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS); throw Exception("Database " + backQuoteIfNeed(db) + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS);
} }
@ -801,7 +795,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta
if (shared->databases.end() == it) if (shared->databases.end() == it)
{ {
if (exception) if (exception)
*exception = Exception("Database " + db + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE); *exception = Exception("Database " + backQuoteIfNeed(db) + " doesn't exist", ErrorCodes::UNKNOWN_DATABASE);
return {}; return {};
} }
@ -809,7 +803,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta
if (!table) if (!table)
{ {
if (exception) if (exception)
*exception = Exception("Table " + db + "." + table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE); *exception = Exception("Table " + backQuoteIfNeed(db) + "." + backQuoteIfNeed(table_name) + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
return {}; return {};
} }
@ -820,7 +814,7 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta
void Context::addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast) void Context::addExternalTable(const String & table_name, const StoragePtr & storage, const ASTPtr & ast)
{ {
if (external_tables.end() != external_tables.find(table_name)) if (external_tables.end() != external_tables.find(table_name))
throw Exception("Temporary table " + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS); throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
external_tables[table_name] = std::pair(storage, ast); external_tables[table_name] = std::pair(storage, ast);
} }
@ -926,7 +920,7 @@ ASTPtr Context::getCreateExternalTableQuery(const String & table_name) const
{ {
TableAndCreateASTs::const_iterator jt = external_tables.find(table_name); TableAndCreateASTs::const_iterator jt = external_tables.find(table_name);
if (external_tables.end() == jt) if (external_tables.end() == jt)
throw Exception("Temporary Table" + table_name + " doesn't exist", ErrorCodes::UNKNOWN_TABLE); throw Exception("Temporary table " + backQuoteIfNeed(table_name) + " doesn't exist", ErrorCodes::UNKNOWN_TABLE);
return jt->second.second; return jt->second.second;
} }
@ -1326,21 +1320,13 @@ DDLWorker & Context::getDDLWorker() const
return *shared->ddl_worker; return *shared->ddl_worker;
} }
void Context::setZooKeeper(zkutil::ZooKeeperPtr zookeeper)
{
std::lock_guard<std::mutex> lock(shared->zookeeper_mutex);
if (shared->zookeeper)
throw Exception("ZooKeeper client has already been set.", ErrorCodes::LOGICAL_ERROR);
shared->zookeeper = std::move(zookeeper);
}
zkutil::ZooKeeperPtr Context::getZooKeeper() const zkutil::ZooKeeperPtr Context::getZooKeeper() const
{ {
std::lock_guard<std::mutex> lock(shared->zookeeper_mutex); std::lock_guard<std::mutex> lock(shared->zookeeper_mutex);
if (shared->zookeeper && shared->zookeeper->expired()) if (!shared->zookeeper)
shared->zookeeper = std::make_shared<zkutil::ZooKeeper>(getConfigRef(), "zookeeper");
else if (shared->zookeeper->expired())
shared->zookeeper = shared->zookeeper->startNewSession(); shared->zookeeper = shared->zookeeper->startNewSession();
return shared->zookeeper; return shared->zookeeper;

View File

@ -140,7 +140,6 @@ public:
/// Global application configuration settings. /// Global application configuration settings.
void setConfig(const ConfigurationPtr & config); void setConfig(const ConfigurationPtr & config);
ConfigurationPtr getConfig() const;
Poco::Util::AbstractConfiguration & getConfigRef() const; Poco::Util::AbstractConfiguration & getConfigRef() const;
/** Take the list of users, quotas and configuration profiles from this config. /** Take the list of users, quotas and configuration profiles from this config.
@ -300,7 +299,6 @@ public:
MergeList & getMergeList(); MergeList & getMergeList();
const MergeList & getMergeList() const; const MergeList & getMergeList() const;
void setZooKeeper(std::shared_ptr<zkutil::ZooKeeper> zookeeper);
/// If the current session is expired at the time of the call, synchronously creates and returns a new session with the startNewSession() call. /// If the current session is expired at the time of the call, synchronously creates and returns a new session with the startNewSession() call.
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const; std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
/// Has ready or expired ZooKeeper /// Has ready or expired ZooKeeper

View File

@ -32,6 +32,7 @@
#include <Columns/ColumnArray.h> #include <Columns/ColumnArray.h>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Lock.h> #include <Common/ZooKeeper/Lock.h>
#include <Common/isLocalAddress.h> #include <Common/isLocalAddress.h>
#include <Poco/Timestamp.h> #include <Poco/Timestamp.h>
@ -858,7 +859,7 @@ void DDLWorker::run()
} }
catch (const zkutil::KeeperException & e) catch (const zkutil::KeeperException & e)
{ {
if (!e.isHardwareError()) if (!zkutil::isHardwareError(e.code))
throw; throw;
} }
} }
@ -886,7 +887,7 @@ void DDLWorker::run()
} }
catch (zkutil::KeeperException & e) catch (zkutil::KeeperException & e)
{ {
if (e.isHardwareError()) if (zkutil::isHardwareError(e.code))
{ {
LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false)); LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false));

View File

@ -1519,7 +1519,9 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
} }
else else
{ {
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(source_columns, settings); NamesAndTypesList temp_columns = source_columns;
temp_columns.insert(temp_columns.end(), aggregated_columns.begin(), aggregated_columns.end());
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(temp_columns, settings);
getRootActions(func->arguments->children.at(0), true, false, temp_actions); getRootActions(func->arguments->children.at(0), true, false, temp_actions);
Block sample_block_with_calculated_columns = temp_actions->getSampleBlock(); Block sample_block_with_calculated_columns = temp_actions->getSampleBlock();

View File

@ -687,19 +687,26 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
stream->addTableLock(table_lock); stream->addTableLock(table_lock);
}); });
/** Set the limits and quota for reading data, the speed and time of the query. /// Set the limits and quota for reading data, the speed and time of the query.
* Such restrictions are checked on the initiating server of the request, and not on remote servers.
* Because the initiating server has a summary of the execution of the request on all servers.
*/
if (to_stage == QueryProcessingStage::Complete)
{ {
IProfilingBlockInputStream::LocalLimits limits; IProfilingBlockInputStream::LocalLimits limits;
limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL;
limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode); limits.size_limits = SizeLimits(settings.max_rows_to_read, settings.max_bytes_to_read, settings.read_overflow_mode);
limits.max_execution_time = settings.max_execution_time; limits.max_execution_time = settings.max_execution_time;
limits.timeout_overflow_mode = settings.timeout_overflow_mode; limits.timeout_overflow_mode = settings.timeout_overflow_mode;
/** Quota and minimal speed restrictions are checked on the initiating server of the request, and not on remote servers,
* because the initiating server has a summary of the execution of the request on all servers.
*
* But limits on data size to read and maximum execution time are reasonable to check both on initiator and
* additionally on each remote server, because these limits are checked per block of data processed,
* and remote servers may process way more blocks of data than are received by initiator.
*/
if (to_stage == QueryProcessingStage::Complete)
{
limits.min_execution_speed = settings.min_execution_speed; limits.min_execution_speed = settings.min_execution_speed;
limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed;
}
QuotaForIntervals & quota = context.getQuota(); QuotaForIntervals & quota = context.getQuota();
@ -708,6 +715,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
if (IProfilingBlockInputStream * p_stream = dynamic_cast<IProfilingBlockInputStream *>(stream.get())) if (IProfilingBlockInputStream * p_stream = dynamic_cast<IProfilingBlockInputStream *>(stream.get()))
{ {
p_stream->setLimits(limits); p_stream->setLimits(limits);
if (to_stage == QueryProcessingStage::Complete)
p_stream->setQuota(quota); p_stream->setQuota(quota);
} }
}); });

View File

@ -17,10 +17,19 @@
#include <boost/algorithm/string.hpp> #include <boost/algorithm/string.hpp>
#include <pcg_random.hpp> #include <pcg_random.hpp>
#include <common/logger_useful.h>
#include <common/ThreadPool.h>
#include <daemon/OwnPatternFormatter.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/getFQDNOrHostName.h> #include <Common/getFQDNOrHostName.h>
#include <Common/isLocalAddress.h> #include <Common/isLocalAddress.h>
#include <Common/typeid_cast.h>
#include <Common/ClickHouseRevision.h>
#include <Common/formatReadable.h>
#include <Common/escapeForFileName.h>
#include <Client/Connection.h> #include <Client/Connection.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/Cluster.h> #include <Interpreters/Cluster.h>
@ -30,12 +39,6 @@
#include <Interpreters/InterpreterShowCreateQuery.h> #include <Interpreters/InterpreterShowCreateQuery.h>
#include <Interpreters/InterpreterDropQuery.h> #include <Interpreters/InterpreterDropQuery.h>
#include <Interpreters/InterpreterCreateQuery.h> #include <Interpreters/InterpreterCreateQuery.h>
#include <common/logger_useful.h>
#include <common/ThreadPool.h>
#include <Common/typeid_cast.h>
#include <Common/ClickHouseRevision.h>
#include <Common/escapeForFileName.h>
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
@ -61,8 +64,6 @@
#include <Storages/StorageDistributed.h> #include <Storages/StorageDistributed.h>
#include <Databases/DatabaseMemory.h> #include <Databases/DatabaseMemory.h>
#include <Server/StatusFile.h> #include <Server/StatusFile.h>
#include <Common/formatReadable.h>
#include <daemon/OwnPatternFormatter.h>
namespace DB namespace DB
@ -715,13 +716,11 @@ class ClusterCopier
{ {
public: public:
ClusterCopier(const ConfigurationPtr & zookeeper_config_, ClusterCopier(const String & task_path_,
const String & task_path_,
const String & host_id_, const String & host_id_,
const String & proxy_database_name_, const String & proxy_database_name_,
Context & context_) Context & context_)
: :
zookeeper_config(zookeeper_config_),
task_zookeeper_path(task_path_), task_zookeeper_path(task_path_),
host_id(host_id_), host_id(host_id_),
working_database_name(proxy_database_name_), working_database_name(proxy_database_name_),
@ -732,7 +731,7 @@ public:
void init() void init()
{ {
auto zookeeper = getZooKeeper(); auto zookeeper = context.getZooKeeper();
task_description_watch_callback = [this] (const ZooKeeperImpl::ZooKeeper::WatchResponse &) task_description_watch_callback = [this] (const ZooKeeperImpl::ZooKeeper::WatchResponse &)
{ {
@ -762,8 +761,8 @@ public:
/// Do not initialize tables, will make deferred initialization in process() /// Do not initialize tables, will make deferred initialization in process()
getZooKeeper()->createAncestors(getWorkersPathVersion() + "/"); zookeeper->createAncestors(getWorkersPathVersion() + "/");
getZooKeeper()->createAncestors(getWorkersPath() + "/"); zookeeper->createAncestors(getWorkersPath() + "/");
} }
template <typename T> template <typename T>
@ -890,7 +889,7 @@ public:
void reloadTaskDescription() void reloadTaskDescription()
{ {
auto zookeeper = getZooKeeper(); auto zookeeper = context.getZooKeeper();
task_description_watch_zookeeper = zookeeper; task_description_watch_zookeeper = zookeeper;
String task_config_str; String task_config_str;
@ -1087,7 +1086,7 @@ protected:
{ {
LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully"); LOG_DEBUG(log, "Check that all shards processed partition " << partition_name << " successfully");
auto zookeeper = getZooKeeper(); auto zookeeper = context.getZooKeeper();
Strings status_paths; Strings status_paths;
for (auto & shard : shards_with_partition) for (auto & shard : shards_with_partition)
@ -1213,7 +1212,7 @@ protected:
{ {
cleaner_holder = zkutil::EphemeralNodeHolder::create(dirt_cleaner_path, *zookeeper, host_id); cleaner_holder = zkutil::EphemeralNodeHolder::create(dirt_cleaner_path, *zookeeper, host_id);
} }
catch (zkutil::KeeperException & e) catch (const zkutil::KeeperException & e)
{ {
if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS)
{ {
@ -1459,7 +1458,7 @@ protected:
TaskTable & task_table = task_shard.task_table; TaskTable & task_table = task_shard.task_table;
ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name);
auto zookeeper = getZooKeeper(); auto zookeeper = context.getZooKeeper();
String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath();
String current_task_is_active_path = task_partition.getActiveWorkerPath(); String current_task_is_active_path = task_partition.getActiveWorkerPath();
@ -1693,7 +1692,7 @@ protected:
status = future_is_dirty_checker->get(); status = future_is_dirty_checker->get();
future_is_dirty_checker.reset(); future_is_dirty_checker.reset();
} }
catch (zkutil::KeeperException & e) catch (const zkutil::KeeperException & e)
{ {
future_is_dirty_checker.reset(); future_is_dirty_checker.reset();
throw; throw;
@ -1995,21 +1994,7 @@ protected:
return successful_shards; return successful_shards;
} }
zkutil::ZooKeeperPtr getZooKeeper()
{
auto zookeeper = context.getZooKeeper();
if (!zookeeper)
{
context.setZooKeeper(std::make_shared<zkutil::ZooKeeper>(*zookeeper_config, "zookeeper"));
zookeeper = context.getZooKeeper();
}
return zookeeper;
}
private: private:
ConfigurationPtr zookeeper_config;
String task_zookeeper_path; String task_zookeeper_path;
String task_description_path; String task_description_path;
String host_id; String host_id;
@ -2152,6 +2137,7 @@ void ClusterCopierApp::mainImpl()
auto context = std::make_unique<Context>(Context::createGlobal()); auto context = std::make_unique<Context>(Context::createGlobal());
SCOPE_EXIT(context->shutdown()); SCOPE_EXIT(context->shutdown());
context->setConfig(zookeeper_configuration);
context->setGlobalContext(*context); context->setGlobalContext(*context);
context->setApplicationType(Context::ApplicationType::LOCAL); context->setApplicationType(Context::ApplicationType::LOCAL);
context->setPath(process_path); context->setPath(process_path);
@ -2165,8 +2151,7 @@ void ClusterCopierApp::mainImpl()
context->addDatabase(default_database, std::make_shared<DatabaseMemory>(default_database)); context->addDatabase(default_database, std::make_shared<DatabaseMemory>(default_database));
context->setCurrentDatabase(default_database); context->setCurrentDatabase(default_database);
std::unique_ptr<ClusterCopier> copier(new ClusterCopier( std::unique_ptr<ClusterCopier> copier = std::make_unique<ClusterCopier>(task_path, host_id, default_database, *context);
zookeeper_configuration, task_path, host_id, default_database, *context));
copier->setSafeMode(is_safe_mode); copier->setSafeMode(is_safe_mode);
copier->setCopyFaultProbability(copy_fault_probability); copier->setCopyFaultProbability(copy_fault_probability);

View File

@ -102,12 +102,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
global_context->setGlobalContext(*global_context); global_context->setGlobalContext(*global_context);
global_context->setApplicationType(Context::ApplicationType::SERVER); global_context->setApplicationType(Context::ApplicationType::SERVER);
bool has_zookeeper = false; bool has_zookeeper = config().has("zookeeper");
if (config().has("zookeeper"))
{
global_context->setZooKeeper(std::make_shared<zkutil::ZooKeeper>(config(), "zookeeper"));
has_zookeeper = true;
}
zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); }); zkutil::ZooKeeperNodeCache main_config_zk_node_cache([&] { return global_context->getZooKeeper(); });
if (loaded_config.has_zk_includes) if (loaded_config.has_zk_includes)

View File

@ -0,0 +1,8 @@
<yandex>
<zookeeper>
<node>
<host>localhost</host>
<port>2181</port>
</node>
</zookeeper>
</yandex>

View File

@ -2189,18 +2189,17 @@ bool MergeTreeData::isPrimaryKeyColumnPossiblyWrappedInFunctions(const ASTPtr &
bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const bool MergeTreeData::mayBenefitFromIndexForIn(const ASTPtr & left_in_operand) const
{ {
/// Make sure that the left side of the IN operator is part of the primary key. /// Make sure that the left side of the IN operator contain part of the primary key.
/// If there is a tuple on the left side of the IN operator, each item of the tuple must be part of the primary key. /// If there is a tuple on the left side of the IN operator, at least one item of the tuple must be part of the primary key (probably wrapped by a chain of some acceptable functions).
const ASTFunction * left_in_operand_tuple = typeid_cast<const ASTFunction *>(left_in_operand.get()); const ASTFunction * left_in_operand_tuple = typeid_cast<const ASTFunction *>(left_in_operand.get());
if (left_in_operand_tuple && left_in_operand_tuple->name == "tuple") if (left_in_operand_tuple && left_in_operand_tuple->name == "tuple")
{ {
for (const auto & item : left_in_operand_tuple->arguments->children) for (const auto & item : left_in_operand_tuple->arguments->children)
if (!isPrimaryKeyColumnPossiblyWrappedInFunctions(item)) if (isPrimaryKeyColumnPossiblyWrappedInFunctions(item))
return true;
/// The tuple itself may be part of the primary key, so check that as a last resort. /// The tuple itself may be part of the primary key, so check that as a last resort.
return isPrimaryKeyColumnPossiblyWrappedInFunctions(left_in_operand); return isPrimaryKeyColumnPossiblyWrappedInFunctions(left_in_operand);
/// tuple() is invalid but can still be found here since this method may be called before the arguments are validated.
return !left_in_operand_tuple->arguments->children.empty();
} }
else else
{ {

View File

@ -1,5 +1,6 @@
#include <memory> #include <memory>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Interpreters/InterpreterAlterQuery.h> #include <Interpreters/InterpreterAlterQuery.h>
#include <Storages/ColumnsDescription.h> #include <Storages/ColumnsDescription.h>
#include <Storages/StorageReplicatedMergeTree.h> #include <Storages/StorageReplicatedMergeTree.h>

View File

@ -5,8 +5,10 @@
#include <Interpreters/PartLog.h> #include <Interpreters/PartLog.h>
#include <DataStreams/IBlockOutputStream.h> #include <DataStreams/IBlockOutputStream.h>
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <IO/Operators.h> #include <IO/Operators.h>
namespace ProfileEvents namespace ProfileEvents
{ {
extern const Event DuplicatedInsertedBlocks; extern const Event DuplicatedInsertedBlocks;
@ -25,6 +27,7 @@ namespace ErrorCodes
extern const int READONLY; extern const int READONLY;
extern const int UNKNOWN_STATUS_OF_INSERT; extern const int UNKNOWN_STATUS_OF_INSERT;
extern const int INSERT_WAS_DEDUPLICATED; extern const int INSERT_WAS_DEDUPLICATED;
extern const int KEEPER_EXCEPTION;
} }

View File

@ -4,6 +4,7 @@
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h> #include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
#include <Storages/MergeTree/ReplicatedMergeTreeAddress.h> #include <Storages/MergeTree/ReplicatedMergeTreeAddress.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/randomSeed.h> #include <Common/randomSeed.h>

View File

@ -1,17 +1,13 @@
#pragma once #pragma once
#include <ext/shared_ptr_helper.h> #include <ext/shared_ptr_helper.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wunused-parameter"
#include <Poco/Data/SessionPool.h>
#pragma GCC diagnostic pop
namespace Poco
{
namespace Data
{
class SessionPool;
}
}
namespace DB namespace DB

View File

@ -23,7 +23,7 @@ void registerStorageJoin(StorageFactory & factory);
void registerStorageView(StorageFactory & factory); void registerStorageView(StorageFactory & factory);
void registerStorageMaterializedView(StorageFactory & factory); void registerStorageMaterializedView(StorageFactory & factory);
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
void registerStorageODBC(StorageFactory & factory); void registerStorageODBC(StorageFactory & factory);
#endif #endif
@ -56,7 +56,7 @@ void registerStorages()
registerStorageView(factory); registerStorageView(factory);
registerStorageMaterializedView(factory); registerStorageMaterializedView(factory);
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
registerStorageODBC(factory); registerStorageODBC(factory);
#endif #endif

View File

@ -6,7 +6,13 @@ list(REMOVE_ITEM clickhouse_table_functions_headers ITableFunction.h TableFuncti
add_library(clickhouse_table_functions ${clickhouse_table_functions_sources}) add_library(clickhouse_table_functions ${clickhouse_table_functions_sources})
target_link_libraries(clickhouse_table_functions dbms clickhouse_storages_system ${Poco_Foundation_LIBRARY}) target_link_libraries(clickhouse_table_functions dbms clickhouse_storages_system ${Poco_Foundation_LIBRARY})
if (Poco_SQLODBC_FOUND)
target_link_libraries (clickhouse_table_functions ${Poco_SQLODBC_LIBRARY})
target_include_directories (clickhouse_table_functions PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIRS})
endif ()
if (Poco_DataODBC_FOUND) if (Poco_DataODBC_FOUND)
target_link_libraries (clickhouse_table_functions ${Poco_DataODBC_LIBRARY}) target_link_libraries (clickhouse_table_functions ${Poco_DataODBC_LIBRARY})
target_include_directories (clickhouse_table_functions PRIVATE ${ODBC_INCLUDE_DIRECTORIES}) target_include_directories (clickhouse_table_functions PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIRS})
endif () endif ()

View File

@ -1,6 +1,6 @@
#include <TableFunctions/TableFunctionODBC.h> #include <TableFunctions/TableFunctionODBC.h>
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
#include <type_traits> #include <type_traits>
#include <ext/scope_guard.h> #include <ext/scope_guard.h>

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <Common/config.h> #include <Common/config.h>
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
#include <TableFunctions/ITableFunction.h> #include <TableFunctions/ITableFunction.h>

View File

@ -11,7 +11,7 @@ void registerTableFunctionRemote(TableFunctionFactory & factory);
void registerTableFunctionShardByHash(TableFunctionFactory & factory); void registerTableFunctionShardByHash(TableFunctionFactory & factory);
void registerTableFunctionNumbers(TableFunctionFactory & factory); void registerTableFunctionNumbers(TableFunctionFactory & factory);
void registerTableFunctionCatBoostPool(TableFunctionFactory & factory); void registerTableFunctionCatBoostPool(TableFunctionFactory & factory);
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
void registerTableFunctionODBC(TableFunctionFactory & factory); void registerTableFunctionODBC(TableFunctionFactory & factory);
#endif #endif
@ -30,7 +30,7 @@ void registerTableFunctions()
registerTableFunctionNumbers(factory); registerTableFunctionNumbers(factory);
registerTableFunctionCatBoostPool(factory); registerTableFunctionCatBoostPool(factory);
#if Poco_DataODBC_FOUND #if Poco_SQLODBC_FOUND || Poco_DataODBC_FOUND
registerTableFunctionODBC(factory); registerTableFunctionODBC(factory);
#endif #endif

View File

@ -0,0 +1 @@
Still alive

View File

@ -0,0 +1 @@
Still alive

View File

@ -0,0 +1,13 @@
all
1 [1]
2 [2]
key, arrayJoin(arr) in (1, 1)
1 1
key, arrayJoin(arr) in ((1, 1), (2, 2))
1 1
2 2
(key, left array join arr) in (1, 1)
1
(key, left array join arr) in ((1, 1), (2, 2))
1
2

View File

@ -0,0 +1,16 @@
create database if not exists test;
drop table if exists test.tab;
create table test.tab (key UInt64, arr Array(UInt64)) Engine = MergeTree order by key;
insert into test.tab values (1, [1]);
insert into test.tab values (2, [2]);
select 'all';
select * from test.tab order by key;
select 'key, arrayJoin(arr) in (1, 1)';
select key, arrayJoin(arr) as val from test.tab where (key, val) in (1, 1);
select 'key, arrayJoin(arr) in ((1, 1), (2, 2))';
select key, arrayJoin(arr) as val from test.tab where (key, val) in ((1, 1), (2, 2)) order by key;
select '(key, left array join arr) in (1, 1)';
select key from test.tab left array join arr as val where (key, val) in (1, 1);
select '(key, left array join arr) in ((1, 1), (2, 2))';
select key from test.tab left array join arr as val where (key, val) in ((1, 1), (2, 2)) order by key;

View File

@ -0,0 +1,10 @@
0
1
2
3
4
5
6
7
8
9

View File

@ -0,0 +1,2 @@
SET max_execution_time = 1, timeout_overflow_mode = 'break';
SELECT DISTINCT * FROM remote('127.0.0.{2,3}', system.numbers) WHERE number < 10;

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (1.1.54371) unstable; urgency=low clickhouse (1.1.54372) unstable; urgency=low
* Modified source code * Modified source code
-- <robot-metrika-test@yandex-team.ru> Fri, 23 Mar 2018 01:11:42 +0300 -- <robot-metrika-test@yandex-team.ru> Mon, 02 Apr 2018 22:13:54 +0300

5
debian/control vendored
View File

@ -50,11 +50,14 @@ Description: Server binary for clickhouse
. .
This package provides clickhouse common configuration files This package provides clickhouse common configuration files
Package: clickhouse-common-dbg Package: clickhouse-common-static-dbg
Architecture: any Architecture: any
Section: debug Section: debug
Priority: extra Priority: extra
Depends: ${misc:Depends}, clickhouse-common-static (= ${binary:Version}) Depends: ${misc:Depends}, clickhouse-common-static (= ${binary:Version})
Replaces: clickhouse-common-dbg
Provides: clickhouse-common-dbg
Conflicts: clickhouse-common-dbg
Description: debugging symbols for clickhouse-common-static Description: debugging symbols for clickhouse-common-static
This package contains the debugging symbols for clickhouse-common. This package contains the debugging symbols for clickhouse-common.

2
debian/rules vendored
View File

@ -70,7 +70,7 @@ override_dh_clean:
dh_clean dh_clean
override_dh_strip: override_dh_strip:
dh_strip -pclickhouse-common-static --dbg-package=clickhouse-common-dbg dh_strip -pclickhouse-common-static --dbg-package=clickhouse-common-static-dbg
override_dh_install: override_dh_install:
# Making docs # Making docs

View File

@ -31,7 +31,7 @@ Then run:
```bash ```bash
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional
sudo apt-get update sudo apt-get update
sudo apt-get install clickhouse-client clickhouse-server sudo apt-get install clickhouse-client clickhouse-server-common
``` ```
You can also download and install packages manually from here: You can also download and install packages manually from here:

View File

@ -33,6 +33,9 @@ There are libraries for working with ClickHouse for:
- [ClickHouse-Net](https://github.com/killwort/ClickHouse-Net) - [ClickHouse-Net](https://github.com/killwort/ClickHouse-Net)
- C++ - C++
- [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp/) - [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp/)
- Elixir
- [clickhousex](https://github.com/appodeal/clickhousex/)
- [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto)
We have not tested these libraries. They are listed in random order. We have not tested these libraries. They are listed in random order.

View File

@ -31,7 +31,7 @@ deb http://repo.yandex.ru/clickhouse/deb/stable/ main/
```bash ```bash
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional
sudo apt-get update sudo apt-get update
sudo apt-get install clickhouse-client clickhouse-server sudo apt-get install clickhouse-client clickhouse-server-common
``` ```
Также можно скачать и установить пакеты вручную, отсюда: <https://repo.yandex.ru/clickhouse/deb/stable/main/>. Также можно скачать и установить пакеты вручную, отсюда: <https://repo.yandex.ru/clickhouse/deb/stable/main/>.

View File

@ -33,5 +33,8 @@
- [ClickHouse-Net](https://github.com/killwort/ClickHouse-Net) - [ClickHouse-Net](https://github.com/killwort/ClickHouse-Net)
- C++ - C++
- [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp/) - [clickhouse-cpp](https://github.com/artpaul/clickhouse-cpp/)
- Elixir
- [clickhousex](https://github.com/appodeal/clickhousex/)
- [clickhouse_ecto](https://github.com/appodeal/clickhouse_ecto)
Библиотеки не тестировались нами. Порядок перечисления произвольный. Библиотеки не тестировались нами. Порядок перечисления произвольный.

View File

@ -38,7 +38,7 @@ MergeTree(EventDate, intHash32(UserID), (CounterID, EventDate, intHash32(UserID)
Для столбцов также пишутся «засечки» каждую index_granularity строку, чтобы данные можно было читать в определённом диапазоне. Для столбцов также пишутся «засечки» каждую index_granularity строку, чтобы данные можно было читать в определённом диапазоне.
При чтении из таблицы, запрос SELECT анализируется на предмет того, можно ли использовать индексы. При чтении из таблицы, запрос SELECT анализируется на предмет того, можно ли использовать индексы.
Индекс может использоваться, если в секции WHERE/PREWHERE, в качестве одного из элементов конъюнкции, или целиком, есть выражение, представляющее операции сравнения на равенства, неравенства, а также IN над столбцами, входящими в первичный ключ / дату, а также логические связки над ними. Индекс может использоваться, если в секции WHERE/PREWHERE, в качестве одного из элементов конъюнкции, или целиком, есть выражение, представляющее операции сравнения на равенства, неравенства, а также IN или LIKE с фиксированным префиксом, над столбцами или выражениями, входящими в первичный ключ или ключ партиционирования, либо над некоторыми частистично монотонными функциями от этих столбцов, а также логические связки над такими выражениями.
Таким образом, обеспечивается возможность быстро выполнять запросы по одному или многим диапазонам первичного ключа. Например, в указанном примере будут быстро работать запросы для конкретного счётчика; для конкретного счётчика и диапазона дат; для конкретного счётчика и даты, для нескольких счётчиков и диапазона дат и т. п. Таким образом, обеспечивается возможность быстро выполнять запросы по одному или многим диапазонам первичного ключа. Например, в указанном примере будут быстро работать запросы для конкретного счётчика; для конкретного счётчика и диапазона дат; для конкретного счётчика и даты, для нескольких счётчиков и диапазона дат и т. п.

View File

@ -206,13 +206,13 @@ int main(int argc, char ** argv)
} }
} }
catch (zkutil::KeeperException & e) catch (const zkutil::KeeperException & e)
{ {
std::cerr << "KeeperException: " << e.displayText() << std::endl; std::cerr << "KeeperException: " << e.displayText() << std::endl;
} }
} }
} }
catch (zkutil::KeeperException & e) catch (const zkutil::KeeperException & e)
{ {
std::cerr << "KeeperException: " << e.displayText() << std::endl; std::cerr << "KeeperException: " << e.displayText() << std::endl;
return 1; return 1;

View File

@ -3,6 +3,7 @@
#include <boost/program_options.hpp> #include <boost/program_options.hpp>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/KeeperException.h>
int main(int argc, char ** argv) int main(int argc, char ** argv)

View File

@ -439,7 +439,7 @@ Then run:
%% %%
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional
sudo apt-get update sudo apt-get update
sudo apt-get install -y clickhouse-client clickhouse-server sudo apt-get install clickhouse-client clickhouse-server-common
%% %%
You can also download and install packages manually from here: You can also download and install packages manually from here:
@ -709,7 +709,7 @@ echo &#39;DROP TABLE t&#39; | POST &#39;http://localhost:8123/&#39;
For successful requests that don&#39;t return a data table, an empty response body is returned. For successful requests that don&#39;t return a data table, an empty response body is returned.
You can use compression when transmitting data. The compressed data has a non-standard format, and you will need to use a special clickhouse-compressor program to work with it (%%sudo apt-get install clickhouse-utils%%). You can use compression when transmitting data. The compressed data has a non-standard format, and you will need to use a special compressor program to work with it (%%sudo apt-get install clickhouse-compressor%%).
If you specified &#39;compress=1&#39; in the URL, the server will compress the data it sends you. If you specified &#39;compress=1&#39; in the URL, the server will compress the data it sends you.
If you specified &#39;decompress=1&#39; in the URL, the server will decompress the same data that you pass in the POST method. If you specified &#39;decompress=1&#39; in the URL, the server will decompress the same data that you pass in the POST method.

View File

@ -449,7 +449,7 @@ deb http://repo.yandex.ru/clickhouse/trusty stable main
%% %%
sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional
sudo apt-get update sudo apt-get update
sudo apt-get install -y clickhouse-client clickhouse-server sudo apt-get install clickhouse-client clickhouse-server-common
%% %%
Также можно скачать и установить пакеты вручную, отсюда: Также можно скачать и установить пакеты вручную, отсюда:
@ -725,7 +725,7 @@ echo 'DROP TABLE t' | POST 'http://localhost:8123/'
Для запросов, которые не возвращают таблицу с данными, в случае успеха, выдаётся пустое тело ответа. Для запросов, которые не возвращают таблицу с данными, в случае успеха, выдаётся пустое тело ответа.
Вы можете использовать сжатие при передаче данных. Формат сжатых данных нестандартный, и вам придётся использовать для работы с ним специальную программу clickhouse-compressor (%%sudo apt-get install clickhouse-utils%%). Вы можете использовать сжатие при передаче данных. Формат сжатых данных нестандартный, и вам придётся использовать для работы с ним специальную программу compressor (%%sudo apt-get install clickhouse-compressor%%).
Если вы указали в URL compress=1, то сервер будет сжимать отправляемые вам данные. Если вы указали в URL compress=1, то сервер будет сжимать отправляемые вам данные.
Если вы указали в URL decompress=1, то сервер будет разжимать те данные, которые вы передаёте ему POST-ом. Если вы указали в URL decompress=1, то сервер будет разжимать те данные, которые вы передаёте ему POST-ом.

View File

@ -393,7 +393,7 @@ sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional
sudo apt-add-repository "deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" sudo apt-add-repository "deb http://repo.yandex.ru/clickhouse/deb/stable/ main/"
sudo apt-get update sudo apt-get update
sudo apt-get install -y clickhouse-server clickhouse-client sudo apt-get install clickhouse-server-common clickhouse-client -y
sudo service clickhouse-server start sudo service clickhouse-server start
clickhouse-client clickhouse-client

View File

@ -51,7 +51,7 @@
<p><b>clickhouse-client</b> package contains <a <p><b>clickhouse-client</b> package contains <a
href="docs/en/interfaces/cli/">clickhouse-client</a> application — href="docs/en/interfaces/cli/">clickhouse-client</a> application —
interactive ClickHouse client. <b>clickhouse-common</b> contains a clickhouse-server binary file. <b>clickhouse-server</b> interactive ClickHouse client. <b>clickhouse-server-base</b> contains a clickhouse-server binary file. <b>clickhouse-server-common</b>
— contains config files for the clickhouse-server.</p> — contains config files for the clickhouse-server.</p>
<p>Server config files are located in /etc/clickhouse-server/. Before getting to work please notice the <b>path</b> <p>Server config files are located in /etc/clickhouse-server/. Before getting to work please notice the <b>path</b>